From 5b2b07641f71f7c057f3156a964f56c95cf17229 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 25 May 2017 10:08:54 +0200 Subject: [PATCH 1/2] [FLINK-6716] Suppress load errors in checkstyle JavadocMethod --- tools/maven/strict-checkstyle.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tools/maven/strict-checkstyle.xml b/tools/maven/strict-checkstyle.xml index 931ec1bd290c5..0d50e3577940b 100644 --- a/tools/maven/strict-checkstyle.xml +++ b/tools/maven/strict-checkstyle.xml @@ -262,6 +262,7 @@ This file is based on the checkstyle file of Apache Beam. + From f80d1c88162a6567c51c040987701114c8cc2e58 Mon Sep 17 00:00:00 2001 From: dawidwys Date: Wed, 24 May 2017 22:27:06 +0200 Subject: [PATCH 2/2] [FLINK-6137] Activate strict checkstyle for flink-cep --- .../flink/cep/scala/pattern/Pattern.scala | 4 +- flink-libraries/flink-cep/pom.xml | 36 + .../main/java/org/apache/flink/cep/CEP.java | 2 +- .../cep/NonDuplicatingTypeSerializer.java | 12 +- .../org/apache/flink/cep/PatternStream.java | 7 +- .../org/apache/flink/cep/nfa/DeweyNumber.java | 6 +- .../java/org/apache/flink/cep/nfa/NFA.java | 52 +- .../apache/flink/cep/nfa/SharedBuffer.java | 27 +- .../java/org/apache/flink/cep/nfa/State.java | 12 +- .../apache/flink/cep/nfa/StateTransition.java | 4 + .../flink/cep/nfa/compiler/NFACompiler.java | 33 +- .../AbstractKeyedCEPPatternOperator.java | 9 +- .../flink/cep/operator/CEPOperatorUtils.java | 8 +- .../cep/operator/StreamRecordComparator.java | 2 +- .../flink/cep/pattern/AndFilterFunction.java | 7 +- .../flink/cep/pattern/OrFilterFunction.java | 7 +- .../org/apache/flink/cep/pattern/Pattern.java | 22 +- .../apache/flink/cep/pattern/Quantifier.java | 23 +- .../cep/pattern/SubtypeFilterFunction.java | 7 +- .../pattern/conditions/BooleanConditions.java | 1 + .../conditions/IterativeCondition.java | 4 +- .../java/org/apache/flink/cep/CEPITCase.java | 20 +- .../test/java/org/apache/flink/cep/Event.java | 3 + .../java/org/apache/flink/cep/SubEvent.java | 3 + .../apache/flink/cep/nfa/DeweyNumberTest.java | 4 + .../cep/nfa/IterativeConditionsITCase.java | 421 ++++ .../org/apache/flink/cep/nfa/NFAITCase.java | 1948 +---------------- .../org/apache/flink/cep/nfa/NFATest.java | 8 +- .../flink/cep/nfa/NFATestUtilities.java | 115 + .../flink/cep/nfa/NotPatternITCase.java | 1036 +++++++++ .../flink/cep/nfa/SameElementITCase.java | 407 ++++ .../flink/cep/nfa/SharedBufferTest.java | 10 +- .../cep/nfa/compiler/NFACompilerTest.java | 6 +- .../cep/operator/CEPFrom12MigrationTest.java | 3 +- .../cep/operator/CEPMigration11to13Test.java | 6 +- .../flink/cep/operator/CEPOperatorTest.java | 26 +- .../flink/cep/operator/CEPRescalingTest.java | 6 +- .../apache/flink/cep/pattern/PatternTest.java | 14 +- 38 files changed, 2298 insertions(+), 2023 deletions(-) create mode 100644 flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java create mode 100644 flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java create mode 100644 flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java create mode 100644 flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java diff --git a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala index fe7a30ccdd45e..fdc9f0e36c473 100644 --- a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala +++ b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala @@ -271,7 +271,7 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) { * {{{A1 A2 B}}} appears, this will generate patterns: * {{{A1 B}}} and {{{A1 A2 B}}}. See also {{{allowCombinations()}}}. * - * @return The same pattern with a [[Quantifier.ONE_OR_MORE()]] quantifier applied. + * @return The same pattern with a [[Quantifier.oneOrMore()]] quantifier applied. * @throws MalformedPatternException if the quantifier is not applicable to this pattern. */ def oneOrMore: Pattern[T, F] = { @@ -292,7 +292,7 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) { } /** - * Applicable only to [[Quantifier.ONE_OR_MORE()]] and [[Quantifier.TIMES()]] patterns, + * Applicable only to [[Quantifier.oneOrMore()]] and [[Quantifier.times()]] patterns, * this option allows more flexibility to the matching events. * * If {{{allowCombinations()}}} is not applied for a diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml index 35045c03d9c19..6622b3623ff8d 100644 --- a/flink-libraries/flink-cep/pom.xml +++ b/flink-libraries/flink-cep/pom.xml @@ -106,6 +106,42 @@ under the License. + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + + com.puppycrawl.tools + checkstyle + 6.19 + + + + /tools/maven/strict-checkstyle.xml + /tools/maven/suppressions.xml + true + true + true + + + + + test-compile + + check + + + + diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/CEP.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/CEP.java index 9ce9f77d163e2..0ef9c21d0d9c0 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/CEP.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/CEP.java @@ -24,7 +24,7 @@ /** * Utility class for complex event processing. * - * Methods which transform a {@link DataStream} into a {@link PatternStream} to do CEP. + *

Methods which transform a {@link DataStream} into a {@link PatternStream} to do CEP. */ public class CEP { /** diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java index 9e22fc2706d43..f9e13fe0bcfa5 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java @@ -35,7 +35,7 @@ * serialized once. If the same object shall be serialized again, then a reference handle is * written instead. * - * Avoiding duplication is achieved by keeping an internal identity hash map. This map contains + *

Avoiding duplication is achieved by keeping an internal identity hash map. This map contains * all serialized objects. To make the serializer work it is important that the same serializer * is used for a coherent serialization run. After the serialization has stopped, the identity * hash map should be cleared. @@ -107,8 +107,8 @@ public int getLength() { /** * Serializes the given record. - *

- * First a boolean indicating whether a reference handle (true) or the object (false) is + * + *

First a boolean indicating whether a reference handle (true) or the object (false) is * written. Then, either the reference handle or the object is written. * * @param record The record to serialize. @@ -128,8 +128,8 @@ public void serialize(T record, DataOutputView target) throws IOException { /** * Deserializes an object from the input view. - *

- * First it reads a boolean indicating whether a reference handle or a serialized object + * + *

First it reads a boolean indicating whether a reference handle or a serialized object * follows. * * @param source The input view from which to read the data. @@ -172,7 +172,7 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean equals(Object obj) { if (obj instanceof NonDuplicatingTypeSerializer) { @SuppressWarnings("unchecked") - NonDuplicatingTypeSerializer other = (NonDuplicatingTypeSerializer)obj; + NonDuplicatingTypeSerializer other = (NonDuplicatingTypeSerializer) obj; return (other.canEqual(this) && typeSerializer.equals(other.typeSerializer)); } else { diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java index 55446896ace91..71614cf81e586 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java @@ -40,7 +40,7 @@ * {@link org.apache.flink.cep.nfa.NFA}. In order to process the detected sequences, the user * has to specify a {@link PatternSelectFunction} or a {@link PatternFlatSelectFunction}. * - * Additionally it allows to handle partially matched event patterns which have timed out. For this + *

Additionally it allows to handle partially matched event patterns which have timed out. For this * the user has to specify a {@link PatternTimeoutFunction} or a {@link PatternFlatTimeoutFunction}. * * @param Type of the events @@ -119,7 +119,7 @@ public SingleOutputStreamOperator select(final PatternSelectFunctionApplies a timeout function to a partial pattern sequence which has timed out. For each * partial pattern sequence the provided {@link PatternTimeoutFunction} is called. The pattern * timeout function can produce exactly one resulting element. * @@ -220,7 +220,7 @@ public SingleOutputStreamOperator flatSelect(final PatternFlatSelectFunct * the provided {@link PatternFlatSelectFunction} is called. The pattern flat select function * can produce an arbitrary number of resulting elements. * - * Applies a timeout function to a partial pattern sequence which has timed out. For each + *

Applies a timeout function to a partial pattern sequence which has timed out. For each * partial pattern sequence the provided {@link PatternFlatTimeoutFunction} is called. The * pattern timeout function can produce an arbitrary number of resulting elements. * @@ -397,7 +397,6 @@ public PatternFlatSelectMapper(PatternFlatSelectFunction patternFlatSelect this.patternFlatSelectFunction = patternFlatSelectFunction; } - @Override public void flatMap(Map> value, Collector out) throws Exception { patternFlatSelectFunction.flatSelect(value, out); diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java index 3827956b2b52b..f066141538c3c 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java @@ -31,7 +31,7 @@ /** * Versioning scheme which allows to retrieve dependencies between different versions. * - * A dewey number consists of a sequence of digits d1.d2.d3. ... .dn. A dewey number v is compatible + *

A dewey number consists of a sequence of digits d1.d2.d3. ... .dn. A dewey number v is compatible * to v' iff v contains v' as a prefix or if both dewey number differ only in the last digit and * the last digit of v is greater than v'. * @@ -58,7 +58,7 @@ private DeweyNumber(int[] deweyNumber) { /** * Checks whether this dewey number is compatible to the other dewey number. * - * True iff this contains other as a prefix or iff they differ only in the last digit whereas + *

True iff this contains other as a prefix or iff they differ only in the last digit whereas * the last digit of this is greater than the last digit of other. * * @param other The other dewey number to check compatibility against @@ -106,7 +106,7 @@ public DeweyNumber increase() { /** * Creates a new dewey number from this such that its last digit is increased by the supplied - * number + * number. * * @param times how many times to increase the Dewey number * @return A new dewey number derived from this whose last digit is increased by given number diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java index 2be09adabbdb5..f4389152d9693 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java @@ -18,9 +18,6 @@ package org.apache.flink.cep.nfa; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterators; -import com.google.common.collect.ListMultimap; import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; @@ -29,9 +26,9 @@ import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer; import org.apache.flink.api.common.typeutils.base.EnumSerializer; -import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; import org.apache.flink.cep.NonDuplicatingTypeSerializer; @@ -44,7 +41,12 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.util.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterators; +import com.google.common.collect.ListMultimap; + import javax.annotation.Nullable; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -68,27 +70,26 @@ /** * Non-deterministic finite automaton implementation. - *

- * The {@link org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator CEP operator} + * + *

The {@link org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator CEP operator} * keeps one NFA per key, for keyed input streams, and a single global NFA for non-keyed ones. * When an event gets processed, it updates the NFA's internal state machine. - *

- * An event that belongs to a partially matched sequence is kept in an internal + * + *

An event that belongs to a partially matched sequence is kept in an internal * {@link SharedBuffer buffer}, which is a memory-optimized data-structure exactly for * this purpose. Events in the buffer are removed when all the matched sequences that * contain them are: *

    - *
  1. emitted (success)
  2. - *
  3. discarded (patterns containing NOT)
  4. - *
  5. timed-out (windowed patterns)
  6. + *
  7. emitted (success)
  8. + *
  9. discarded (patterns containing NOT)
  10. + *
  11. timed-out (windowed patterns)
  12. *
* - * The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams". - * - * @see - * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf + *

The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams". * * @param Type of the processed events + * @see + * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf */ public class NFA implements Serializable { @@ -289,7 +290,7 @@ public Tuple2>>, Collection 0L) { + if (windowTime > 0L) { long pruningTimestamp = timestamp - windowTime; if (pruningTimestamp < timestamp) { @@ -360,6 +361,7 @@ void add(StateTransition edge) { int getTotalIgnoreBranches() { return totalIgnoreBranches; } + int getTotalTakeBranches() { return totalTakeBranches; } @@ -672,7 +674,7 @@ Map> extractCurrentMatches(final ComputationState computation ////////////////////// Fault-Tolerance / Migration ////////////////////// - private final static String BEGINNING_STATE_NAME = "$beginningState$"; + private static final String BEGINNING_STATE_NAME = "$beginningState$"; private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { ois.defaultReadObject(); @@ -747,7 +749,6 @@ public boolean apply(@Nullable StateTransition input) { } }).getTargetState().getName(); - final State previousState = convertedStates.get(previousName); computationStates.add(ComputationState.createState( @@ -783,16 +784,16 @@ public boolean apply(@Nullable State input) { @SuppressWarnings("unchecked") private ComputationState readComputationState(ObjectInputStream ois) throws IOException, ClassNotFoundException { - final State state = (State)ois.readObject(); + final State state = (State) ois.readObject(); State previousState; try { - previousState = (State)ois.readObject(); + previousState = (State) ois.readObject(); } catch (OptionalDataException e) { previousState = null; } final long timestamp = ois.readLong(); - final DeweyNumber version = (DeweyNumber)ois.readObject(); + final DeweyNumber version = (DeweyNumber) ois.readObject(); final long startTimestamp = ois.readLong(); final boolean hasEvent = ois.readBoolean(); @@ -915,7 +916,7 @@ public void serialize(NFA record, DataOutputView target) throws IOException { serializeStates(record.states, target); target.writeLong(record.windowTime); target.writeBoolean(record.handleTimeout); - + sharedBufferSerializer.serialize(record.eventSharedBuffer, target); target.writeInt(record.computationStates.size()); @@ -948,10 +949,10 @@ public NFA deserialize(DataInputView source) throws IOException { Set> states = deserializeStates(source); long windowTime = source.readLong(); boolean handleTimeout = source.readBoolean(); - + NFA nfa = new NFA<>(eventSerializer, windowTime, handleTimeout); nfa.states = states; - + nfa.eventSharedBuffer = sharedBufferSerializer.deserialize(source); Queue> computationStates = new LinkedList<>(); @@ -1132,7 +1133,6 @@ private Set> deserializeStates(DataInputView in) throws IOException { TypeSerializer stateTypeSerializer = new EnumSerializer<>(State.StateType.class); TypeSerializer actionSerializer = new EnumSerializer<>(StateTransitionAction.class); - final int noOfStates = in.readInt(); Map> states = new HashMap<>(noOfStates); @@ -1250,7 +1250,7 @@ public NFA copy(NFA from) { ois.close(); bais.close(); return copy; - } catch (IOException|ClassNotFoundException e) { + } catch (IOException | ClassNotFoundException e) { throw new RuntimeException("Could not copy NFA.", e); } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java index 91fce1f4fdee3..a44b333040b53 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java @@ -18,9 +18,6 @@ package org.apache.flink.cep.nfa; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ListMultimap; -import org.apache.commons.lang3.StringUtils; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; @@ -37,6 +34,10 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.util.Preconditions; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; +import org.apache.commons.lang3.StringUtils; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -57,15 +58,15 @@ /** * A shared buffer implementation which stores values under a key. Additionally, the values can be * versioned such that it is possible to retrieve their predecessor element in the buffer. - *

- * The idea of the implementation is to have for each key a dedicated {@link SharedBufferPage}. Each + * + *

The idea of the implementation is to have for each key a dedicated {@link SharedBufferPage}. Each * buffer page maintains a collection of the inserted values. * - * The values are wrapped in a {@link SharedBufferEntry}. The shared buffer entry allows to store + *

The values are wrapped in a {@link SharedBufferEntry}. The shared buffer entry allows to store * relations between different entries. A dewey versioning scheme allows to discriminate between * different relations (e.g. preceding element). * - * The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams". + *

The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams". * * @see * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf @@ -245,7 +246,7 @@ public Collection> extractPatterns( if (currentEntry == null) { final ListMultimap completePath = ArrayListMultimap.create(); - while(!currentPath.isEmpty()) { + while (!currentPath.isEmpty()) { final SharedBufferEntry currentPathEntry = currentPath.pop(); completePath.put(currentPathEntry.getKey(), currentPathEntry.getValueTime().getValue()); @@ -398,7 +399,7 @@ private void internalRemove(final SharedBufferEntry entry) { public String toString() { StringBuilder builder = new StringBuilder(); - for(Map.Entry> entry: pages.entrySet()){ + for (Map.Entry> entry : pages.entrySet()) { builder.append("Key: ").append(entry.getKey()).append("\n"); builder.append("Value: ").append(entry.getValue()).append("\n"); } @@ -644,7 +645,7 @@ public int hashCode() { } /** - * Versioned edge between two shared buffer entries + * Versioned edge between two shared buffer entries. * * @param Type of the key * @param Type of the value @@ -747,7 +748,7 @@ public String toString() { public boolean equals(Object obj) { if (obj instanceof ValueTimeWrapper) { @SuppressWarnings("unchecked") - ValueTimeWrapper other = (ValueTimeWrapper)obj; + ValueTimeWrapper other = (ValueTimeWrapper) obj; return timestamp == other.getTimestamp() && value.equals(other.getValue()) && counter == other.getCounter(); } else { @@ -928,7 +929,7 @@ public void serialize(SharedBuffer record, DataOutputView target) throws IOExcep // key for the current page keySerializer.serialize(page.getKey(), target); - + // number of page entries target.writeInt(page.entries.size()); @@ -1182,7 +1183,7 @@ private void readObject(ObjectInputStream ois) throws IOException, ClassNotFound for (int i = 0; i < numberPages; i++) { // key of the page @SuppressWarnings("unchecked") - K key = (K)ois.readObject(); + K key = (K) ois.readObject(); SharedBufferPage page = new SharedBufferPage<>(key); diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java index 14395b1d10f37..035674c8818f4 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java @@ -30,8 +30,8 @@ /** * Represents a state of the {@link NFA}. - *

- * Each state is identified by a name and a state type. Furthermore, it contains a collection of + * + *

Each state is identified by a name and a state type. Furthermore, it contains a collection of * state transitions. The state transitions describe under which conditions it is possible to enter * a new state. * @@ -59,7 +59,9 @@ public boolean isFinal() { return stateType == StateType.Final; } - public boolean isStart() { return stateType == StateType.Start; } + public boolean isStart() { + return stateType == StateType.Start; + } public String getName() { return name; @@ -84,7 +86,7 @@ public void addIgnore(final IterativeCondition condition) { addStateTransition(StateTransitionAction.IGNORE, this, condition); } - public void addIgnore(final State targetState,final IterativeCondition condition) { + public void addIgnore(final State targetState, final IterativeCondition condition) { addStateTransition(StateTransitionAction.IGNORE, targetState, condition); } @@ -104,7 +106,7 @@ public void addTake(final IterativeCondition condition) { public boolean equals(Object obj) { if (obj instanceof State) { @SuppressWarnings("unchecked") - State other = (State)obj; + State other = (State) obj; return name.equals(other.name) && stateType == other.stateType && diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java index c6850cc71dbdd..a583bb0a2c4b3 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java @@ -25,6 +25,10 @@ import java.io.Serializable; import java.util.Objects; +/** + * Represents a transition from one {@link State} to another. + * @param type of events that are handled by the {@link IterativeCondition} + */ public class StateTransition implements Serializable { private static final long serialVersionUID = -4825345749997891838L; diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java index 1b31485625476..8d1d366931f0e 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java @@ -18,18 +18,6 @@ package org.apache.flink.cep.nfa.compiler; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterators; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import javax.annotation.Nullable; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; @@ -46,6 +34,21 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.util.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterators; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + /** * Compiler class containing methods to compile a {@link Pattern} into a {@link NFA} or a * {@link NFAFactory}. @@ -396,7 +399,7 @@ private void addStopStateToLooping(final State loopingState) { /** * Creates a "complex" state consisting of given number of states with - * same {@link IterativeCondition} + * same {@link IterativeCondition}. * * @param sinkState the state that the created state should point to * @param times number of times the state should be copied @@ -720,8 +723,8 @@ public interface NFAFactory extends Serializable { /** * Implementation of the {@link NFAFactory} interface. - *

- * The implementation takes the input type serializer, the window time and the set of + * + *

The implementation takes the input type serializer, the window time and the set of * states and their transitions to be able to create an NFA from them. * * @param Type of the input events which are processed by the NFA diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java index af4b53e59e487..2e3aefd0dc73b 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java @@ -30,9 +30,9 @@ import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer; import org.apache.flink.api.common.typeutils.base.CollectionSerializerConfigSnapshot; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.common.typeutils.base.ListSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cep.nfa.NFA; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.core.fs.FSDataInputStream; @@ -40,13 +40,13 @@ import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.migration.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; +import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.CheckpointedRestoringOperator; import org.apache.flink.streaming.api.operators.InternalTimer; import org.apache.flink.streaming.api.operators.InternalTimerService; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.CheckpointedRestoringOperator; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Triggerable; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; @@ -337,7 +337,6 @@ public void restoreState(FSDataInputStream in) throws Exception { ) ); - if (migratingFromOldKeyedOperator) { int numberEntries = inputView.readInt(); for (int i = 0; i < numberEntries; i++) { diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java index e7b7e65ddb9f1..d00e5e9d449a5 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java @@ -28,6 +28,7 @@ import org.apache.flink.api.java.typeutils.EitherTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.cep.PatternStream; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.cep.pattern.Pattern; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -39,6 +40,9 @@ import java.util.List; import java.util.Map; +/** + * Utility methods for creating {@link PatternStream}. + */ public class CEPOperatorUtils { /** @@ -61,7 +65,7 @@ public static SingleOutputStreamOperator>> createPatt if (inputStream instanceof KeyedStream) { // We have to use the KeyedCEPPatternOperator which can deal with keyed input streams - KeyedStream keyedStream= (KeyedStream) inputStream; + KeyedStream keyedStream = (KeyedStream) inputStream; TypeSerializer keySerializer = keyedStream.getKeyType().createSerializer(keyedStream.getExecutionConfig()); @@ -122,7 +126,7 @@ public static SingleOutputStreamOperator keyedStream= (KeyedStream) inputStream; + KeyedStream keyedStream = (KeyedStream) inputStream; TypeSerializer keySerializer = keyedStream.getKeyType().createSerializer(keyedStream.getExecutionConfig()); diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/StreamRecordComparator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/StreamRecordComparator.java index b290e7b61b95f..30fbc2641d922 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/StreamRecordComparator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/StreamRecordComparator.java @@ -24,7 +24,7 @@ import java.util.Comparator; /** - * Compares two {@link StreamRecord}s based on their timestamp + * Compares two {@link StreamRecord}s based on their timestamp. * * @param Type of the value field of the StreamRecord */ diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java index a7391d5d17460..ef3071fe7e0bf 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java @@ -21,13 +21,12 @@ import org.apache.flink.api.common.functions.FilterFunction; /** - * @deprecated This is only used when migrating from an older Flink version. - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead. - * - *

A filter function which combines two filter functions with a logical and. Thus, the filter + * A filter function which combines two filter functions with a logical and. Thus, the filter * function only returns true, iff both filters return true. * * @param Type of the element to filter + * @deprecated This is only used when migrating from an older Flink version. + * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead. */ @Deprecated public class AndFilterFunction implements FilterFunction { diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java index 3620cae832ccd..d1c406ac65276 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java @@ -21,13 +21,12 @@ import org.apache.flink.api.common.functions.FilterFunction; /** - * @deprecated This is only used when migrating from an older Flink version. - * Use the {@link org.apache.flink.cep.pattern.conditions.OrCondition} instead. - * - *

A filter function which combines two filter functions with a logical or. Thus, the filter + * A filter function which combines two filter functions with a logical or. Thus, the filter * function only returns true, iff at least one of the filter functions holds true. * * @param Type of the element to filter + * @deprecated This is only used when migrating from an older Flink version. + * Use the {@link org.apache.flink.cep.pattern.conditions.OrCondition} instead. */ @Deprecated public class OrFilterFunction implements FilterFunction { diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java index 2d10b41a388d5..2676994255a50 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java @@ -30,10 +30,10 @@ /** * Base class for a pattern definition. - *

- * A pattern definition is used by {@link org.apache.flink.cep.nfa.compiler.NFACompiler} to create a {@link NFA}. * - *

{@code
+ * 

A pattern definition is used by {@link org.apache.flink.cep.nfa.compiler.NFACompiler} to create a {@link NFA}. + * + *

{@code
  * Pattern pattern = Pattern.begin("start")
  *   .next("middle").subtype(F.class)
  *   .followedBy("end").where(new MyCondition());
@@ -57,8 +57,8 @@ public class Pattern {
 	/** Window length in which the pattern match has to occur. */
 	private Time windowTime;
 
-	/** A quantifier for the pattern. By default set to {@link Quantifier#ONE(ConsumingStrategy)}. */
-	private Quantifier quantifier = Quantifier.ONE(ConsumingStrategy.STRICT);
+	/** A quantifier for the pattern. By default set to {@link Quantifier#one(ConsumingStrategy)}. */
+	private Quantifier quantifier = Quantifier.one(ConsumingStrategy.STRICT);
 
 	/**
 	 * Applicable to a {@code times} pattern, and holds
@@ -77,7 +77,7 @@ protected Pattern(
 			final ConsumingStrategy consumingStrategy) {
 		this.name = name;
 		this.previous = previous;
-		this.quantifier = Quantifier.ONE(consumingStrategy);
+		this.quantifier = Quantifier.one(consumingStrategy);
 	}
 
 	public Pattern getPrevious() {
@@ -295,13 +295,13 @@ public Pattern optional() {
 	 * {@code A1 A2 B} appears, this will generate patterns:
 	 * {@code A1 B} and {@code A1 A2 B}. See also {@link #allowCombinations()}.
 	 *
-	 * @return The same pattern with a {@link Quantifier#ONE_OR_MORE(ConsumingStrategy)} quantifier applied.
+	 * @return The same pattern with a {@link Quantifier#oneOrMore(ConsumingStrategy)} quantifier applied.
 	 * @throws MalformedPatternException if the quantifier is not applicable to this pattern.
 	 */
 	public Pattern oneOrMore() {
 		checkIfNoNotPattern();
 		checkIfQuantifierApplied();
-		this.quantifier = Quantifier.ONE_OR_MORE(quantifier.getConsumingStrategy());
+		this.quantifier = Quantifier.oneOrMore(quantifier.getConsumingStrategy());
 		return this;
 	}
 
@@ -317,14 +317,14 @@ public Pattern times(int times) {
 		checkIfNoNotPattern();
 		checkIfQuantifierApplied();
 		Preconditions.checkArgument(times > 0, "You should give a positive number greater than 0.");
-		this.quantifier = Quantifier.TIMES(quantifier.getConsumingStrategy());
+		this.quantifier = Quantifier.times(quantifier.getConsumingStrategy());
 		this.times = times;
 		return this;
 	}
 
 	/**
-	 * Applicable only to {@link Quantifier#ONE_OR_MORE(ConsumingStrategy)} and
-	 * {@link Quantifier#TIMES(ConsumingStrategy)} patterns, this option allows more flexibility to the matching events.
+	 * Applicable only to {@link Quantifier#oneOrMore(ConsumingStrategy)} and
+	 * {@link Quantifier#times(ConsumingStrategy)} patterns, this option allows more flexibility to the matching events.
 	 *
 	 * 

If {@code allowCombinations()} is not applied for a * pattern {@code A.oneOrMore().followedBy(B)} and a sequence of events diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java index 382c3ba9eeb92..a0c80b214fe3c 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java @@ -15,11 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.cep.pattern; import java.util.EnumSet; import java.util.Objects; +/** + * A quantifier describing the Pattern. There are three main groups of {@link Quantifier}. + * + *

    + *
  1. Single
  2. + *
  3. Looping
  4. + *
  5. Times
  6. + *
+ * + *

Each {@link Pattern} can be optional and have a {@link ConsumingStrategy}. Looping and Times have also additional + * inner consuming strategy that is applied between accepted events in the pattern + * + */ public class Quantifier { private final EnumSet properties; @@ -36,15 +50,15 @@ private Quantifier( this.consumingStrategy = consumingStrategy; } - public static Quantifier ONE(final ConsumingStrategy consumingStrategy) { + public static Quantifier one(final ConsumingStrategy consumingStrategy) { return new Quantifier(consumingStrategy, QuantifierProperty.SINGLE); } - public static Quantifier ONE_OR_MORE(final ConsumingStrategy consumingStrategy) { + public static Quantifier oneOrMore(final ConsumingStrategy consumingStrategy) { return new Quantifier(consumingStrategy, QuantifierProperty.LOOPING); } - public static Quantifier TIMES(final ConsumingStrategy consumingStrategy) { + public static Quantifier times(final ConsumingStrategy consumingStrategy) { return new Quantifier(consumingStrategy, QuantifierProperty.TIMES); } @@ -118,6 +132,9 @@ public enum QuantifierProperty { OPTIONAL } + /** + * Describes strategy which events are matched in this {@link Pattern}. See docs for more info. + */ public enum ConsumingStrategy { STRICT, SKIP_TILL_NEXT, diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java index ae48df36d627e..f5008c19a6d7c 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java @@ -21,13 +21,12 @@ import org.apache.flink.api.common.functions.FilterFunction; /** - * @deprecated This is only used when migrating from an older Flink version. - * Use the {@link org.apache.flink.cep.pattern.conditions.SubtypeCondition} instead. - * - *

A filter function which filters elements of the given type. A element if filtered out iff it + * A filter function which filters elements of the given type. A element if filtered out iff it * is not assignable to the given subtype of T. * * @param Type of the elements to be filtered + * @deprecated This is only used when migrating from an older Flink version. + * Use the {@link org.apache.flink.cep.pattern.conditions.SubtypeCondition} instead. */ @Deprecated public class SubtypeFilterFunction implements FilterFunction { diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/BooleanConditions.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/BooleanConditions.java index d67b40789080d..aea5a3bdd4398 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/BooleanConditions.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/BooleanConditions.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.cep.pattern.conditions; /** diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java index 016cdefaa1506..e7c814f1a4718 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java @@ -65,8 +65,8 @@ public abstract class IterativeCondition implements Function, Serializable { /** * The filter function that evaluates the predicate. - *

- * IMPORTANT: The system assumes that the function does not + * + *

IMPORTANT: The system assumes that the function does not * modify the elements on which the predicate is applied. Violating this assumption * can lead to incorrect results. * diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java index 9a0865941bdb7..66eeca8bd48d5 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.cep.nfa.NFA; import org.apache.flink.cep.pattern.Pattern; import org.apache.flink.cep.pattern.conditions.SimpleCondition; import org.apache.flink.core.fs.FileSystem; @@ -31,8 +32,8 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; - import org.apache.flink.types.Either; + import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -42,6 +43,9 @@ import java.util.List; import java.util.Map; +/** + * End to end tests of both CEP operators and {@link NFA}. + */ @SuppressWarnings("serial") public class CEPITCase extends StreamingMultipleProgramsTestBase { @@ -70,7 +74,8 @@ public void after() throws Exception { } /** - * Checks that a certain event sequence is recognized + * Checks that a certain event sequence is recognized. + * * @throws Exception */ @Test @@ -224,7 +229,7 @@ public void testSimplePatternEventTime() throws Exception { Tuple2.of(new Event(5, "middle", 5.0), 7L), // last element for high final watermark Tuple2.of(new Event(5, "middle", 5.0), 100L) - ).assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks>() { + ).assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks>() { @Override public long extractTimestamp(Tuple2 element, long previousTimestamp) { @@ -307,7 +312,7 @@ public void testSimpleKeyedPatternEventTime() throws Exception { Tuple2.of(new Event(1, "middle", 5.0), 7L), Tuple2.of(new Event(3, "middle", 6.0), 9L), Tuple2.of(new Event(3, "end", 7.0), 7L) - ).assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks>() { + ).assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks>() { @Override public long extractTimestamp(Tuple2 element, long currentTimestamp) { @@ -444,7 +449,7 @@ public void testTimeoutHandling() throws Exception { Tuple2.of(new Event(1, "middle", 2.0), 5L), Tuple2.of(new Event(1, "start", 2.0), 4L), Tuple2.of(new Event(1, "end", 2.0), 6L) - ).assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks>() { + ).assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks>() { @Override public long extractTimestamp(Tuple2 element, long currentTimestamp) { @@ -515,7 +520,8 @@ public String select(Map> pattern) { } /** - * Checks that a certain event sequence is recognized with an OR filter + * Checks that a certain event sequence is recognized with an OR filter. + * * @throws Exception */ @Test @@ -580,4 +586,4 @@ public String select(Map> pattern) { env.execute(); } -} \ No newline at end of file +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/Event.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/Event.java index efe56b78ea5fb..ef072ce772f3a 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/Event.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/Event.java @@ -25,6 +25,9 @@ import java.util.Objects; +/** + * Exemplary event for usage in tests of CEP. See also {@link SubEvent} + */ public class Event { private String name; private double price; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/SubEvent.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/SubEvent.java index effb382de1086..cf5dc9d2df1a7 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/SubEvent.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/SubEvent.java @@ -18,6 +18,9 @@ package org.apache.flink.cep; +/** + * A subclass of {@link Event} for usage in tests. + */ public class SubEvent extends Event { private final double volume; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/DeweyNumberTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/DeweyNumberTest.java index 8bc010a008bb9..e28e77dcd9433 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/DeweyNumberTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/DeweyNumberTest.java @@ -19,12 +19,16 @@ package org.apache.flink.cep.nfa; import org.apache.flink.util.TestLogger; + import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +/** + * Tests for {@link DeweyNumber}. + */ public class DeweyNumberTest extends TestLogger { @Test diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java new file mode 100644 index 0000000000000..910907fa42f9b --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java @@ -0,0 +1,421 @@ +/* + * 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. + */ + +package org.apache.flink.cep.nfa; + +import org.apache.flink.cep.Event; +import org.apache.flink.cep.SubEvent; +import org.apache.flink.cep.nfa.compiler.NFACompiler; +import org.apache.flink.cep.pattern.Pattern; +import org.apache.flink.cep.pattern.conditions.IterativeCondition; +import org.apache.flink.cep.pattern.conditions.SimpleCondition; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.TestLogger; + +import com.google.common.collect.Lists; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps; +import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA; + +/** + * IT tests covering {@link IterativeCondition} usage. + */ +@SuppressWarnings("unchecked") +public class IterativeConditionsITCase extends TestLogger { + + ////////////////////// Iterative BooleanConditions ///////////////////////// + + private final Event startEvent1 = new Event(40, "start", 1.0); + private final Event startEvent2 = new Event(40, "start", 2.0); + private final Event startEvent3 = new Event(40, "start", 3.0); + private final Event startEvent4 = new Event(40, "start", 4.0); + private final SubEvent middleEvent1 = new SubEvent(41, "foo1", 1.0, 10); + private final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10); + private final SubEvent middleEvent3 = new SubEvent(43, "foo3", 3.0, 10); + private final SubEvent middleEvent4 = new SubEvent(43, "foo4", 1.0, 10); + private final Event nextOne = new Event(44, "next-one", 1.0); + private final Event endEvent = new Event(46, "end", 1.0); + + @Test + public void testIterativeWithBranchingPatternEager() { + List> actual = testIterativeWithBranchingPattern(true); + + compareMaps(actual, + Lists.>newArrayList( + Lists.newArrayList(startEvent1, endEvent, middleEvent1, middleEvent2, middleEvent4), + Lists.newArrayList(startEvent1, endEvent, middleEvent2, middleEvent1), + Lists.newArrayList(startEvent1, endEvent, middleEvent1), + Lists.newArrayList(startEvent2, endEvent, middleEvent3, middleEvent4), + Lists.newArrayList(startEvent2, endEvent, middleEvent3) + ) + ); + } + + @Test + public void testIterativeWithBranchingPatternCombinations() { + List> actual = testIterativeWithBranchingPattern(false); + + compareMaps(actual, + Lists.>newArrayList( + Lists.newArrayList(startEvent1, endEvent, middleEvent1, middleEvent2, middleEvent4), + Lists.newArrayList(startEvent1, endEvent, middleEvent2, middleEvent1), + Lists.newArrayList(startEvent1, endEvent, middleEvent3, middleEvent1), + Lists.newArrayList(startEvent2, endEvent, middleEvent3, middleEvent4), + Lists.newArrayList(startEvent1, endEvent, middleEvent4, middleEvent1), + Lists.newArrayList(startEvent1, endEvent, middleEvent1), + Lists.newArrayList(startEvent2, endEvent, middleEvent3) + ) + ); + } + + private List> testIterativeWithBranchingPattern(boolean eager) { + List> inputEvents = new ArrayList<>(); + + inputEvents.add(new StreamRecord<>(startEvent1, 1)); + inputEvents.add(new StreamRecord(middleEvent1, 2)); + inputEvents.add(new StreamRecord(middleEvent2, 3)); + inputEvents.add(new StreamRecord<>(startEvent2, 4)); + inputEvents.add(new StreamRecord(middleEvent3, 5)); + inputEvents.add(new StreamRecord(middleEvent4, 5)); + inputEvents.add(new StreamRecord<>(nextOne, 6)); + inputEvents.add(new StreamRecord<>(endEvent, 8)); + + Pattern pattern = eager + ? Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("start"); + } + }) + .followedBy("middle").subtype(SubEvent.class).where(new MySubeventIterCondition()).oneOrMore() + .followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 7056763917392056548L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("end"); + } + }) + : Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("start"); + } + }) + .followedBy("middle").subtype(SubEvent.class).where(new MySubeventIterCondition()).oneOrMore().allowCombinations() + .followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 7056763917392056548L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("end"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + return feedNFA(inputEvents, nfa); + } + + private static class MySubeventIterCondition extends IterativeCondition { + + private static final long serialVersionUID = 6215754202506583964L; + + @Override + public boolean filter (SubEvent value, Context < SubEvent > ctx) throws Exception { + if (!value.getName().startsWith("foo")) { + return false; + } + + double sum = 0.0; + for (Event event : ctx.getEventsForPattern("middle")) { + sum += event.getPrice(); + } + sum += value.getPrice(); + return Double.compare(sum, 5.0) < 0; + } + } + + @Test + public void testIterativeWithLoopingStartingEager() { + List> actual = testIterativeWithLoopingStarting(true); + + compareMaps(actual, + Lists.>newArrayList( + Lists.newArrayList(startEvent1, startEvent2, endEvent), + Lists.newArrayList(startEvent1, endEvent), + Lists.newArrayList(startEvent2, endEvent), + Lists.newArrayList(startEvent3, endEvent), + Lists.newArrayList(endEvent) + ) + ); + } + + @Test + public void testIterativeWithLoopingStartingCombination() { + List> actual = testIterativeWithLoopingStarting(false); + + compareMaps(actual, + Lists.>newArrayList( + Lists.newArrayList(startEvent1, startEvent2, endEvent), + Lists.newArrayList(startEvent1, startEvent3, endEvent), + Lists.newArrayList(startEvent1, endEvent), + Lists.newArrayList(startEvent2, endEvent), + Lists.newArrayList(startEvent3, endEvent), + Lists.newArrayList(endEvent) + ) + ); + } + + private List> testIterativeWithLoopingStarting(boolean eager) { + List> inputEvents = new ArrayList<>(); + + inputEvents.add(new StreamRecord<>(startEvent1, 1L)); + inputEvents.add(new StreamRecord<>(startEvent2, 2L)); + inputEvents.add(new StreamRecord<>(startEvent3, 3L)); + inputEvents.add(new StreamRecord<>(endEvent, 4L)); + + // for now, a pattern inherits its continuity property from the followedBy() or next(), and the default + // behavior (which is the one applied in the case that the pattern graph starts with such a pattern) + // of a looping pattern is with relaxed continuity (as in followedBy). + + Pattern pattern = eager + ? Pattern.begin("start").where(new MyEventIterCondition()).oneOrMore().optional() + .followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 7056763917392056548L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("end"); + } + }) + : Pattern.begin("start").where(new MyEventIterCondition()).oneOrMore().allowCombinations().optional() + .followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 7056763917392056548L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("end"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + return feedNFA(inputEvents, nfa); + } + + private static class MyEventIterCondition extends IterativeCondition { + + private static final long serialVersionUID = 6215754202506583964L; + + @Override + public boolean filter(Event value, Context ctx) throws Exception { + if (!value.getName().equals("start")) { + return false; + } + + double sum = 0.0; + for (Event event : ctx.getEventsForPattern("start")) { + sum += event.getPrice(); + } + sum += value.getPrice(); + return Double.compare(sum, 5.0) < 0; + } + } + + @Test + public void testIterativeWithPrevPatternDependency() { + List> inputEvents = new ArrayList<>(); + + inputEvents.add(new StreamRecord<>(startEvent1, 1L)); + inputEvents.add(new StreamRecord<>(startEvent2, 2L)); + inputEvents.add(new StreamRecord<>(endEvent, 4L)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 6215754202506583964L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("start"); + } + }).oneOrMore().followedBy("end").where(new IterativeCondition() { + private static final long serialVersionUID = 7056763917392056548L; + + @Override + public boolean filter(Event value, Context ctx) throws Exception { + if (!value.getName().equals("end")) { + return false; + } + + double sum = 0.0; + for (Event event : ctx.getEventsForPattern("start")) { + sum += event.getPrice(); + } + return Double.compare(sum, 2.0) >= 0; + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, + Lists.>newArrayList( + Lists.newArrayList(startEvent1, startEvent2, endEvent), + Lists.newArrayList(startEvent2, endEvent) + ) + ); + } + + @Test + public void testIterativeWithABACPattern() { + List> inputEvents = new ArrayList<>(); + + inputEvents.add(new StreamRecord<>(startEvent1, 1L)); //1 + inputEvents.add(new StreamRecord(middleEvent1, 2L)); //1 + + inputEvents.add(new StreamRecord<>(startEvent2, 2L)); //2 + inputEvents.add(new StreamRecord<>(startEvent3, 2L)); //3 + inputEvents.add(new StreamRecord(middleEvent2, 2L)); //2 + + inputEvents.add(new StreamRecord<>(startEvent4, 2L)); //4 + inputEvents.add(new StreamRecord(middleEvent3, 2L)); //3 + inputEvents.add(new StreamRecord(middleEvent4, 2L)); //1 + inputEvents.add(new StreamRecord<>(endEvent, 4L)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 6215754202506583964L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("start"); + } + }).followedByAny("middle1").subtype(SubEvent.class).where(new SimpleCondition() { + private static final long serialVersionUID = 2178338526904474690L; + + @Override + public boolean filter(SubEvent value) throws Exception { + return value.getName().startsWith("foo"); + } + }).followedBy("middle2").where(new IterativeCondition() { + private static final long serialVersionUID = -1223388426808292695L; + + @Override + public boolean filter(Event value, Context ctx) throws Exception { + if (!value.getName().equals("start")) { + return false; + } + + double sum = 0.0; + for (Event e: ctx.getEventsForPattern("middle2")) { + sum += e.getPrice(); + } + sum += value.getPrice(); + return Double.compare(sum, 5.0) <= 0; + } + }).oneOrMore().followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 562590474115118323L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("end"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, + Lists.>newArrayList( + Lists.newArrayList(startEvent1, startEvent2, startEvent3, middleEvent1, endEvent), + Lists.newArrayList(startEvent1, middleEvent1, startEvent2, endEvent), + Lists.newArrayList(startEvent1, middleEvent2, startEvent4, endEvent), + Lists.newArrayList(startEvent2, middleEvent2, startEvent4, endEvent), + Lists.newArrayList(startEvent3, middleEvent2, startEvent4, endEvent) + ) + ); + } + + @Test + public void testIterativeWithPrevPatternDependencyAfterBranching() { + List> inputEvents = new ArrayList<>(); + + inputEvents.add(new StreamRecord<>(startEvent1, 1L)); + inputEvents.add(new StreamRecord<>(startEvent2, 2L)); + inputEvents.add(new StreamRecord(middleEvent1, 4L)); + inputEvents.add(new StreamRecord<>(startEvent3, 5L)); + inputEvents.add(new StreamRecord(middleEvent2, 6L)); + inputEvents.add(new StreamRecord<>(endEvent, 7L)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 6215754202506583964L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("start"); + } + }).oneOrMore().followedByAny("middle1").subtype(SubEvent.class).where(new SimpleCondition() { + private static final long serialVersionUID = 2178338526904474690L; + + @Override + public boolean filter(SubEvent value) throws Exception { + return value.getName().startsWith("foo"); + } + }).followedByAny("end").where(new IterativeCondition() { + private static final long serialVersionUID = 7056763917392056548L; + + @Override + public boolean filter(Event value, Context ctx) throws Exception { + if (!value.getName().equals("end")) { + return false; + } + + double sum = 0.0; + for (Event event : ctx.getEventsForPattern("start")) { + sum += event.getPrice(); + } + return Double.compare(sum, 2.0) >= 0; + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, + Lists.>newArrayList( + Lists.newArrayList(startEvent1, startEvent2, middleEvent1, endEvent), + Lists.newArrayList(startEvent2, middleEvent1, endEvent), + Lists.newArrayList(startEvent1, startEvent2, middleEvent2, endEvent), + Lists.newArrayList(startEvent1, startEvent2, startEvent3, middleEvent2, endEvent), + Lists.newArrayList(startEvent2, startEvent3, middleEvent2, endEvent), + Lists.newArrayList(startEvent2, middleEvent2, endEvent), + Lists.newArrayList(startEvent3, middleEvent2, endEvent) + ) + ); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java index d00bbb72da9b1..92b49d3b0456e 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java @@ -18,34 +18,37 @@ package org.apache.flink.cep.nfa; -import com.google.common.collect.Lists; -import com.google.common.primitives.Doubles; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cep.Event; import org.apache.flink.cep.SubEvent; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.cep.pattern.Pattern; import org.apache.flink.cep.pattern.Quantifier; -import org.apache.flink.cep.pattern.conditions.IterativeCondition; import org.apache.flink.cep.pattern.conditions.SimpleCondition; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; -import org.junit.Assert; + +import com.google.common.collect.Lists; import org.junit.Test; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps; +import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA; import static org.junit.Assert.assertEquals; +/** + * General tests for {@link NFA} features. See also {@link IterativeConditionsITCase}, {@link NotPatternITCase}, + * {@link SameElementITCase} for more specific tests. + */ @SuppressWarnings("unchecked") public class NFAITCase extends TestLogger { @@ -234,7 +237,7 @@ public boolean filter(Event value) throws Exception { /** * Tests that the NFA successfully filters out expired elements with respect to the window - * length + * length. */ @Test public void testSimplePatternWithTimeWindowNFA() { @@ -251,7 +254,6 @@ public void testSimplePatternWithTimeWindowNFA() { events.add(new StreamRecord<>(endEvent = new Event(5, "end", 1.0), 11)); events.add(new StreamRecord<>(new Event(6, "end", 1.0), 13)); - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { private static final long serialVersionUID = 7907391379273505897L; @@ -373,7 +375,7 @@ public void testBranchingPattern() { SubEvent middleEvent3 = new SubEvent(43, "foo3", 1.0, 10.0); SubEvent nextOne1 = new SubEvent(44, "next-one", 1.0, 2.0); SubEvent nextOne2 = new SubEvent(45, "next-one", 1.0, 2.0); - Event endEvent= new Event(46, "end", 1.0); + Event endEvent = new Event(46, "end", 1.0); inputEvents.add(new StreamRecord<>(startEvent, 1)); inputEvents.add(new StreamRecord(middleEvent1, 3)); @@ -1418,7 +1420,7 @@ public boolean filter(Event value) throws Exception { final List> resultingPatterns = feedNFA(inputEvents, nfa); compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, end1), + Lists.newArrayList(startEvent, end1), Lists.newArrayList(end1) )); } @@ -1458,9 +1460,9 @@ public boolean filter(Event value) throws Exception { final List> resultingPatterns = feedNFA(inputEvents, nfa); compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2), - Lists.newArrayList(startEvent, middleEvent1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2), + Lists.newArrayList(startEvent, middleEvent1), Lists.newArrayList(startEvent) )); } @@ -1499,10 +1501,10 @@ public boolean filter(Event value) throws Exception { final List> resultingPatterns = feedNFA(inputEvents, nfa); compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(middleEvent1, middleEvent2, middleEvent3), - Lists.newArrayList(middleEvent1, middleEvent2), + Lists.newArrayList(middleEvent1, middleEvent2, middleEvent3), + Lists.newArrayList(middleEvent1, middleEvent2), Lists.newArrayList(middleEvent1), - Lists.newArrayList(middleEvent2, middleEvent3), + Lists.newArrayList(middleEvent2, middleEvent3), Lists.newArrayList(middleEvent2), Lists.newArrayList(middleEvent3) )); @@ -1539,7 +1541,7 @@ public boolean filter(Event value) throws Exception { final List> resultingPatterns = feedNFA(inputEvents, nfa); compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, middleEvent1), + Lists.newArrayList(startEvent, middleEvent1), Lists.newArrayList(startEvent) )); } @@ -1579,9 +1581,9 @@ public boolean filter(Event value) throws Exception { final List> resultingPatterns = feedNFA(inputEvents, nfa); compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2), - Lists.newArrayList(startEvent, middleEvent1) + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2), + Lists.newArrayList(startEvent, middleEvent1) )); } @@ -1905,12 +1907,12 @@ public boolean filter(Event value) throws Exception { /////////////////////////////// Consecutive //////////////////////////////////////// private static class ConsecutiveData { - static final Event startEvent = new Event(40, "c", 1.0); - static final Event middleEvent1 = new Event(41, "a", 2.0); - static final Event middleEvent2 = new Event(42, "a", 3.0); - static final Event middleEvent3 = new Event(43, "a", 4.0); - static final Event middleEvent4 = new Event(43, "a", 5.0); - static final Event end = new Event(44, "b", 5.0); + private static final Event startEvent = new Event(40, "c", 1.0); + private static final Event middleEvent1 = new Event(41, "a", 2.0); + private static final Event middleEvent2 = new Event(42, "a", 3.0); + private static final Event middleEvent3 = new Event(43, "a", 4.0); + private static final Event middleEvent4 = new Event(43, "a", 5.0); + private static final Event end = new Event(44, "b", 5.0); private ConsecutiveData() { } @@ -2374,7 +2376,6 @@ public boolean filter(Event value) throws Exception { assertEquals(true, nfa.isEmpty()); } - @Test public void testZeroOrMoreClearingBuffer() { Event startEvent = new Event(40, "c", 1.0); @@ -2418,385 +2419,6 @@ public boolean filter(Event value) throws Exception { assertEquals(true, nfa.isEmpty()); } - - ////////////////////// Iterative BooleanConditions ///////////////////////// - - private final Event startEvent1 = new Event(40, "start", 1.0); - private final Event startEvent2 = new Event(40, "start", 2.0); - private final Event startEvent3 = new Event(40, "start", 3.0); - private final Event startEvent4 = new Event(40, "start", 4.0); - private final SubEvent middleEvent1 = new SubEvent(41, "foo1", 1.0, 10); - private final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10); - private final SubEvent middleEvent3 = new SubEvent(43, "foo3", 3.0, 10); - private final SubEvent middleEvent4 = new SubEvent(43, "foo4", 1.0, 10); - private final Event nextOne = new Event(44, "next-one", 1.0); - private final Event endEvent = new Event(46, "end", 1.0); - - @Test - public void testIterativeWithBranchingPatternEager() { - List> actual = testIterativeWithBranchingPattern(true); - - compareMaps(actual, - Lists.>newArrayList( - Lists.newArrayList(startEvent1, endEvent, middleEvent1, middleEvent2, middleEvent4), - Lists.newArrayList(startEvent1, endEvent, middleEvent2, middleEvent1), - Lists.newArrayList(startEvent1, endEvent, middleEvent1), - Lists.newArrayList(startEvent2, endEvent, middleEvent3, middleEvent4), - Lists.newArrayList(startEvent2, endEvent, middleEvent3) - ) - ); - } - - @Test - public void testIterativeWithBranchingPatternCombinations() { - List> actual = testIterativeWithBranchingPattern(false); - - compareMaps(actual, - Lists.>newArrayList( - Lists.newArrayList(startEvent1, endEvent, middleEvent1, middleEvent2, middleEvent4), - Lists.newArrayList(startEvent1, endEvent, middleEvent2, middleEvent1), - Lists.newArrayList(startEvent1, endEvent, middleEvent3, middleEvent1), - Lists.newArrayList(startEvent2, endEvent, middleEvent3, middleEvent4), - Lists.newArrayList(startEvent1, endEvent, middleEvent4, middleEvent1), - Lists.newArrayList(startEvent1, endEvent, middleEvent1), - Lists.newArrayList(startEvent2, endEvent, middleEvent3) - ) - ); - } - - private List> testIterativeWithBranchingPattern(boolean eager) { - List> inputEvents = new ArrayList<>(); - - inputEvents.add(new StreamRecord<>(startEvent1, 1)); - inputEvents.add(new StreamRecord(middleEvent1, 2)); - inputEvents.add(new StreamRecord(middleEvent2, 3)); - inputEvents.add(new StreamRecord<>(startEvent2, 4)); - inputEvents.add(new StreamRecord(middleEvent3, 5)); - inputEvents.add(new StreamRecord(middleEvent4, 5)); - inputEvents.add(new StreamRecord<>(nextOne, 6)); - inputEvents.add(new StreamRecord<>(endEvent, 8)); - - Pattern pattern = eager - ? Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("start"); - } - }) - .followedBy("middle").subtype(SubEvent.class).where(new MySubeventIterCondition()).oneOrMore() - .followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 7056763917392056548L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("end"); - } - }) - : Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("start"); - } - }) - .followedBy("middle").subtype(SubEvent.class).where(new MySubeventIterCondition()).oneOrMore().allowCombinations() - .followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 7056763917392056548L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("end"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - return feedNFA(inputEvents, nfa); - } - - private static class MySubeventIterCondition extends IterativeCondition { - - private static final long serialVersionUID = 6215754202506583964L; - - @Override - public boolean filter (SubEvent value, Context < SubEvent > ctx) throws Exception { - if (!value.getName().startsWith("foo")) { - return false; - } - - double sum = 0.0; - for (Event event : ctx.getEventsForPattern("middle")) { - sum += event.getPrice(); - } - sum += value.getPrice(); - return Double.compare(sum, 5.0) < 0; - } - } - - @Test - public void testIterativeWithLoopingStartingEager() { - List> actual = testIterativeWithLoopingStarting(true); - - compareMaps(actual, - Lists.>newArrayList( - Lists.newArrayList(startEvent1, startEvent2, endEvent), - Lists.newArrayList(startEvent1, endEvent), - Lists.newArrayList(startEvent2, endEvent), - Lists.newArrayList(startEvent3, endEvent), - Lists.newArrayList(endEvent) - ) - ); - } - - @Test - public void testIterativeWithLoopingStartingCombination() { - List> actual = testIterativeWithLoopingStarting(false); - - compareMaps(actual, - Lists.>newArrayList( - Lists.newArrayList(startEvent1, startEvent2, endEvent), - Lists.newArrayList(startEvent1, startEvent3, endEvent), - Lists.newArrayList(startEvent1, endEvent), - Lists.newArrayList(startEvent2, endEvent), - Lists.newArrayList(startEvent3, endEvent), - Lists.newArrayList(endEvent) - ) - ); - } - - private List> testIterativeWithLoopingStarting(boolean eager) { - List> inputEvents = new ArrayList<>(); - - inputEvents.add(new StreamRecord<>(startEvent1, 1L)); - inputEvents.add(new StreamRecord<>(startEvent2, 2L)); - inputEvents.add(new StreamRecord<>(startEvent3, 3L)); - inputEvents.add(new StreamRecord<>(endEvent, 4L)); - - // for now, a pattern inherits its continuity property from the followedBy() or next(), and the default - // behavior (which is the one applied in the case that the pattern graph starts with such a pattern) - // of a looping pattern is with relaxed continuity (as in followedBy). - - Pattern pattern = eager - ? Pattern.begin("start").where(new MyEventIterCondition()).oneOrMore().optional() - .followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 7056763917392056548L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("end"); - } - }) - : Pattern.begin("start").where(new MyEventIterCondition()).oneOrMore().allowCombinations().optional() - .followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 7056763917392056548L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("end"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - return feedNFA(inputEvents, nfa); - } - - private static class MyEventIterCondition extends IterativeCondition { - - private static final long serialVersionUID = 6215754202506583964L; - - @Override - public boolean filter(Event value, Context ctx) throws Exception { - if (!value.getName().equals("start")) { - return false; - } - - double sum = 0.0; - for (Event event : ctx.getEventsForPattern("start")) { - sum += event.getPrice(); - } - sum += value.getPrice(); - return Double.compare(sum, 5.0) < 0; - } - } - - @Test - public void testIterativeWithPrevPatternDependency() { - List> inputEvents = new ArrayList<>(); - - inputEvents.add(new StreamRecord<>(startEvent1, 1L)); - inputEvents.add(new StreamRecord<>(startEvent2, 2L)); - inputEvents.add(new StreamRecord<>(endEvent, 4L)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 6215754202506583964L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("start"); - } - }).oneOrMore().followedBy("end").where(new IterativeCondition() { - private static final long serialVersionUID = 7056763917392056548L; - - @Override - public boolean filter(Event value, Context ctx) throws Exception { - if (!value.getName().equals("end")) { - return false; - } - - double sum = 0.0; - for (Event event : ctx.getEventsForPattern("start")) { - sum += event.getPrice(); - } - return Double.compare(sum, 2.0) >= 0; - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, - Lists.>newArrayList( - Lists.newArrayList(startEvent1, startEvent2, endEvent), - Lists.newArrayList(startEvent2, endEvent) - ) - ); - } - - @Test - public void testIterativeWithABACPattern() { - List> inputEvents = new ArrayList<>(); - - inputEvents.add(new StreamRecord<>(startEvent1, 1L)); //1 - inputEvents.add(new StreamRecord(middleEvent1, 2L)); //1 - - inputEvents.add(new StreamRecord<>(startEvent2, 2L)); //2 - inputEvents.add(new StreamRecord<>(startEvent3, 2L)); //3 - inputEvents.add(new StreamRecord(middleEvent2, 2L)); //2 - - inputEvents.add(new StreamRecord<>(startEvent4, 2L)); //4 - inputEvents.add(new StreamRecord(middleEvent3, 2L)); //3 - inputEvents.add(new StreamRecord(middleEvent4, 2L)); //1 - inputEvents.add(new StreamRecord<>(endEvent, 4L)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 6215754202506583964L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("start"); - } - }).followedByAny("middle1").subtype(SubEvent.class).where(new SimpleCondition() { - private static final long serialVersionUID = 2178338526904474690L; - - @Override - public boolean filter(SubEvent value) throws Exception { - return value.getName().startsWith("foo"); - } - }).followedBy("middle2").where(new IterativeCondition() { - private static final long serialVersionUID = -1223388426808292695L; - - @Override - public boolean filter(Event value, Context ctx) throws Exception { - if (!value.getName().equals("start")) { - return false; - } - - double sum = 0.0; - for (Event e: ctx.getEventsForPattern("middle2")) { - sum += e.getPrice(); - } - sum += value.getPrice(); - return Double.compare(sum, 5.0) <= 0; - } - }).oneOrMore().followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 562590474115118323L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("end"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, - Lists.>newArrayList( - Lists.newArrayList(startEvent1, startEvent2, startEvent3, middleEvent1, endEvent), - Lists.newArrayList(startEvent1, middleEvent1, startEvent2, endEvent), - Lists.newArrayList(startEvent1, middleEvent2, startEvent4, endEvent), - Lists.newArrayList(startEvent2, middleEvent2, startEvent4, endEvent), - Lists.newArrayList(startEvent3, middleEvent2, startEvent4, endEvent) - ) - ); - } - - @Test - public void testIterativeWithPrevPatternDependencyAfterBranching() { - List> inputEvents = new ArrayList<>(); - - inputEvents.add(new StreamRecord<>(startEvent1, 1L)); - inputEvents.add(new StreamRecord<>(startEvent2, 2L)); - inputEvents.add(new StreamRecord(middleEvent1, 4L)); - inputEvents.add(new StreamRecord<>(startEvent3, 5L)); - inputEvents.add(new StreamRecord(middleEvent2, 6L)); - inputEvents.add(new StreamRecord<>(endEvent, 7L)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 6215754202506583964L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("start"); - } - }).oneOrMore().followedByAny("middle1").subtype(SubEvent.class).where(new SimpleCondition() { - private static final long serialVersionUID = 2178338526904474690L; - - @Override - public boolean filter(SubEvent value) throws Exception { - return value.getName().startsWith("foo"); - } - }).followedByAny("end").where(new IterativeCondition() { - private static final long serialVersionUID = 7056763917392056548L; - - @Override - public boolean filter(Event value, Context ctx) throws Exception { - if (!value.getName().equals("end")) { - return false; - } - - double sum = 0.0; - for (Event event : ctx.getEventsForPattern("start")) { - sum += event.getPrice(); - } - return Double.compare(sum, 2.0) >= 0; - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, - Lists.>newArrayList( - Lists.newArrayList(startEvent1, startEvent2, middleEvent1, endEvent), - Lists.newArrayList(startEvent2, middleEvent1, endEvent), - Lists.newArrayList(startEvent1, startEvent2, middleEvent2, endEvent), - Lists.newArrayList(startEvent1, startEvent2, startEvent3, middleEvent2, endEvent), - Lists.newArrayList(startEvent2, startEvent3, middleEvent2, endEvent), - Lists.newArrayList(startEvent2, middleEvent2, endEvent), - Lists.newArrayList(startEvent3, middleEvent2, endEvent) - ) - ); - } - - /////////////////////////////////////// Skip till next ///////////////////////////// @Test @@ -2809,7 +2431,7 @@ public void testBranchingPatternSkipTillNext() { SubEvent middleEvent3 = new SubEvent(43, "foo3", 1.0, 10.0); SubEvent nextOne1 = new SubEvent(44, "next-one", 1.0, 2.0); SubEvent nextOne2 = new SubEvent(45, "next-one", 1.0, 2.0); - Event endEvent= new Event(46, "end", 1.0); + Event endEvent = new Event(46, "end", 1.0); inputEvents.add(new StreamRecord<>(startEvent, 1)); inputEvents.add(new StreamRecord(middleEvent1, 3)); @@ -2868,7 +2490,7 @@ public void testBranchingPatternMixedFollowedBy() { SubEvent middleEvent3 = new SubEvent(43, "foo3", 1.0, 10.0); SubEvent nextOne1 = new SubEvent(44, "next-one", 1.0, 2.0); SubEvent nextOne2 = new SubEvent(45, "next-one", 1.0, 2.0); - Event endEvent= new Event(46, "end", 1.0); + Event endEvent = new Event(46, "end", 1.0); inputEvents.add(new StreamRecord<>(startEvent, 1)); inputEvents.add(new StreamRecord(middleEvent1, 3)); @@ -2919,1130 +2541,76 @@ public boolean filter(Event value) throws Exception { )); } - - ///////////////////////////////////////// Not pattern ///////////////////////////////////////////////// - @Test - public void testNotNext() { + public void testMultipleTakesVersionCollision() { List> inputEvents = new ArrayList<>(); - Event a1 = new Event(40, "a", 1.0); - Event c1 = new Event(41, "c", 2.0); - Event b1 = new Event(42, "b", 3.0); - Event c2 = new Event(43, "c", 4.0); - Event d = new Event(43, "d", 4.0); + Event startEvent = new Event(40, "c", 1.0); + Event middleEvent1 = new Event(41, "a", 2.0); + Event middleEvent2 = new Event(41, "a", 3.0); + Event middleEvent3 = new Event(41, "a", 4.0); + Event middleEvent4 = new Event(41, "a", 5.0); + Event middleEvent5 = new Event(41, "a", 6.0); + Event end = new Event(44, "b", 5.0); - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(c1, 2)); - inputEvents.add(new StreamRecord<>(b1, 3)); - inputEvents.add(new StreamRecord<>(c2, 4)); - inputEvents.add(new StreamRecord<>(d, 5)); + inputEvents.add(new StreamRecord<>(startEvent, 1)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(middleEvent2, 4)); + inputEvents.add(new StreamRecord<>(middleEvent3, 5)); + inputEvents.add(new StreamRecord<>(middleEvent4, 6)); + inputEvents.add(new StreamRecord<>(middleEvent5, 7)); + inputEvents.add(new StreamRecord<>(end, 10)); Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5167288560432018992L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).notNext("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = 2242479288129905510L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 1404509325548220892L; + private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("c"); } - }).followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = -8907427230007830915L; + }).followedBy("middle1").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); + return value.getName().equals("a"); } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(a1, c1, d), - Lists.newArrayList(a1, c2, d) - )); - } - - @Test - public void testNotNextNoMatches() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event b1 = new Event(42, "b", 3.0); - Event c1 = new Event(41, "c", 2.0); - Event c2 = new Event(43, "c", 4.0); - Event d = new Event(43, "d", 4.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(b1, 2)); - inputEvents.add(new StreamRecord<>(c1, 3)); - inputEvents.add(new StreamRecord<>(c2, 4)); - inputEvents.add(new StreamRecord<>(d, 5)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = -339500190577666439L; + }).oneOrMore().allowCombinations().followedBy("middle2").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("a"); } - }).notNext("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = -6913980632538046451L; + }).oneOrMore().allowCombinations().followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("b"); } - }).followedBy("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 3332196998905139891L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 2086563479959018387L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } }); NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> matches = feedNFA(inputEvents, nfa); - - assertEquals(0, matches.size()); - } + final List> resultingPatterns = feedNFA(inputEvents, nfa); - @Test - public void testNotNextNoMatchesAtTheEnd() { - List> inputEvents = new ArrayList<>(); + compareMaps(resultingPatterns, Lists.>newArrayList( - Event a1 = new Event(40, "a", 1.0); - Event c1 = new Event(41, "c", 2.0); - Event c2 = new Event(43, "c", 4.0); - Event d = new Event(43, "d", 4.0); - Event b1 = new Event(42, "b", 3.0); + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, middleEvent5, end), - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(c1, 2)); - inputEvents.add(new StreamRecord<>(c2, 3)); - inputEvents.add(new StreamRecord<>(d, 4)); - inputEvents.add(new StreamRecord<>(b1, 5)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 1672995058886176627L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 6003621617520261554L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedByAny("end").where(new SimpleCondition() { - private static final long serialVersionUID = 887700237024758417L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }).notNext("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = 5239529076086933032L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - assertEquals(0, matches.size()); - } - - @Test - public void testNotFollowedBy() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event c1 = new Event(41, "c", 2.0); - Event b1 = new Event(42, "b", 3.0); - Event c2 = new Event(43, "c", 4.0); - Event d = new Event(43, "d", 4.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(c1, 2)); - inputEvents.add(new StreamRecord<>(b1, 3)); - inputEvents.add(new StreamRecord<>(c2, 4)); - inputEvents.add(new StreamRecord<>(d, 5)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = -2641662468313191976L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).notFollowedBy("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = -3632144132379494778L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 3818766882138348167L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 2033204730795451288L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches,Lists.>newArrayList( - Lists.newArrayList(a1, c1, d) - )); - } - - @Test - public void testNotFollowedByBeforeOptional() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event c1 = new Event(41, "c", 2.0); - Event b1 = new Event(42, "b", 3.0); - Event c2 = new Event(43, "c", 4.0); - Event d = new Event(43, "d", 4.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(c1, 2)); - inputEvents.add(new StreamRecord<>(b1, 3)); - inputEvents.add(new StreamRecord<>(c2, 4)); - inputEvents.add(new StreamRecord<>(d, 5)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = -2454396370205097543L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).notFollowedBy("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = 2749547391611263290L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = -4989511337298217255L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).optional().followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = -8466223836652936608L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches,Lists.>newArrayList( - Lists.newArrayList(a1, c1, d) - )); - } - - @Test - public void testTimesWithNotFollowedBy() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event b1 = new Event(41, "b", 2.0); - Event c = new Event(42, "c", 3.0); - Event b2 = new Event(43, "b", 4.0); - Event d = new Event(43, "d", 4.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(b1, 2)); - inputEvents.add(new StreamRecord<>(c, 3)); - inputEvents.add(new StreamRecord<>(b2, 4)); - inputEvents.add(new StreamRecord<>(d, 5)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = -2568839911852184515L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = -3632232424064269636L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).times(2).notFollowedBy("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = 3685596793523534611L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 1960758663575587243L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches,Lists.>newArrayList()); - } - - @Test - public void testIgnoreStateOfTimesWithNotFollowedBy() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event e = new Event(41, "e", 2.0); - Event c1 = new Event(42, "c", 3.0); - Event b1 = new Event(43, "b", 4.0); - Event c2 = new Event(44, "c", 5.0); - Event d1 = new Event(45, "d", 6.0); - Event d2 = new Event(46, "d", 7.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(d1, 2)); - inputEvents.add(new StreamRecord<>(e, 1)); - inputEvents.add(new StreamRecord<>(b1, 3)); - inputEvents.add(new StreamRecord<>(c1, 2)); - inputEvents.add(new StreamRecord<>(c2, 4)); - inputEvents.add(new StreamRecord<>(d2, 5)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 2814850350025111940L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).notFollowedBy("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = 4988756153568853834L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = -225909103322018778L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).times(2).optional().followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = -924294627956373696L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(a1, d1) - )); - } - - @Test - public void testTimesWithNotFollowedByAfter() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event e = new Event(41, "e", 2.0); - Event c1 = new Event(42, "c", 3.0); - Event b1 = new Event(43, "b", 4.0); - Event b2 = new Event(44, "b", 5.0); - Event d1 = new Event(46, "d", 7.0); - Event d2 = new Event(47, "d", 8.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(d1, 2)); - inputEvents.add(new StreamRecord<>(e, 1)); - inputEvents.add(new StreamRecord<>(b1, 3)); - inputEvents.add(new StreamRecord<>(b2, 3)); - inputEvents.add(new StreamRecord<>(c1, 2)); - inputEvents.add(new StreamRecord<>(d2, 5)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 6193105689601702341L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 5195859580923169111L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).times(2).notFollowedBy("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = 4973027956103783831L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 2724622546678984894L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches, Lists.>newArrayList()); - } - - @Test - public void testNotFollowedByBeforeOptionalAtTheEnd() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event c1 = new Event(41, "c", 2.0); - Event b1 = new Event(42, "b", 3.0); - Event c2 = new Event(43, "c", 4.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(c1, 2)); - inputEvents.add(new StreamRecord<>(b1, 3)); - inputEvents.add(new StreamRecord<>(c2, 4)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = -4289351792573443294L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).notFollowedBy("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = -4989574608417523507L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).followedByAny("end").where(new SimpleCondition() { - private static final long serialVersionUID = -5940131818629290579L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).optional(); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches,Lists.>newArrayList( - Lists.newArrayList(a1, c1), - Lists.newArrayList(a1) - )); - } - - @Test - public void testNotFollowedByBeforeOptionalTimes() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event c1 = new Event(41, "c", 2.0); - Event b1 = new Event(42, "b", 3.0); - Event c2 = new Event(43, "c", 4.0); - Event d = new Event(43, "d", 4.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(c1, 2)); - inputEvents.add(new StreamRecord<>(b1, 3)); - inputEvents.add(new StreamRecord<>(c2, 4)); - inputEvents.add(new StreamRecord<>(d, 5)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = -7885381452276160322L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).notFollowedBy("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = 3471511260235826653L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 9073793782452363833L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).times(2).optional().followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 7972902718259767076L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches,Lists.>newArrayList( - Lists.newArrayList(a1, c1, c2, d) - )); - } - - @Test - public void testNotFollowedByWithBranchingAtStart() { - List> inputEvents = new ArrayList<>(); - - Event a1 = new Event(40, "a", 1.0); - Event b1 = new Event(42, "b", 3.0); - Event c1 = new Event(41, "c", 2.0); - Event a2 = new Event(41, "a", 4.0); - Event c2 = new Event(43, "c", 5.0); - Event d = new Event(43, "d", 6.0); - - inputEvents.add(new StreamRecord<>(a1, 1)); - inputEvents.add(new StreamRecord<>(b1, 2)); - inputEvents.add(new StreamRecord<>(c1, 3)); - inputEvents.add(new StreamRecord<>(a2, 4)); - inputEvents.add(new StreamRecord<>(c2, 5)); - inputEvents.add(new StreamRecord<>(d, 6)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = -7866220136345465444L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).notFollowedBy("notPattern").where(new SimpleCondition() { - private static final long serialVersionUID = 4957837489028234932L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).followedBy("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 5569569968862808007L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = -8579678167937416269L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> matches = feedNFA(inputEvents, nfa); - - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(a2, c2, d) - )); - } - - private static class NotFollowByData { - static final Event a1 = new Event(40, "a", 1.0); - static final Event b1 = new Event(41, "b", 2.0); - static final Event b2 = new Event(42, "b", 3.0); - static final Event b3 = new Event(42, "b", 4.0); - static final Event c1 = new Event(43, "c", 5.0); - static final Event b4 = new Event(42, "b", 6.0); - static final Event b5 = new Event(42, "b", 7.0); - static final Event b6 = new Event(42, "b", 8.0); - static final Event d1 = new Event(43, "d", 9.0); - - private NotFollowByData() { - } - } - - @Test - public void testNotNextAfterOneOrMoreSkipTillNext() { - final List> matches = testNotNextAfterOneOrMore(false); - assertEquals(0, matches.size()); - } - - @Test - public void testNotNextAfterOneOrMoreSkipTillAny() { - final List> matches = testNotNextAfterOneOrMore(true); - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b2, NotFollowByData.d1) - )); - } - - private List> testNotNextAfterOneOrMore(boolean allMatches) { - List> inputEvents = new ArrayList<>(); - - int i = 0; - inputEvents.add(new StreamRecord<>(NotFollowByData.a1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.c1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b2, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.d1, i++)); - - Pattern pattern = Pattern - .begin("a").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }); - - pattern = (allMatches ? pattern.followedByAny("b*") : pattern.followedBy("b*")).where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).oneOrMore() - .notNext("not c").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }) - .followedBy("d").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - return feedNFA(inputEvents, nfa); - } - - @Test - public void testNotFollowedByNextAfterOneOrMoreEager() { - final List> matches = testNotFollowedByAfterOneOrMore(true, false); - assertEquals(0, matches.size()); - } - - @Test - public void testNotFollowedByAnyAfterOneOrMoreEager() { - final List> matches = testNotFollowedByAfterOneOrMore(true, true); - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b6, NotFollowByData.d1) - )); - } - - @Test - public void testNotFollowedByNextAfterOneOrMoreCombinations() { - final List> matches = testNotFollowedByAfterOneOrMore(false, false); - assertEquals(0, matches.size()); - } - - @Test - public void testNotFollowedByAnyAfterOneOrMoreCombinations() { - final List> matches = testNotFollowedByAfterOneOrMore(false, true); - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b4, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b6, NotFollowByData.d1) - )); - } - - private List> testNotFollowedByAfterOneOrMore(boolean eager, boolean allMatches) { - List> inputEvents = new ArrayList<>(); - - int i = 0; - inputEvents.add(new StreamRecord<>(NotFollowByData.a1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b2, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b3, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.c1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b4, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b5, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b6, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.d1, i)); - - Pattern pattern = Pattern - .begin("a").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }); - - pattern = (allMatches ? pattern.followedByAny("b*") : pattern.followedBy("b*")) - .where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }); - - pattern = (eager ? pattern.oneOrMore() : pattern.oneOrMore().allowCombinations()) - .notFollowedBy("not c").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }) - .followedBy("d").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - return feedNFA(inputEvents, nfa); - } - - @Test - public void testNotFollowedByAnyBeforeOneOrMoreEager() { - final List> matches = testNotFollowedByBeforeOneOrMore(true, true); - - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.d1) - )); - } - - @Test - public void testNotFollowedByAnyBeforeOneOrMoreCombinations() { - final List> matches = testNotFollowedByBeforeOneOrMore(false, true); - - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.d1) - )); - } - - @Test - public void testNotFollowedByBeforeOneOrMoreEager() { - final List> matches = testNotFollowedByBeforeOneOrMore(true, false); - - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.d1) - )); - } - - @Test - public void testNotFollowedByBeforeOneOrMoreCombinations() { - final List> matches = testNotFollowedByBeforeOneOrMore(false, false); - - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.d1) - )); - } - - private List> testNotFollowedByBeforeOneOrMore(boolean eager, boolean allMatches) { - List> inputEvents = new ArrayList<>(); - - int i = 0; - inputEvents.add(new StreamRecord<>(NotFollowByData.a1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.c1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b4, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b5, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b6, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.d1, i)); - - Pattern pattern = Pattern - .begin("a").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }) - .notFollowedBy("not c").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }); - - pattern = (allMatches ? pattern.followedByAny("b*") : pattern.followedBy("b*")) - .where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).oneOrMore(); - - pattern = (eager ? pattern : pattern.allowCombinations()) - .followedBy("d").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - return feedNFA(inputEvents, nfa); - } - - @Test - public void testNotFollowedByBeforeZeroOrMoreEagerSkipTillNext() { - final List> matches = testNotFollowedByBeforeZeroOrMore(true, false); - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.d1) - )); - } - - @Test - public void testNotFollowedByBeforeZeroOrMoreCombinationsSkipTillNext() { - final List> matches = testNotFollowedByBeforeZeroOrMore(false, false); - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b6, NotFollowByData.d1) - )); - } - - @Test - public void testNotFollowedByBeforeZeroOrMoreEagerSkipTillAny() { - final List> matches = testNotFollowedByBeforeZeroOrMore(true, true); - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.d1) - )); - } - - @Test - public void testNotFollowedByBeforeZeroOrMoreCombinationsSkipTillAny() { - final List> matches = testNotFollowedByBeforeZeroOrMore(false, true); - compareMaps(matches, Lists.>newArrayList( - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b4, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b5, NotFollowByData.b6, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b5, NotFollowByData.d1), - Lists.newArrayList(NotFollowByData.a1, NotFollowByData.b1, NotFollowByData.b6, NotFollowByData.d1) - )); - } - - private List> testNotFollowedByBeforeZeroOrMore(boolean eager, boolean allMatches) { - List> inputEvents = new ArrayList<>(); - - int i = 0; - inputEvents.add(new StreamRecord<>(NotFollowByData.a1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.c1, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b4, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b5, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.b6, i++)); - inputEvents.add(new StreamRecord<>(NotFollowByData.d1, i)); - - Pattern pattern = Pattern - .begin("a").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }) - .notFollowedBy("not c").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }); - - pattern = (allMatches ? pattern.followedByAny("b*") : pattern.followedBy("b*")) - .where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).oneOrMore().optional(); - - pattern = (eager ? pattern : pattern.allowCombinations()) - .followedBy("d").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("d"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - return feedNFA(inputEvents, nfa); - } - - @Test - public void testEagerZeroOrMoreSameElement() { - List> inputEvents = new ArrayList<>(); - - Event startEvent = new Event(40, "c", 1.0); - Event middleEvent1 = new Event(41, "a", 2.0); - Event middleEvent2 = new Event(42, "a", 3.0); - Event middleEvent3 = new Event(43, "a", 4.0); - Event end1 = new Event(44, "b", 5.0); - - inputEvents.add(new StreamRecord<>(startEvent, 1)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(middleEvent2, 4)); - inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 5)); - inputEvents.add(new StreamRecord<>(middleEvent3, 6)); - inputEvents.add(new StreamRecord<>(middleEvent3, 6)); - inputEvents.add(new StreamRecord<>(end1, 7)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedBy("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).oneOrMore().optional().followedBy("end1").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, middleEvent1, middleEvent1, middleEvent1, middleEvent2, middleEvent3, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1, middleEvent1, middleEvent2, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1, middleEvent1, middleEvent2, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1, middleEvent1, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1, end1), - Lists.newArrayList(startEvent, middleEvent1, end1), - Lists.newArrayList(startEvent, end1) - )); - } - - @Test - public void testMultipleTakesVersionCollision() { - List> inputEvents = new ArrayList<>(); - - Event startEvent = new Event(40, "c", 1.0); - Event middleEvent1 = new Event(41, "a", 2.0); - Event middleEvent2 = new Event(41, "a", 3.0); - Event middleEvent3 = new Event(41, "a", 4.0); - Event middleEvent4 = new Event(41, "a", 5.0); - Event middleEvent5 = new Event(41, "a", 6.0); - Event end = new Event(44, "b", 5.0); - - inputEvents.add(new StreamRecord<>(startEvent, 1)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(middleEvent2, 4)); - inputEvents.add(new StreamRecord<>(middleEvent3, 5)); - inputEvents.add(new StreamRecord<>(middleEvent4, 6)); - inputEvents.add(new StreamRecord<>(middleEvent5, 7)); - inputEvents.add(new StreamRecord<>(end, 10)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedBy("middle1").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).oneOrMore().allowCombinations().followedBy("middle2").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).oneOrMore().allowCombinations().followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, Lists.>newArrayList( - - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, middleEvent5, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, middleEvent5, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, middleEvent5, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, middleEvent5, end), - - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent4, middleEvent5, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent5, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent3, middleEvent4, middleEvent5, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent3, middleEvent4, middleEvent5, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent5, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent4, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent4, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent3, middleEvent4, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent3, middleEvent4, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent4, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent5, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent4, middleEvent5, end), Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, end), Lists.newArrayList(startEvent, middleEvent1, middleEvent3, middleEvent4, end), @@ -4054,386 +2622,4 @@ public boolean filter(Event value) throws Exception { Lists.newArrayList(startEvent, middleEvent1, middleEvent2, end) )); } - - @Test - public void testZeroOrMoreSameElement() { - List> inputEvents = new ArrayList<>(); - - Event startEvent = new Event(40, "c", 1.0); - Event middleEvent1 = new Event(41, "a", 2.0); - Event middleEvent1a = new Event(41, "a", 2.0); - Event middleEvent2 = new Event(42, "a", 3.0); - Event middleEvent3 = new Event(43, "a", 4.0); - Event middleEvent3a = new Event(43, "a", 4.0); - Event end1 = new Event(44, "b", 5.0); - - inputEvents.add(new StreamRecord<>(startEvent, 1)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(middleEvent1a, 3)); - inputEvents.add(new StreamRecord<>(middleEvent2, 4)); - inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 5)); - inputEvents.add(new StreamRecord<>(middleEvent3, 6)); - inputEvents.add(new StreamRecord<>(middleEvent3a, 6)); - inputEvents.add(new StreamRecord<>(end1, 7)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).oneOrMore().optional().allowCombinations().followedByAny("end1").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent2, middleEvent3, middleEvent3a, end1), - - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent2, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent2, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent3, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent2, middleEvent3, middleEvent3a, end1), - - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent2, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent3, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent2, middleEvent3, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent2, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent2, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent3, middleEvent3a, end1), - - Lists.newArrayList(startEvent, middleEvent1, middleEvent1, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent2, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent1, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent2, end1), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent2, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent2, middleEvent3a, end1), - Lists.newArrayList(startEvent, middleEvent3, middleEvent3a, end1), - - Lists.newArrayList(startEvent, middleEvent1, end1), - Lists.newArrayList(startEvent, middleEvent1a, end1), - Lists.newArrayList(startEvent, middleEvent2, end1), - Lists.newArrayList(startEvent, middleEvent3, end1), - Lists.newArrayList(startEvent, middleEvent3a, end1), - - Lists.newArrayList(startEvent, end1) - )); - } - - @Test - public void testSimplePatternWSameElement() throws Exception { - List> inputEvents = new ArrayList<>(); - - Event startEvent = new Event(40, "c", 1.0); - Event middleEvent1 = new Event(41, "a", 2.0); - Event end1 = new Event(44, "b", 5.0); - - inputEvents.add(new StreamRecord<>(startEvent, 1)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(end1, 7)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).followedBy("end1").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, middleEvent1, end1), - Lists.newArrayList(startEvent, middleEvent1, end1) - )); - } - - @Test - public void testIterativeConditionWSameElement() throws Exception { - List> inputEvents = new ArrayList<>(); - - Event startEvent = new Event(40, "c", 1.0); - Event middleEvent1 = new Event(41, "a", 2.0); - Event middleEvent1a = new Event(41, "a", 2.0); - Event middleEvent1b = new Event(41, "a", 2.0); - final Event end = new Event(44, "b", 5.0); - - inputEvents.add(new StreamRecord<>(startEvent, 1)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(middleEvent1a, 3)); - inputEvents.add(new StreamRecord<>(middleEvent1b, 3)); - inputEvents.add(new StreamRecord<>(end, 7)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).oneOrMore().optional().allowCombinations().followedBy("end").where(new IterativeCondition() { - - private static final long serialVersionUID = -5566639743229703237L; - - @Override - public boolean filter(Event value, Context ctx) throws Exception { - double sum = 0.0; - for (Event event: ctx.getEventsForPattern("middle")) { - sum += event.getPrice(); - } - return Double.compare(sum, 4.0) == 0; - } - - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, end), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent1b), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent1b, end) - )); - } - - @Test - public void testEndWLoopingWSameElement() throws Exception { - List> inputEvents = new ArrayList<>(); - - Event startEvent = new Event(40, "c", 1.0); - Event middleEvent1 = new Event(41, "a", 2.0); - Event middleEvent1a = new Event(41, "a", 2.0); - Event middleEvent1b = new Event(41, "a", 2.0); - final Event end = new Event(44, "b", 5.0); - - inputEvents.add(new StreamRecord<>(startEvent, 1)); - inputEvents.add(new StreamRecord<>(middleEvent1, 3)); - inputEvents.add(new StreamRecord<>(middleEvent1a, 3)); - inputEvents.add(new StreamRecord<>(middleEvent1b, 3)); - inputEvents.add(new StreamRecord<>(end, 7)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedByAny("middle").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).oneOrMore().optional(); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent), - Lists.newArrayList(startEvent, middleEvent1), - Lists.newArrayList(startEvent, middleEvent1a), - Lists.newArrayList(startEvent, middleEvent1b), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a), - Lists.newArrayList(startEvent, middleEvent1a, middleEvent1b), - Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent1b) - )); - } - - @Test - public void testRepeatingPatternWSameElement() throws Exception { - List> inputEvents = new ArrayList<>(); - - Event startEvent = new Event(40, "c", 1.0); - Event middle1Event1 = new Event(40, "a", 2.0); - Event middle1Event2 = new Event(40, "a", 3.0); - Event middle1Event3 = new Event(40, "a", 4.0); - Event middle2Event1 = new Event(40, "b", 5.0); - - inputEvents.add(new StreamRecord<>(startEvent, 1)); - inputEvents.add(new StreamRecord<>(middle1Event1, 3)); - inputEvents.add(new StreamRecord<>(middle1Event1, 3)); - inputEvents.add(new StreamRecord<>(middle1Event2, 3)); - inputEvents.add(new StreamRecord<>(new Event(40, "d", 6.0), 5)); - inputEvents.add(new StreamRecord<>(middle2Event1, 6)); - inputEvents.add(new StreamRecord<>(middle1Event3, 7)); - - Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("c"); - } - }).followedBy("middle1").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }).oneOrMore().optional().followedBy("middle2").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("b"); - } - }).optional().followedBy("end").where(new SimpleCondition() { - private static final long serialVersionUID = 5726188262756267490L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("a"); - } - }); - - NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - - final List> resultingPatterns = feedNFA(inputEvents, nfa); - - compareMaps(resultingPatterns, Lists.>newArrayList( - Lists.newArrayList(startEvent, middle1Event1), - - Lists.newArrayList(startEvent, middle1Event1, middle1Event1), - Lists.newArrayList(startEvent, middle2Event1, middle1Event3), - - Lists.newArrayList(startEvent, middle1Event1, middle1Event1, middle1Event2), - Lists.newArrayList(startEvent, middle1Event1, middle2Event1, middle1Event3), - - Lists.newArrayList(startEvent, middle1Event1, middle1Event1, middle1Event2, middle1Event3), - Lists.newArrayList(startEvent, middle1Event1, middle1Event1, middle2Event1, middle1Event3), - - Lists.newArrayList(startEvent, middle1Event1, middle1Event1, middle1Event2, middle2Event1, middle1Event3) - )); - } - - ///////////////////////////////////////// Utility ///////////////////////////////////////////////// - - private List> feedNFA(List> inputEvents, NFA nfa) { - List> resultingPatterns = new ArrayList<>(); - - for (StreamRecord inputEvent : inputEvents) { - Collection>> patterns = nfa.process( - inputEvent.getValue(), - inputEvent.getTimestamp()).f0; - - for (Map> p: patterns) { - List res = new ArrayList<>(); - for (List le: p.values()) { - res.addAll(le); - } - resultingPatterns.add(res); - } - } - return resultingPatterns; - } - - private void compareMaps(List> actual, List> expected) { - Assert.assertEquals(expected.size(), actual.size()); - - for (List p: actual) { - Collections.sort(p, new EventComparator()); - } - - for (List p: expected) { - Collections.sort(p, new EventComparator()); - } - - Collections.sort(actual, new ListEventComparator()); - Collections.sort(expected, new ListEventComparator()); - Assert.assertArrayEquals(expected.toArray(), actual.toArray()); - } - - private class ListEventComparator implements Comparator> { - - @Override - public int compare(List o1, List o2) { - int sizeComp = Integer.compare(o1.size(), o2.size()); - if (sizeComp == 0) { - EventComparator comp = new EventComparator(); - for (int i = 0; i < o1.size(); i++) { - int eventComp = comp.compare(o1.get(i), o2.get(i)); - if (eventComp != 0) { - return eventComp; - } - } - return 0; - } else { - return sizeComp; - } - } - } - - private class EventComparator implements Comparator { - - @Override - public int compare(Event o1, Event o2) { - int nameComp = o1.getName().compareTo(o2.getName()); - int priceComp = Doubles.compare(o1.getPrice(), o2.getPrice()); - int idComp = Integer.compare(o1.getId(), o2.getId()); - if (nameComp == 0) { - if (priceComp == 0) { - return idComp; - } else { - return priceComp; - } - } else { - return nameComp; - } - } - } } diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java index 2619764d152f9..25863423fbc00 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java @@ -18,7 +18,6 @@ package org.apache.flink.cep.nfa; -import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.flink.cep.Event; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.cep.pattern.Pattern; @@ -29,6 +28,8 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; + +import org.apache.commons.io.output.ByteArrayOutputStream; import org.junit.Test; import java.io.ByteArrayInputStream; @@ -44,6 +45,9 @@ import static org.junit.Assert.assertEquals; +/** + * Tests for {@link NFA}. + */ public class NFATest extends TestLogger { @Test public void testSimpleNFA() { @@ -147,7 +151,7 @@ public void testWindowBorders() { /** * Tests that pruning shared buffer elements and computations state use the same window border - * semantics (left side inclusive and right side exclusive) + * semantics (left side inclusive and right side exclusive). */ @Test public void testTimeoutWindowPruningWindowBorders() { diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java new file mode 100644 index 0000000000000..7bf0767755418 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java @@ -0,0 +1,115 @@ +/* + * 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. + */ + +package org.apache.flink.cep.nfa; + +import org.apache.flink.cep.Event; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import com.google.common.primitives.Doubles; +import org.junit.Assert; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +/** + * Base method for IT tests of {@link NFA}. It provides utility methods. + */ +public class NFATestUtilities { + + public static List> feedNFA(List> inputEvents, NFA nfa) { + List> resultingPatterns = new ArrayList<>(); + + for (StreamRecord inputEvent : inputEvents) { + Collection>> patterns = nfa.process( + inputEvent.getValue(), + inputEvent.getTimestamp()).f0; + + for (Map> p: patterns) { + List res = new ArrayList<>(); + for (List le: p.values()) { + res.addAll(le); + } + resultingPatterns.add(res); + } + } + return resultingPatterns; + } + + public static void compareMaps(List> actual, List> expected) { + Assert.assertEquals(expected.size(), actual.size()); + + for (List p: actual) { + Collections.sort(p, new EventComparator()); + } + + for (List p: expected) { + Collections.sort(p, new EventComparator()); + } + + Collections.sort(actual, new ListEventComparator()); + Collections.sort(expected, new ListEventComparator()); + Assert.assertArrayEquals(expected.toArray(), actual.toArray()); + } + + private static class ListEventComparator implements Comparator> { + + @Override + public int compare(List o1, List o2) { + int sizeComp = Integer.compare(o1.size(), o2.size()); + if (sizeComp == 0) { + EventComparator comp = new EventComparator(); + for (int i = 0; i < o1.size(); i++) { + int eventComp = comp.compare(o1.get(i), o2.get(i)); + if (eventComp != 0) { + return eventComp; + } + } + return 0; + } else { + return sizeComp; + } + } + } + + private static class EventComparator implements Comparator { + + @Override + public int compare(Event o1, Event o2) { + int nameComp = o1.getName().compareTo(o2.getName()); + int priceComp = Doubles.compare(o1.getPrice(), o2.getPrice()); + int idComp = Integer.compare(o1.getId(), o2.getId()); + if (nameComp == 0) { + if (priceComp == 0) { + return idComp; + } else { + return priceComp; + } + } else { + return nameComp; + } + } + } + + private NFATestUtilities() { + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java new file mode 100644 index 0000000000000..3b95eb4b6064b --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java @@ -0,0 +1,1036 @@ +/* + * 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. + */ + +package org.apache.flink.cep.nfa; + +import org.apache.flink.cep.Event; +import org.apache.flink.cep.nfa.compiler.NFACompiler; +import org.apache.flink.cep.pattern.Pattern; +import org.apache.flink.cep.pattern.conditions.SimpleCondition; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.TestLogger; + +import com.google.common.collect.Lists; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps; +import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link Pattern#notFollowedBy(String)} and {@link Pattern#notNext(String)}. + */ +@SuppressWarnings("unchecked") +public class NotPatternITCase extends TestLogger { + + @Test + public void testNotNext() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event c1 = new Event(41, "c", 2.0); + Event b1 = new Event(42, "b", 3.0); + Event c2 = new Event(43, "c", 4.0); + Event d = new Event(43, "d", 4.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(c1, 2)); + inputEvents.add(new StreamRecord<>(b1, 3)); + inputEvents.add(new StreamRecord<>(c2, 4)); + inputEvents.add(new StreamRecord<>(d, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5167288560432018992L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).notNext("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = 2242479288129905510L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 1404509325548220892L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = -8907427230007830915L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(a1, c1, d), + Lists.newArrayList(a1, c2, d) + )); + } + + @Test + public void testNotNextNoMatches() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event b1 = new Event(42, "b", 3.0); + Event c1 = new Event(41, "c", 2.0); + Event c2 = new Event(43, "c", 4.0); + Event d = new Event(43, "d", 4.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(b1, 2)); + inputEvents.add(new StreamRecord<>(c1, 3)); + inputEvents.add(new StreamRecord<>(c2, 4)); + inputEvents.add(new StreamRecord<>(d, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = -339500190577666439L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).notNext("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = -6913980632538046451L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).followedBy("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 3332196998905139891L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 2086563479959018387L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + assertEquals(0, matches.size()); + } + + @Test + public void testNotNextNoMatchesAtTheEnd() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event c1 = new Event(41, "c", 2.0); + Event c2 = new Event(43, "c", 4.0); + Event d = new Event(43, "d", 4.0); + Event b1 = new Event(42, "b", 3.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(c1, 2)); + inputEvents.add(new StreamRecord<>(c2, 3)); + inputEvents.add(new StreamRecord<>(d, 4)); + inputEvents.add(new StreamRecord<>(b1, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 1672995058886176627L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 6003621617520261554L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedByAny("end").where(new SimpleCondition() { + private static final long serialVersionUID = 887700237024758417L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }).notNext("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = 5239529076086933032L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + assertEquals(0, matches.size()); + } + + @Test + public void testNotFollowedBy() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event c1 = new Event(41, "c", 2.0); + Event b1 = new Event(42, "b", 3.0); + Event c2 = new Event(43, "c", 4.0); + Event d = new Event(43, "d", 4.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(c1, 2)); + inputEvents.add(new StreamRecord<>(b1, 3)); + inputEvents.add(new StreamRecord<>(c2, 4)); + inputEvents.add(new StreamRecord<>(d, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = -2641662468313191976L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).notFollowedBy("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = -3632144132379494778L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 3818766882138348167L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 2033204730795451288L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(a1, c1, d) + )); + } + + @Test + public void testNotFollowedByBeforeOptional() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event c1 = new Event(41, "c", 2.0); + Event b1 = new Event(42, "b", 3.0); + Event c2 = new Event(43, "c", 4.0); + Event d = new Event(43, "d", 4.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(c1, 2)); + inputEvents.add(new StreamRecord<>(b1, 3)); + inputEvents.add(new StreamRecord<>(c2, 4)); + inputEvents.add(new StreamRecord<>(d, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = -2454396370205097543L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).notFollowedBy("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = 2749547391611263290L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = -4989511337298217255L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).optional().followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = -8466223836652936608L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(a1, c1, d) + )); + } + + @Test + public void testTimesWithNotFollowedBy() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event b1 = new Event(41, "b", 2.0); + Event c = new Event(42, "c", 3.0); + Event b2 = new Event(43, "b", 4.0); + Event d = new Event(43, "d", 4.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(b1, 2)); + inputEvents.add(new StreamRecord<>(c, 3)); + inputEvents.add(new StreamRecord<>(b2, 4)); + inputEvents.add(new StreamRecord<>(d, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = -2568839911852184515L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = -3632232424064269636L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).times(2).notFollowedBy("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = 3685596793523534611L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 1960758663575587243L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList()); + } + + @Test + public void testIgnoreStateOfTimesWithNotFollowedBy() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event e = new Event(41, "e", 2.0); + Event c1 = new Event(42, "c", 3.0); + Event b1 = new Event(43, "b", 4.0); + Event c2 = new Event(44, "c", 5.0); + Event d1 = new Event(45, "d", 6.0); + Event d2 = new Event(46, "d", 7.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(d1, 2)); + inputEvents.add(new StreamRecord<>(e, 1)); + inputEvents.add(new StreamRecord<>(b1, 3)); + inputEvents.add(new StreamRecord<>(c1, 2)); + inputEvents.add(new StreamRecord<>(c2, 4)); + inputEvents.add(new StreamRecord<>(d2, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 2814850350025111940L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).notFollowedBy("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = 4988756153568853834L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = -225909103322018778L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).times(2).optional().followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = -924294627956373696L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(a1, d1) + )); + } + + @Test + public void testTimesWithNotFollowedByAfter() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event e = new Event(41, "e", 2.0); + Event c1 = new Event(42, "c", 3.0); + Event b1 = new Event(43, "b", 4.0); + Event b2 = new Event(44, "b", 5.0); + Event d1 = new Event(46, "d", 7.0); + Event d2 = new Event(47, "d", 8.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(d1, 2)); + inputEvents.add(new StreamRecord<>(e, 1)); + inputEvents.add(new StreamRecord<>(b1, 3)); + inputEvents.add(new StreamRecord<>(b2, 3)); + inputEvents.add(new StreamRecord<>(c1, 2)); + inputEvents.add(new StreamRecord<>(d2, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 6193105689601702341L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 5195859580923169111L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).times(2).notFollowedBy("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = 4973027956103783831L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 2724622546678984894L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList()); + } + + @Test + public void testNotFollowedByBeforeOptionalAtTheEnd() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event c1 = new Event(41, "c", 2.0); + Event b1 = new Event(42, "b", 3.0); + Event c2 = new Event(43, "c", 4.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(c1, 2)); + inputEvents.add(new StreamRecord<>(b1, 3)); + inputEvents.add(new StreamRecord<>(c2, 4)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = -4289351792573443294L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).notFollowedBy("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = -4989574608417523507L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).followedByAny("end").where(new SimpleCondition() { + private static final long serialVersionUID = -5940131818629290579L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).optional(); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(a1, c1), + Lists.newArrayList(a1) + )); + } + + @Test + public void testNotFollowedByBeforeOptionalTimes() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event c1 = new Event(41, "c", 2.0); + Event b1 = new Event(42, "b", 3.0); + Event c2 = new Event(43, "c", 4.0); + Event d = new Event(43, "d", 4.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(c1, 2)); + inputEvents.add(new StreamRecord<>(b1, 3)); + inputEvents.add(new StreamRecord<>(c2, 4)); + inputEvents.add(new StreamRecord<>(d, 5)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = -7885381452276160322L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).notFollowedBy("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = 3471511260235826653L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 9073793782452363833L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).times(2).optional().followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 7972902718259767076L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(a1, c1, c2, d) + )); + } + + @Test + public void testNotFollowedByWithBranchingAtStart() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(40, "a", 1.0); + Event b1 = new Event(42, "b", 3.0); + Event c1 = new Event(41, "c", 2.0); + Event a2 = new Event(41, "a", 4.0); + Event c2 = new Event(43, "c", 5.0); + Event d = new Event(43, "d", 6.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(b1, 2)); + inputEvents.add(new StreamRecord<>(c1, 3)); + inputEvents.add(new StreamRecord<>(a2, 4)); + inputEvents.add(new StreamRecord<>(c2, 5)); + inputEvents.add(new StreamRecord<>(d, 6)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = -7866220136345465444L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).notFollowedBy("notPattern").where(new SimpleCondition() { + private static final long serialVersionUID = 4957837489028234932L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).followedBy("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 5569569968862808007L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = -8579678167937416269L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> matches = feedNFA(inputEvents, nfa); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(a2, c2, d) + )); + } + + private static class NotFollowByData { + static final Event A_1 = new Event(40, "a", 1.0); + static final Event B_1 = new Event(41, "b", 2.0); + static final Event B_2 = new Event(42, "b", 3.0); + static final Event B_3 = new Event(42, "b", 4.0); + static final Event C_1 = new Event(43, "c", 5.0); + static final Event B_4 = new Event(42, "b", 6.0); + static final Event B_5 = new Event(42, "b", 7.0); + static final Event B_6 = new Event(42, "b", 8.0); + static final Event D_1 = new Event(43, "d", 9.0); + + private NotFollowByData() { + } + } + + @Test + public void testNotNextAfterOneOrMoreSkipTillNext() { + final List> matches = testNotNextAfterOneOrMore(false); + assertEquals(0, matches.size()); + } + + @Test + public void testNotNextAfterOneOrMoreSkipTillAny() { + final List> matches = testNotNextAfterOneOrMore(true); + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_2, NotFollowByData.D_1) + )); + } + + private List> testNotNextAfterOneOrMore(boolean allMatches) { + List> inputEvents = new ArrayList<>(); + + int i = 0; + inputEvents.add(new StreamRecord<>(NotFollowByData.A_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.C_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_2, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.D_1, i++)); + + Pattern pattern = Pattern + .begin("a").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }); + + pattern = (allMatches ? pattern.followedByAny("b*") : pattern.followedBy("b*")).where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).oneOrMore() + .notNext("not c").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }) + .followedBy("d").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + return feedNFA(inputEvents, nfa); + } + + @Test + public void testNotFollowedByNextAfterOneOrMoreEager() { + final List> matches = testNotFollowedByAfterOneOrMore(true, false); + assertEquals(0, matches.size()); + } + + @Test + public void testNotFollowedByAnyAfterOneOrMoreEager() { + final List> matches = testNotFollowedByAfterOneOrMore(true, true); + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_6, NotFollowByData.D_1) + )); + } + + @Test + public void testNotFollowedByNextAfterOneOrMoreCombinations() { + final List> matches = testNotFollowedByAfterOneOrMore(false, false); + assertEquals(0, matches.size()); + } + + @Test + public void testNotFollowedByAnyAfterOneOrMoreCombinations() { + final List> matches = testNotFollowedByAfterOneOrMore(false, true); + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_6, NotFollowByData.D_1) + )); + } + + private List> testNotFollowedByAfterOneOrMore(boolean eager, boolean allMatches) { + List> inputEvents = new ArrayList<>(); + + int i = 0; + inputEvents.add(new StreamRecord<>(NotFollowByData.A_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_2, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_3, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.C_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_4, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_5, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_6, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.D_1, i)); + + Pattern pattern = Pattern + .begin("a").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }); + + pattern = (allMatches ? pattern.followedByAny("b*") : pattern.followedBy("b*")) + .where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }); + + pattern = (eager ? pattern.oneOrMore() : pattern.oneOrMore().allowCombinations()) + .notFollowedBy("not c").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }) + .followedBy("d").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + return feedNFA(inputEvents, nfa); + } + + @Test + public void testNotFollowedByAnyBeforeOneOrMoreEager() { + final List> matches = testNotFollowedByBeforeOneOrMore(true, true); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.D_1) + )); + } + + @Test + public void testNotFollowedByAnyBeforeOneOrMoreCombinations() { + final List> matches = testNotFollowedByBeforeOneOrMore(false, true); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.D_1) + )); + } + + @Test + public void testNotFollowedByBeforeOneOrMoreEager() { + final List> matches = testNotFollowedByBeforeOneOrMore(true, false); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.D_1) + )); + } + + @Test + public void testNotFollowedByBeforeOneOrMoreCombinations() { + final List> matches = testNotFollowedByBeforeOneOrMore(false, false); + + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.D_1) + )); + } + + private List> testNotFollowedByBeforeOneOrMore(boolean eager, boolean allMatches) { + List> inputEvents = new ArrayList<>(); + + int i = 0; + inputEvents.add(new StreamRecord<>(NotFollowByData.A_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.C_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_4, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_5, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_6, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.D_1, i)); + + Pattern pattern = Pattern + .begin("a").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }) + .notFollowedBy("not c").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }); + + pattern = (allMatches ? pattern.followedByAny("b*") : pattern.followedBy("b*")) + .where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).oneOrMore(); + + pattern = (eager ? pattern : pattern.allowCombinations()) + .followedBy("d").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + return feedNFA(inputEvents, nfa); + } + + @Test + public void testNotFollowedByBeforeZeroOrMoreEagerSkipTillNext() { + final List> matches = testNotFollowedByBeforeZeroOrMore(true, false); + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.D_1) + )); + } + + @Test + public void testNotFollowedByBeforeZeroOrMoreCombinationsSkipTillNext() { + final List> matches = testNotFollowedByBeforeZeroOrMore(false, false); + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_6, NotFollowByData.D_1) + )); + } + + @Test + public void testNotFollowedByBeforeZeroOrMoreEagerSkipTillAny() { + final List> matches = testNotFollowedByBeforeZeroOrMore(true, true); + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.D_1) + )); + } + + @Test + public void testNotFollowedByBeforeZeroOrMoreCombinationsSkipTillAny() { + final List> matches = testNotFollowedByBeforeZeroOrMore(false, true); + compareMaps(matches, Lists.>newArrayList( + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_5, NotFollowByData.D_1), + Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_6, NotFollowByData.D_1) + )); + } + + private List> testNotFollowedByBeforeZeroOrMore(boolean eager, boolean allMatches) { + List> inputEvents = new ArrayList<>(); + + int i = 0; + inputEvents.add(new StreamRecord<>(NotFollowByData.A_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.C_1, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_4, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_5, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.B_6, i++)); + inputEvents.add(new StreamRecord<>(NotFollowByData.D_1, i)); + + Pattern pattern = Pattern + .begin("a").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }) + .notFollowedBy("not c").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }); + + pattern = (allMatches ? pattern.followedByAny("b*") : pattern.followedBy("b*")) + .where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).oneOrMore().optional(); + + pattern = (eager ? pattern : pattern.allowCombinations()) + .followedBy("d").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("d"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + return feedNFA(inputEvents, nfa); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java new file mode 100644 index 0000000000000..d378a741768e7 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java @@ -0,0 +1,407 @@ +/* + * 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. + */ + +package org.apache.flink.cep.nfa; + +import org.apache.flink.cep.Event; +import org.apache.flink.cep.nfa.compiler.NFACompiler; +import org.apache.flink.cep.pattern.Pattern; +import org.apache.flink.cep.pattern.conditions.IterativeCondition; +import org.apache.flink.cep.pattern.conditions.SimpleCondition; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.TestLogger; + +import com.google.common.collect.Lists; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps; +import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA; + +/** + * Tests for handling Events that are equal in case of {@link Object#equals(Object)} and have same timestamps. + */ +@SuppressWarnings("unchecked") +public class SameElementITCase extends TestLogger { + + @Test + public void testEagerZeroOrMoreSameElement() { + List> inputEvents = new ArrayList<>(); + + Event startEvent = new Event(40, "c", 1.0); + Event middleEvent1 = new Event(41, "a", 2.0); + Event middleEvent2 = new Event(42, "a", 3.0); + Event middleEvent3 = new Event(43, "a", 4.0); + Event end1 = new Event(44, "b", 5.0); + + inputEvents.add(new StreamRecord<>(startEvent, 1)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(middleEvent2, 4)); + inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 5)); + inputEvents.add(new StreamRecord<>(middleEvent3, 6)); + inputEvents.add(new StreamRecord<>(middleEvent3, 6)); + inputEvents.add(new StreamRecord<>(end1, 7)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedBy("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).oneOrMore().optional().followedBy("end1").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, Lists.>newArrayList( + Lists.newArrayList(startEvent, middleEvent1, middleEvent1, middleEvent1, middleEvent2, middleEvent3, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1, middleEvent1, middleEvent2, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1, middleEvent1, middleEvent2, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1, middleEvent1, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1, end1), + Lists.newArrayList(startEvent, middleEvent1, end1), + Lists.newArrayList(startEvent, end1) + )); + } + + @Test + public void testZeroOrMoreSameElement() { + List> inputEvents = new ArrayList<>(); + + Event startEvent = new Event(40, "c", 1.0); + Event middleEvent1 = new Event(41, "a", 2.0); + Event middleEvent1a = new Event(41, "a", 2.0); + Event middleEvent2 = new Event(42, "a", 3.0); + Event middleEvent3 = new Event(43, "a", 4.0); + Event middleEvent3a = new Event(43, "a", 4.0); + Event end1 = new Event(44, "b", 5.0); + + inputEvents.add(new StreamRecord<>(startEvent, 1)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(middleEvent1a, 3)); + inputEvents.add(new StreamRecord<>(middleEvent2, 4)); + inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 5)); + inputEvents.add(new StreamRecord<>(middleEvent3, 6)); + inputEvents.add(new StreamRecord<>(middleEvent3a, 6)); + inputEvents.add(new StreamRecord<>(end1, 7)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).oneOrMore().optional().allowCombinations().followedByAny("end1").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, Lists.>newArrayList( + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent2, middleEvent3, middleEvent3a, end1), + + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent2, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent2, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent3, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent2, middleEvent3, middleEvent3a, end1), + + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent2, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent3, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent2, middleEvent3, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent2, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent2, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent3, middleEvent3a, end1), + + Lists.newArrayList(startEvent, middleEvent1, middleEvent1, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent2, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent1, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent2, end1), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent2, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent2, middleEvent3a, end1), + Lists.newArrayList(startEvent, middleEvent3, middleEvent3a, end1), + + Lists.newArrayList(startEvent, middleEvent1, end1), + Lists.newArrayList(startEvent, middleEvent1a, end1), + Lists.newArrayList(startEvent, middleEvent2, end1), + Lists.newArrayList(startEvent, middleEvent3, end1), + Lists.newArrayList(startEvent, middleEvent3a, end1), + + Lists.newArrayList(startEvent, end1) + )); + } + + @Test + public void testSimplePatternWSameElement() throws Exception { + List> inputEvents = new ArrayList<>(); + + Event startEvent = new Event(40, "c", 1.0); + Event middleEvent1 = new Event(41, "a", 2.0); + Event end1 = new Event(44, "b", 5.0); + + inputEvents.add(new StreamRecord<>(startEvent, 1)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(end1, 7)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).followedBy("end1").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, Lists.>newArrayList( + Lists.newArrayList(startEvent, middleEvent1, end1), + Lists.newArrayList(startEvent, middleEvent1, end1) + )); + } + + @Test + public void testIterativeConditionWSameElement() throws Exception { + List> inputEvents = new ArrayList<>(); + + Event startEvent = new Event(40, "c", 1.0); + Event middleEvent1 = new Event(41, "a", 2.0); + Event middleEvent1a = new Event(41, "a", 2.0); + Event middleEvent1b = new Event(41, "a", 2.0); + final Event end = new Event(44, "b", 5.0); + + inputEvents.add(new StreamRecord<>(startEvent, 1)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(middleEvent1a, 3)); + inputEvents.add(new StreamRecord<>(middleEvent1b, 3)); + inputEvents.add(new StreamRecord<>(end, 7)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).oneOrMore().optional().allowCombinations().followedBy("end").where(new IterativeCondition() { + + private static final long serialVersionUID = -5566639743229703237L; + + @Override + public boolean filter(Event value, Context ctx) throws Exception { + double sum = 0.0; + for (Event event: ctx.getEventsForPattern("middle")) { + sum += event.getPrice(); + } + return Double.compare(sum, 4.0) == 0; + } + + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, Lists.>newArrayList( + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, end), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent1b), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent1b, end) + )); + } + + @Test + public void testEndWLoopingWSameElement() throws Exception { + List> inputEvents = new ArrayList<>(); + + Event startEvent = new Event(40, "c", 1.0); + Event middleEvent1 = new Event(41, "a", 2.0); + Event middleEvent1a = new Event(41, "a", 2.0); + Event middleEvent1b = new Event(41, "a", 2.0); + final Event end = new Event(44, "b", 5.0); + + inputEvents.add(new StreamRecord<>(startEvent, 1)); + inputEvents.add(new StreamRecord<>(middleEvent1, 3)); + inputEvents.add(new StreamRecord<>(middleEvent1a, 3)); + inputEvents.add(new StreamRecord<>(middleEvent1b, 3)); + inputEvents.add(new StreamRecord<>(end, 7)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedByAny("middle").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).oneOrMore().optional(); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, Lists.>newArrayList( + Lists.newArrayList(startEvent), + Lists.newArrayList(startEvent, middleEvent1), + Lists.newArrayList(startEvent, middleEvent1a), + Lists.newArrayList(startEvent, middleEvent1b), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a), + Lists.newArrayList(startEvent, middleEvent1a, middleEvent1b), + Lists.newArrayList(startEvent, middleEvent1, middleEvent1a, middleEvent1b) + )); + } + + @Test + public void testRepeatingPatternWSameElement() throws Exception { + List> inputEvents = new ArrayList<>(); + + Event startEvent = new Event(40, "c", 1.0); + Event middle1Event1 = new Event(40, "a", 2.0); + Event middle1Event2 = new Event(40, "a", 3.0); + Event middle1Event3 = new Event(40, "a", 4.0); + Event middle2Event1 = new Event(40, "b", 5.0); + + inputEvents.add(new StreamRecord<>(startEvent, 1)); + inputEvents.add(new StreamRecord<>(middle1Event1, 3)); + inputEvents.add(new StreamRecord<>(middle1Event1, 3)); + inputEvents.add(new StreamRecord<>(middle1Event2, 3)); + inputEvents.add(new StreamRecord<>(new Event(40, "d", 6.0), 5)); + inputEvents.add(new StreamRecord<>(middle2Event1, 6)); + inputEvents.add(new StreamRecord<>(middle1Event3, 7)); + + Pattern pattern = Pattern.begin("start").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("c"); + } + }).followedBy("middle1").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).oneOrMore().optional().followedBy("middle2").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("b"); + } + }).optional().followedBy("end").where(new SimpleCondition() { + private static final long serialVersionUID = 5726188262756267490L; + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + final List> resultingPatterns = feedNFA(inputEvents, nfa); + + compareMaps(resultingPatterns, Lists.>newArrayList( + Lists.newArrayList(startEvent, middle1Event1), + + Lists.newArrayList(startEvent, middle1Event1, middle1Event1), + Lists.newArrayList(startEvent, middle2Event1, middle1Event3), + + Lists.newArrayList(startEvent, middle1Event1, middle1Event1, middle1Event2), + Lists.newArrayList(startEvent, middle1Event1, middle2Event1, middle1Event3), + + Lists.newArrayList(startEvent, middle1Event1, middle1Event1, middle1Event2, middle1Event3), + Lists.newArrayList(startEvent, middle1Event1, middle1Event1, middle2Event1, middle1Event3), + + Lists.newArrayList(startEvent, middle1Event1, middle1Event1, middle1Event2, middle2Event1, middle1Event3) + )); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java index bd828b67c7ce6..44033c1690b9b 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java @@ -18,20 +18,19 @@ package org.apache.flink.cep.nfa; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ListMultimap; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.cep.Event; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.util.TestLogger; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; import org.junit.Test; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.util.Collection; import java.util.Collections; @@ -39,6 +38,9 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +/** + * Tests for {@link SharedBuffer}. + */ public class SharedBufferTest extends TestLogger { @Test diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java index 26b8ce997b2f3..cd120716c3fb7 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java @@ -32,6 +32,7 @@ import org.apache.flink.cep.pattern.Pattern; import org.apache.flink.cep.pattern.conditions.SimpleCondition; import org.apache.flink.util.TestLogger; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -45,6 +46,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +/** + * Tests for {@link NFACompiler}. + */ public class NFACompilerTest extends TestLogger { private static final SimpleCondition startFilter = new SimpleCondition() { @@ -116,7 +120,7 @@ public boolean filter(Event value) throws Exception { } /** - * Tests that the NFACompiler generates the correct NFA from a given Pattern + * Tests that the NFACompiler generates the correct NFA from a given Pattern. */ @Test public void testNFACompilerWithSimplePattern() { diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java index fb05901bde9ee..8608f7646b5e1 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.cep.operator; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; @@ -33,6 +34,7 @@ import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OperatorSnapshotUtil; + import org.junit.Ignore; import org.junit.Test; @@ -439,7 +441,6 @@ public Integer getKey(Event value) throws Exception { harness.close(); } - @Test public void testSinglePatternAfterMigration() throws Exception { diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java index d575e4309941f..d48f3d76ff269 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java @@ -35,6 +35,7 @@ import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; + import org.junit.Test; import java.net.URL; @@ -45,6 +46,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +/** + * Tests for migration from 1.1.x to 1.3.x. + */ public class CEPMigration11to13Test { private static String getResourceFilename(String filename) { @@ -193,7 +197,7 @@ public void testNonKeyedCEPFunctionMigration() throws Exception { final Event startEvent = new Event(42, "start", 1.0); final SubEvent middleEvent = new SubEvent(42, "foo", 1.0, 10.0); - final Event endEvent= new Event(42, "end", 1.0); + final Event endEvent = new Event(42, "end", 1.0); // uncomment these lines for regenerating the snapshot on Flink 1.1 /* diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java index ab63479595139..37e4309a16f62 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java @@ -18,7 +18,6 @@ package org.apache.flink.cep.operator; -import com.google.common.collect.Lists; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.base.IntSerializer; @@ -42,13 +41,13 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.types.Either; import org.apache.flink.util.TestLogger; + +import com.google.common.collect.Lists; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import static org.junit.Assert.*; - import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -57,6 +56,12 @@ import java.util.Map; import java.util.Queue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link KeyedCEPPatternOperator} and {@link TimeoutKeyedCEPPatternOperator}. + */ public class CEPOperatorTest extends TestLogger { @Rule @@ -87,7 +92,7 @@ public void testKeyedCEPOperatorCheckpointing() throws Exception { Event startEvent = new Event(42, "start", 1.0); SubEvent middleEvent = new SubEvent(42, "foo", 1.0, 10.0); - Event endEvent= new Event(42, "end", 1.0); + Event endEvent = new Event(42, "end", 1.0); harness.processElement(new StreamRecord<>(startEvent, 1L)); harness.processElement(new StreamRecord<>(new Event(42, "foobar", 1.0), 2L)); @@ -153,7 +158,7 @@ public void testKeyedCEPOperatorCheckpointingWithRocksDB() throws Exception { Event startEvent = new Event(42, "start", 1.0); SubEvent middleEvent = new SubEvent(42, "foo", 1.0, 10.0); - Event endEvent= new Event(42, "end", 1.0); + Event endEvent = new Event(42, "end", 1.0); harness.processElement(new StreamRecord<>(startEvent, 1L)); harness.processElement(new StreamRecord<>(new Event(42, "foobar", 1.0), 2L)); @@ -261,7 +266,7 @@ public void testKeyedAdvancingTimeWithoutElements() throws Exception { assertTrue(resultObject instanceof StreamRecord); StreamRecord>, Long>, Map>>> streamRecord = - (StreamRecord>,Long>,Map>>>) resultObject; + (StreamRecord>, Long>, Map>>>) resultObject; assertTrue(streamRecord.getValue() instanceof Either.Left); @@ -291,8 +296,8 @@ public void testCEPOperatorCleanupEventTime() throws Exception { SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0); SubEvent middleEvent2 = new SubEvent(42, "foo2", 1.0, 10.0); SubEvent middleEvent3 = new SubEvent(42, "foo3", 1.0, 10.0); - Event endEvent1 = new Event(42, "end", 1.0); - Event endEvent2 = new Event(42, "end", 2.0); + Event endEvent1 = new Event(42, "end", 1.0); + Event endEvent2 = new Event(42, "end", 2.0); Event startEventK2 = new Event(43, "start", 1.0); @@ -476,8 +481,8 @@ public void testCEPOperatorCleanupProcessingTime() throws Exception { SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0); SubEvent middleEvent2 = new SubEvent(42, "foo2", 1.0, 10.0); SubEvent middleEvent3 = new SubEvent(42, "foo3", 1.0, 10.0); - Event endEvent1 = new Event(42, "end", 1.0); - Event endEvent2 = new Event(42, "end", 2.0); + Event endEvent1 = new Event(42, "end", 1.0); + Event endEvent2 = new Event(42, "end", 2.0); Event startEventK2 = new Event(43, "start", 1.0); @@ -720,7 +725,6 @@ private void compareMaps(List> actual, List> expected) { Assert.assertArrayEquals(expected.toArray(), actual.toArray()); } - private class ListEventComparator implements Comparator> { @Override diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java index 45d7215ba47dd..d6bba3ee573f4 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java @@ -35,6 +35,7 @@ import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; + import org.junit.Test; import java.util.List; @@ -44,6 +45,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +/** + * Tests for rescaling of CEP operators. + */ public class CEPRescalingTest { @Test @@ -64,7 +68,7 @@ public Integer getKey(Event value) throws Exception { Event startEvent1 = new Event(7, "start", 1.0); SubEvent middleEvent1 = new SubEvent(7, "foo", 1.0, 10.0); - Event endEvent1= new Event(7, "end", 1.0); + Event endEvent1 = new Event(7, "end", 1.0); int keygroup = KeyGroupRangeAssignment.assignToKeyGroup(keySelector.getKey(startEvent1), maxParallelism); assertEquals(1, keygroup); diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java index b6fb48418d905..e00384bcc810b 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java @@ -25,15 +25,23 @@ import org.apache.flink.cep.pattern.conditions.SimpleCondition; import org.apache.flink.cep.pattern.conditions.SubtypeCondition; import org.apache.flink.util.TestLogger; + import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +/** + * Tests for constructing {@link Pattern}. + */ public class PatternTest extends TestLogger { + /** - * These test simply test that the pattern construction completes without failure + * These test simply test that the pattern construction completes without failure. */ - @Test public void testStrictContiguity() { Pattern pattern = Pattern.begin("start").next("next").next("end");