From 14edb2657bc6c884a0cfffb4b2226bbaea7a74ea Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 7 Feb 2018 13:55:11 +0100 Subject: [PATCH 1/2] [FLINK-8725] Separate state from NFA in CEP library This also changes the serialization of state to not include the static NFA parts and to also not include any user code. --- .../flink/cep/nfa/ComputationState.java | 114 +-- .../java/org/apache/flink/cep/nfa/NFA.java | 679 +++++------------- .../org/apache/flink/cep/nfa/NFAState.java | 144 ++++ .../flink/cep/nfa/NFAStateSerializer.java | 267 +++++++ .../nfa/NFAStateSerializerConfigSnapshot.java | 45 ++ .../apache/flink/cep/nfa/SharedBuffer.java | 9 +- .../flink/cep/nfa/compiler/NFACompiler.java | 7 +- .../AbstractKeyedCEPPatternOperator.java | 67 +- .../conditions/IterativeCondition.java | 2 +- .../org/apache/flink/cep/nfa/GroupITCase.java | 6 +- .../org/apache/flink/cep/nfa/NFAITCase.java | 66 +- .../flink/cep/nfa/NFAStatusChangeITCase.java | 75 +- .../org/apache/flink/cep/nfa/NFATest.java | 68 +- .../flink/cep/nfa/NFATestUtilities.java | 28 +- .../flink/cep/nfa/SameElementITCase.java | 12 +- .../flink/cep/nfa/UntilConditionITCase.java | 60 +- .../cep/nfa/compiler/NFACompilerTest.java | 3 +- .../flink/cep/operator/CEPMigrationTest.java | 2 +- .../flink/cep/operator/CEPOperatorTest.java | 56 +- ...gration-after-branching-flink1.5-snapshot} | Bin 18866 -> 17433 bytes ...ep-migration-conditions-flink1.5-snapshot} | Bin 19503 -> 17117 bytes ...ngle-pattern-afterwards-flink1.5-snapshot} | Bin 16848 -> 16850 bytes ...on-starting-new-pattern-flink1.5-snapshot} | Bin 19058 -> 17241 bytes 23 files changed, 906 insertions(+), 804 deletions(-) create mode 100644 flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java create mode 100644 flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java create mode 100644 flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java rename flink-libraries/flink-cep/src/test/resources/{cep-migration-starting-new-pattern-flink1.4-snapshot => cep-migration-after-branching-flink1.5-snapshot} (80%) rename flink-libraries/flink-cep/src/test/resources/{cep-migration-conditions-flink1.4-snapshot => cep-migration-conditions-flink1.5-snapshot} (77%) rename flink-libraries/flink-cep/src/test/resources/{cep-migration-single-pattern-afterwards-flink1.4-snapshot => cep-migration-single-pattern-afterwards-flink1.5-snapshot} (90%) rename flink-libraries/flink-cep/src/test/resources/{cep-migration-after-branching-flink1.4-snapshot => cep-migration-starting-new-pattern-flink1.5-snapshot} (79%) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java index 88ef3d3288a7b..c9aef27a30d1a 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java @@ -18,13 +18,8 @@ package org.apache.flink.cep.nfa; -import org.apache.flink.cep.pattern.conditions.IterativeCondition; -import org.apache.flink.util.Preconditions; +import javax.annotation.Nullable; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; import java.util.Objects; /** @@ -36,7 +31,7 @@ */ public class ComputationState { // pointer to the NFA state of the computation - private final State state; + private final String state; // the last taken event private final T event; @@ -52,14 +47,12 @@ public class ComputationState { // Timestamp of the first element in the pattern private final long startTimestamp; - private final State previousState; - - private final ConditionContext conditionContext; + @Nullable + private final String previousState; private ComputationState( - final NFA nfa, - final State currentState, - final State previousState, + final String currentState, + @Nullable final String previousState, final T event, final int counter, final long timestamp, @@ -72,25 +65,12 @@ private ComputationState( this.version = version; this.startTimestamp = startTimestamp; this.previousState = previousState; - this.conditionContext = new ConditionContext(nfa, this); } public int getCounter() { return counter; } - public ConditionContext getConditionContext() { - return conditionContext; - } - - public boolean isFinalState() { - return state.isFinal(); - } - - public boolean isStartState() { - return state.isStart() && event == null; - } - public long getTimestamp() { return timestamp; } @@ -99,11 +79,12 @@ public long getStartTimestamp() { return startTimestamp; } - public State getState() { + public String getState() { return state; } - public State getPreviousState() { + @Nullable + public String getPreviousState() { return previousState; } @@ -137,85 +118,22 @@ public int hashCode() { return Objects.hash(state, event, counter, timestamp, version, startTimestamp, previousState); } - public static ComputationState createStartState(final NFA nfa, final State state) { - Preconditions.checkArgument(state.isStart()); - return new ComputationState<>(nfa, state, null, null, 0, -1L, new DeweyNumber(1), -1L); + public static ComputationState createStartState(final String state) { + return new ComputationState<>(state, null, null, 0, -1L, new DeweyNumber(1), -1L); } - public static ComputationState createStartState(final NFA nfa, final State state, final DeweyNumber version) { - Preconditions.checkArgument(state.isStart()); - return new ComputationState<>(nfa, state, null, null, 0, -1L, version, -1L); + public static ComputationState createStartState(final String state, final DeweyNumber version) { + return new ComputationState(state, null, null, 0, -1L, version, -1L); } public static ComputationState createState( - final NFA nfa, - final State currentState, - final State previousState, + final String currentState, + final String previousState, final T event, final int counter, final long timestamp, final DeweyNumber version, final long startTimestamp) { - return new ComputationState<>(nfa, currentState, previousState, event, counter, timestamp, version, startTimestamp); - } - - public boolean isStopState() { - return state.isStop(); - } - - /** - * The context used when evaluating this computation state. - */ - public class ConditionContext implements IterativeCondition.Context { - - private static final long serialVersionUID = -6733978464782277795L; - - /** - * A flag indicating if we should recompute the matching pattern, so that - * the {@link IterativeCondition iterative condition} can be evaluated. - */ - private boolean shouldUpdate; - - /** The current computation state. */ - private transient ComputationState computationState; - - /** The owning {@link NFA} of this computation state. */ - private final NFA nfa; - - /** - * The matched pattern so far. A condition will be evaluated over this - * pattern. This is evaluated only once, as this is an expensive - * operation that traverses a path in the {@link SharedBuffer}. - */ - private transient Map> matchedEvents; - - public ConditionContext(NFA nfa, ComputationState computationState) { - this.nfa = nfa; - this.computationState = computationState; - this.shouldUpdate = true; - } - - @Override - public Iterable getEventsForPattern(final String key) { - Preconditions.checkNotNull(key); - - // the (partially) matched pattern is computed lazily when this method is called. - // this is to avoid any overheads when using a simple, non-iterative condition. - - if (shouldUpdate) { - this.matchedEvents = nfa.extractCurrentMatches(computationState); - shouldUpdate = false; - } - - return new Iterable() { - @Override - public Iterator iterator() { - List elements = matchedEvents.get(key); - return elements == null - ? Collections.EMPTY_LIST.iterator() - : elements.iterator(); - } - }; - } + return new ComputationState<>(currentState, previousState, event, counter, timestamp, version, startTimestamp); } } 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 5624db9de435a..7324db31cc7c1 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,44 +18,25 @@ package org.apache.flink.cep.nfa; -import org.apache.flink.api.common.typeutils.CompatibilityResult; -import org.apache.flink.api.common.typeutils.CompatibilityUtil; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; -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.LongSerializer; -import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.cep.nfa.compiler.NFAStateNameHandler; import org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator; import org.apache.flink.cep.pattern.conditions.IterativeCondition; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataInputViewStreamWrapper; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.util.Preconditions; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -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.Iterator; import java.util.LinkedHashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Queue; import java.util.Set; import java.util.Stack; @@ -85,16 +66,14 @@ * @see * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf */ -public class NFA implements Serializable { - - private static final long serialVersionUID = 2957674889294717265L; +public class NFA { /** * A set of all the valid NFA states, as returned by the * {@link NFACompiler NFACompiler}. * These are directly derived from the user-specified pattern. */ - private Set> states; + private final Map> states; /** * The length of a windowed pattern, as specified using the @@ -109,82 +88,76 @@ public class NFA implements Serializable { */ private final boolean handleTimeout; - /** - * Current set of {@link ComputationState computation states} within the state machine. - * These are the "active" intermediate states that are waiting for new matching - * events to transition to new valid states. - */ - private transient Queue> computationStates; - - /** - * Buffer used to store the matched events. - */ - private SharedBuffer eventSharedBuffer; + private final TypeSerializer eventSerializer; - private TypeSerializer eventSerializer; - - /** - * Flag indicating whether the matching status of the state machine has changed. - */ - private boolean nfaChanged; - - public NFA(final TypeSerializer eventSerializer, + public NFA( + final TypeSerializer eventSerializer, final long windowTime, - final boolean handleTimeout) { + final boolean handleTimeout, + final Collection> states) { this.eventSerializer = checkNotNull(eventSerializer); this.windowTime = windowTime; this.handleTimeout = handleTimeout; - this.eventSharedBuffer = new SharedBuffer<>(); - this.computationStates = new LinkedList<>(); - this.states = new HashSet<>(); - this.nfaChanged = false; + + this.states = new HashMap<>(); + for (State state : states) { + this.states.put(state.getName(), state); + } } - public Set> getStates() { - return states; + @VisibleForTesting + public Collection> getStates() { + return states.values(); } - public void addStates(final Collection> newStates) { - for (State state: newStates) { - addState(state); + public NFAState createNFAState() { + List> startingStates = new ArrayList<>(); + for (State state : states.values()) { + if (state.isStart()) { + startingStates.add(ComputationState.createStartState(state.getName())); + } } + return new NFAState<>(startingStates); } - public void addState(final State state) { - states.add(state); + private State getState(String state) { + State result = states.get(state); + return result; + } - if (state.isStart()) { - computationStates.add(ComputationState.createStartState(this, state)); - } + private State getState(ComputationState state) { + State result = states.get(state.getState()); + return result; } - /** - * Check if the NFA has finished processing all incoming data so far. That is - * when the buffer keeping the matches is empty. - * - * @return {@code true} if there are no elements in the {@link SharedBuffer}, - * {@code false} otherwise. - */ - public boolean isEmpty() { - return eventSharedBuffer.isEmpty(); + private boolean isStartState(ComputationState state) { + State stateObject = getState(state); + if (stateObject == null) { + throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values()); + } + + return stateObject.isStart(); } - /** - * Check if the matching status of the NFA has changed so far. - * - * @return {@code true} if matching status has changed, {@code false} otherwise - */ - public boolean isNFAChanged() { - return nfaChanged; + private boolean isStopState(ComputationState state) { + State stateObject = getState(state); + if (stateObject == null) { + throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values()); + } + + return stateObject.isStop(); } - /** - * Reset {@link #nfaChanged} to {@code false}. - */ - public void resetNFAChanged() { - this.nfaChanged = false; + private boolean isFinalState(ComputationState state) { + State stateObject = getState(state); + if (stateObject == null) { + throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values()); + } + + return stateObject.isFinal(); } + /** * Processes the next input event. If some of the computations reach a final state then the * resulting event sequences are returned. If computations time out and timeout handling is @@ -193,14 +166,16 @@ public void resetNFAChanged() { *

If computations reach a stop state, the path forward is discarded and currently constructed path is returned * with the element that resulted in the stop state. * + * @param nfaState The NFAState object that we need to affect while processing * @param event The current event to be processed or null if only pruning shall be done * @param timestamp The timestamp of the current event * @return Tuple of the collection of matched patterns (e.g. the result of computations which have * reached a final state) and the collection of timed out patterns (if timeout handling is * activated) */ - public Tuple2>>, Collection>, Long>>> process(final T event, final long timestamp) { - return process(event, timestamp, AfterMatchSkipStrategy.noSkip()); + public Tuple2>>, Collection>, Long>>> process( + NFAState nfaState, final T event, final long timestamp) { + return process(nfaState, event, timestamp, AfterMatchSkipStrategy.noSkip()); } /** @@ -211,6 +186,7 @@ public Tuple2>>, CollectionIf computations reach a stop state, the path forward is discarded and currently constructed path is returned * with the element that resulted in the stop state. * + * @param nfaState The NFAState object that we need to affect while processing * @param event The current event to be processed or null if only pruning shall be done * @param timestamp The timestamp of the current event * @param afterMatchSkipStrategy The skip strategy to use after per match @@ -218,8 +194,15 @@ public Tuple2>>, Collection>>, Collection>, Long>>> process(final T event, - final long timestamp, AfterMatchSkipStrategy afterMatchSkipStrategy) { + public Tuple2>>, Collection>, Long>>> process( + NFAState nfaState, + final T event, + final long timestamp, + AfterMatchSkipStrategy afterMatchSkipStrategy) { + + Queue> computationStates = nfaState.getComputationStates(); + SharedBuffer eventSharedBuffer = nfaState.getEventSharedBuffer(); + final int numberComputationStates = computationStates.size(); final Collection>> result = new ArrayList<>(); final Collection>, Long>> timeoutResult = new ArrayList<>(); @@ -230,31 +213,31 @@ public Tuple2>>, Collection> newComputationStates; - if (!computationState.isStartState() && + if (!isStartState(computationState) && windowTime > 0L && timestamp - computationState.getStartTimestamp() >= windowTime) { if (handleTimeout) { // extract the timed out event pattern - Map> timedOutPattern = extractCurrentMatches(computationState); + Map> timedOutPattern = extractCurrentMatches(eventSharedBuffer, computationState); timeoutResult.add(Tuple2.of(timedOutPattern, timestamp)); } eventSharedBuffer.release( - NFAStateNameHandler.getOriginalNameFromInternal(computationState.getPreviousState().getName()), + NFAStateNameHandler.getOriginalNameFromInternal(computationState.getPreviousState()), computationState.getEvent(), computationState.getTimestamp(), computationState.getCounter()); newComputationStates = Collections.emptyList(); - nfaChanged = true; + nfaState.setStateChanged(true); } else if (event != null) { - newComputationStates = computeNextStates(computationState, event, timestamp); + newComputationStates = computeNextStates(eventSharedBuffer, computationState, event, timestamp); if (newComputationStates.size() != 1) { - nfaChanged = true; + nfaState.setStateChanged(true); } else if (!newComputationStates.iterator().next().equals(computationState)) { - nfaChanged = true; + nfaState.setStateChanged(true); } } else { newComputationStates = Collections.singleton(computationState); @@ -266,24 +249,24 @@ public Tuple2>>, Collection newComputationState: newComputationStates) { - if (newComputationState.isFinalState()) { + if (isFinalState(newComputationState)) { // we've reached a final state and can thus retrieve the matching event sequence - Map> matchedPattern = extractCurrentMatches(newComputationState); + Map> matchedPattern = extractCurrentMatches(eventSharedBuffer, newComputationState); result.add(matchedPattern); // remove found patterns because they are no longer needed eventSharedBuffer.release( NFAStateNameHandler.getOriginalNameFromInternal( - newComputationState.getPreviousState().getName()), + newComputationState.getPreviousState()), newComputationState.getEvent(), newComputationState.getTimestamp(), newComputationState.getCounter()); - } else if (newComputationState.isStopState()) { + } else if (isStopState(newComputationState)) { //reached stop state. release entry for the stop state shouldDiscardPath = true; eventSharedBuffer.release( NFAStateNameHandler.getOriginalNameFromInternal( - newComputationState.getPreviousState().getName()), + newComputationState.getPreviousState()), newComputationState.getEvent(), newComputationState.getTimestamp(), newComputationState.getCounter()); @@ -299,7 +282,7 @@ public Tuple2>>, Collection state : statesToRetain) { eventSharedBuffer.release( NFAStateNameHandler.getOriginalNameFromInternal( - state.getPreviousState().getName()), + state.getPreviousState()), state.getEvent(), state.getTimestamp(), state.getCounter()); @@ -310,7 +293,8 @@ public Tuple2>>, Collection 0L) { @@ -322,7 +306,7 @@ public Tuple2>>, Collection>>, Collection> computationStates, + private void discardComputationStatesAccordingToStrategy( + SharedBuffer eventSharedBuffer, + Queue> computationStates, Collection>> matchedResult, AfterMatchSkipStrategy afterMatchSkipStrategy) { Set discardEvents = new HashSet<>(); switch(afterMatchSkipStrategy.getStrategy()) { @@ -368,14 +354,14 @@ private void discardComputationStatesAccordingToStrategy(Queue> discardStates = new ArrayList<>(); for (ComputationState computationState : computationStates) { - Map> partialMatch = extractCurrentMatches(computationState); + Map> partialMatch = extractCurrentMatches(eventSharedBuffer, computationState); for (List list: partialMatch.values()) { for (T e: list) { if (discardEvents.contains(e)) { // discard the computation state. eventSharedBuffer.release( NFAStateNameHandler.getOriginalNameFromInternal( - computationState.getState().getName()), + computationState.getState()), computationState.getEvent(), computationState.getTimestamp(), computationState.getCounter() @@ -390,25 +376,6 @@ private void discardComputationStatesAccordingToStrategy(Queue other = (NFA) obj; - - return eventSharedBuffer.equals(other.eventSharedBuffer) && - states.equals(other.states) && - windowTime == other.windowTime; - } else { - return false; - } - } - - @Override - public int hashCode() { - return Objects.hash(eventSharedBuffer, states, windowTime); - } - private static boolean isEquivalentState(final State s1, final State s2) { return s1.getName().equals(s2.getName()); } @@ -487,17 +454,19 @@ private boolean isSelfIgnore(final StateTransition edge) { *

  • Release the corresponding entries in {@link SharedBuffer}.
  • * * + * @param eventSharedBuffer The shared buffer that we need to change * @param computationState Current computation state * @param event Current event which is processed * @param timestamp Timestamp of the current event * @return Collection of computation states which result from the current one */ private Collection> computeNextStates( + final SharedBuffer eventSharedBuffer, final ComputationState computationState, final T event, final long timestamp) { - final OutgoingEdges outgoingEdges = createDecisionGraph(computationState, event); + final OutgoingEdges outgoingEdges = createDecisionGraph(eventSharedBuffer, computationState, event); // Create the computing version based on the previously computed edges // We need to defer the creation of computation states until we know how many edges start @@ -511,9 +480,9 @@ private Collection> computeNextStates( for (StateTransition edge : edges) { switch (edge.getAction()) { case IGNORE: { - if (!computationState.isStartState()) { + if (!isStartState(computationState)) { final DeweyNumber version; - if (isEquivalentState(edge.getTargetState(), computationState.getState())) { + if (isEquivalentState(edge.getTargetState(), getState(computationState))) { //Stay in the same state (it can be either looping one or singleton) final int toIncrease = calculateIncreasingSelfState( outgoingEdges.getTotalIgnoreBranches(), @@ -528,9 +497,10 @@ private Collection> computeNextStates( } addComputationState( + eventSharedBuffer, resultingComputationStates, edge.getTargetState(), - computationState.getPreviousState(), + getState(computationState.getPreviousState()), computationState.getEvent(), computationState.getCounter(), computationState.getTimestamp(), @@ -543,7 +513,7 @@ private Collection> computeNextStates( case TAKE: final State nextState = edge.getTargetState(); final State currentState = edge.getSourceState(); - final State previousState = computationState.getPreviousState(); + final State previousState = getState(computationState.getPreviousState()); final T previousEvent = computationState.getEvent(); @@ -553,7 +523,7 @@ private Collection> computeNextStates( final int counter; final long startTimestamp; - if (computationState.isStartState()) { + if (isStartState(computationState)) { startTimestamp = timestamp; counter = eventSharedBuffer.put( NFAStateNameHandler.getOriginalNameFromInternal( @@ -577,6 +547,7 @@ private Collection> computeNextStates( } addComputationState( + eventSharedBuffer, resultingComputationStates, nextState, currentState, @@ -587,9 +558,10 @@ private Collection> computeNextStates( startTimestamp); //check if newly created state is optional (have a PROCEED path to Final state) - final State finalState = findFinalStateAfterProceed(nextState, event, computationState); + final State finalState = findFinalStateAfterProceed(eventSharedBuffer, nextState, event, computationState); if (finalState != null) { addComputationState( + eventSharedBuffer, resultingComputationStates, finalState, currentState, @@ -603,13 +575,13 @@ private Collection> computeNextStates( } } - if (computationState.isStartState()) { + if (isStartState(computationState)) { int totalBranches = calculateIncreasingSelfState( outgoingEdges.getTotalIgnoreBranches(), outgoingEdges.getTotalTakeBranches()); DeweyNumber startVersion = computationState.getVersion().increase(totalBranches); - ComputationState startState = ComputationState.createStartState(this, computationState.getState(), startVersion); + ComputationState startState = ComputationState.createStartState(computationState.getState(), startVersion); resultingComputationStates.add(startState); } @@ -617,7 +589,7 @@ private Collection> computeNextStates( // release the shared entry referenced by the current computation state. eventSharedBuffer.release( NFAStateNameHandler.getOriginalNameFromInternal( - computationState.getPreviousState().getName()), + computationState.getPreviousState()), computationState.getEvent(), computationState.getTimestamp(), computationState.getCounter()); @@ -627,6 +599,7 @@ private Collection> computeNextStates( } private void addComputationState( + SharedBuffer eventSharedBuffer, List> computationStates, State currentState, State previousState, @@ -636,14 +609,17 @@ private void addComputationState( DeweyNumber version, long startTimestamp) { ComputationState computationState = ComputationState.createState( - this, currentState, previousState, event, counter, timestamp, version, startTimestamp); + currentState.getName(), previousState.getName(), event, counter, timestamp, version, startTimestamp); computationStates.add(computationState); String originalStateName = NFAStateNameHandler.getOriginalNameFromInternal(previousState.getName()); eventSharedBuffer.lock(originalStateName, event, timestamp, counter); } - private State findFinalStateAfterProceed(State state, T event, ComputationState computationState) { + private State findFinalStateAfterProceed( + SharedBuffer eventSharedBuffer, + State state, T event, + ComputationState computationState) { final Stack> statesToCheck = new Stack<>(); statesToCheck.push(state); @@ -652,7 +628,7 @@ private State findFinalStateAfterProceed(State state, T event, Computation final State currentState = statesToCheck.pop(); for (StateTransition transition : currentState.getStateTransitions()) { if (transition.getAction() == StateTransitionAction.PROCEED && - checkFilterCondition(computationState, transition.getCondition(), event)) { + checkFilterCondition(eventSharedBuffer, computationState, transition.getCondition(), event)) { if (transition.getTargetState().isFinal()) { return transition.getTargetState(); } else { @@ -672,11 +648,14 @@ private int calculateIncreasingSelfState(int ignoreBranches, int takeBranches) { return takeBranches == 0 && ignoreBranches == 0 ? 0 : ignoreBranches + Math.max(1, takeBranches); } - private OutgoingEdges createDecisionGraph(ComputationState computationState, T event) { - final OutgoingEdges outgoingEdges = new OutgoingEdges<>(computationState.getState()); + private OutgoingEdges createDecisionGraph( + SharedBuffer eventSharedBuffer, + ComputationState computationState, T event) { + State state = getState(computationState); + final OutgoingEdges outgoingEdges = new OutgoingEdges<>(state); final Stack> states = new Stack<>(); - states.push(computationState.getState()); + states.push(state); //First create all outgoing edges, so to be able to reason about the Dewey version while (!states.isEmpty()) { @@ -686,7 +665,7 @@ private OutgoingEdges createDecisionGraph(ComputationState computationStat // check all state transitions for each state for (StateTransition stateTransition : stateTransitions) { try { - if (checkFilterCondition(computationState, stateTransition.getCondition(), event)) { + if (checkFilterCondition(eventSharedBuffer, computationState, stateTransition.getCondition(), event)) { // filter condition is true switch (stateTransition.getAction()) { case PROCEED: @@ -708,8 +687,12 @@ private OutgoingEdges createDecisionGraph(ComputationState computationStat return outgoingEdges; } - private boolean checkFilterCondition(ComputationState computationState, IterativeCondition condition, T event) throws Exception { - return condition == null || condition.filter(event, computationState.getConditionContext()); + private boolean checkFilterCondition( + SharedBuffer eventSharedBuffer, + ComputationState computationState, + IterativeCondition condition, + T event) throws Exception { + return condition == null || condition.filter(event, new ConditionContext<>(this, eventSharedBuffer, computationState)); } /** @@ -717,17 +700,19 @@ private boolean checkFilterCondition(ComputationState computationState, Itera * sequence is returned as a map which contains the events and the names of the states to which * the events were mapped. * + * @param eventSharedBuffer The {@link SharedBuffer} from which to extract the matches * @param computationState The end computation state of the extracted event sequences * @return Collection of event sequences which end in the given computation state */ - Map> extractCurrentMatches(final ComputationState computationState) { + Map> extractCurrentMatches( + SharedBuffer eventSharedBuffer, final ComputationState computationState) { if (computationState.getPreviousState() == null) { return new HashMap<>(); } List>> paths = eventSharedBuffer.extractPatterns( NFAStateNameHandler.getOriginalNameFromInternal( - computationState.getPreviousState().getName()), + computationState.getPreviousState()), computationState.getEvent(), computationState.getTimestamp(), computationState.getCounter(), @@ -758,392 +743,62 @@ Map> extractCurrentMatches(final ComputationState computation return result; } - ////////////////////// Serialization ////////////////////// - - /** - * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state. - */ - public static final class NFASerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { - - private static final int VERSION = 1; - - /** This empty constructor is required for deserializing the configuration. */ - public NFASerializerConfigSnapshot() {} - - public NFASerializerConfigSnapshot( - TypeSerializer eventSerializer, - TypeSerializer> sharedBufferSerializer) { - - super(eventSerializer, sharedBufferSerializer); - } - - @Override - public int getVersion() { - return VERSION; - } - } - /** - * A {@link TypeSerializer} for {@link NFA} that uses Java Serialization. + * The context used when evaluating this computation state. */ - public static class NFASerializer extends TypeSerializer> { - - private static final long serialVersionUID = 2098282423980597010L; - - private final TypeSerializer> sharedBufferSerializer; - - private final TypeSerializer eventSerializer; - - public NFASerializer(TypeSerializer typeSerializer) { - this(typeSerializer, new SharedBuffer.SharedBufferSerializer<>(StringSerializer.INSTANCE, typeSerializer)); - } - - public NFASerializer( - TypeSerializer typeSerializer, - TypeSerializer> sharedBufferSerializer) { - this.eventSerializer = typeSerializer; - this.sharedBufferSerializer = sharedBufferSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public NFASerializer duplicate() { - return new NFASerializer<>(eventSerializer.duplicate()); - } - - @Override - public NFA createInstance() { - return null; - } - - @Override - public NFA copy(NFA from) { - try { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serialize(from, new DataOutputViewStreamWrapper(baos)); - baos.close(); - - byte[] data = baos.toByteArray(); - - ByteArrayInputStream bais = new ByteArrayInputStream(data); - NFA copy = deserialize(new DataInputViewStreamWrapper(bais)); - bais.close(); - return copy; - } catch (IOException e) { - throw new RuntimeException("Could not copy NFA.", e); - } - } - - @Override - public NFA copy(NFA from, NFA reuse) { - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - 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); + public static class ConditionContext implements IterativeCondition.Context { - target.writeInt(record.computationStates.size()); + /** + * A flag indicating if we should recompute the matching pattern, so that + * the {@link IterativeCondition iterative condition} can be evaluated. + */ + private boolean shouldUpdate; - StringSerializer stateNameSerializer = StringSerializer.INSTANCE; - LongSerializer timestampSerializer = LongSerializer.INSTANCE; - DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer(); + /** The current computation state. */ + private ComputationState computationState; - for (ComputationState computationState: record.computationStates) { - stateNameSerializer.serialize(computationState.getState().getName(), target); - stateNameSerializer.serialize(computationState.getPreviousState() == null - ? null : computationState.getPreviousState().getName(), target); + /** + * The matched pattern so far. A condition will be evaluated over this + * pattern. This is evaluated only once, as this is an expensive + * operation that traverses a path in the {@link SharedBuffer}. + */ + private Map> matchedEvents; - timestampSerializer.serialize(computationState.getTimestamp(), target); - versionSerializer.serialize(computationState.getVersion(), target); - timestampSerializer.serialize(computationState.getStartTimestamp(), target); - target.writeInt(computationState.getCounter()); + private NFA nfa; - if (computationState.getEvent() == null) { - target.writeBoolean(false); - } else { - target.writeBoolean(true); - eventSerializer.serialize(computationState.getEvent(), target); - } - } - } - - @Override - 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<>(); - StringSerializer stateNameSerializer = StringSerializer.INSTANCE; - LongSerializer timestampSerializer = LongSerializer.INSTANCE; - DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer(); - - int computationStateNo = source.readInt(); - for (int i = 0; i < computationStateNo; i++) { - State state = getStateByName(stateNameSerializer.deserialize(source), nfa); - State prevState = getStateByName(stateNameSerializer.deserialize(source), nfa); - long timestamp = timestampSerializer.deserialize(source); - DeweyNumber version = versionSerializer.deserialize(source); - long startTimestamp = timestampSerializer.deserialize(source); - int counter = source.readInt(); - - T event = null; - if (source.readBoolean()) { - event = eventSerializer.deserialize(source); - } - - computationStates.add(ComputationState.createState( - nfa, state, prevState, event, counter, timestamp, version, startTimestamp)); - } - - nfa.computationStates = computationStates; - return nfa; - } - - private State getStateByName(String name, NFA nfa) { - for (State state: nfa.states) { - if (state.getName().equals(name)) { - return state; - } - } - return null; - } - - @Override - public NFA deserialize(NFA reuse, DataInputView source) throws IOException { - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - Set> states = deserializeStates(source); - serializeStates(states, target); - - long windowTime = source.readLong(); - target.writeLong(windowTime); - - boolean handleTimeout = source.readBoolean(); - target.writeBoolean(handleTimeout); - - SharedBuffer sharedBuffer = sharedBufferSerializer.deserialize(source); - sharedBufferSerializer.serialize(sharedBuffer, target); - - StringSerializer stateNameSerializer = StringSerializer.INSTANCE; - LongSerializer timestampSerializer = LongSerializer.INSTANCE; - DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer(); - - int computationStateNo = source.readInt(); - target.writeInt(computationStateNo); - - for (int i = 0; i < computationStateNo; i++) { - String stateName = stateNameSerializer.deserialize(source); - stateNameSerializer.serialize(stateName, target); - - String prevStateName = stateNameSerializer.deserialize(source); - stateNameSerializer.serialize(prevStateName, target); - - long timestamp = timestampSerializer.deserialize(source); - timestampSerializer.serialize(timestamp, target); - - DeweyNumber version = versionSerializer.deserialize(source); - versionSerializer.serialize(version, target); - - long startTimestamp = timestampSerializer.deserialize(source); - timestampSerializer.serialize(startTimestamp, target); - - int counter = source.readInt(); - target.writeInt(counter); - - boolean hasEvent = source.readBoolean(); - target.writeBoolean(hasEvent); - if (hasEvent) { - T event = eventSerializer.deserialize(source); - eventSerializer.serialize(event, target); - } - } - } - - @Override - public boolean equals(Object obj) { - return obj == this || - (obj != null && obj.getClass().equals(getClass()) && - sharedBufferSerializer.equals(((NFASerializer) obj).sharedBufferSerializer) && - eventSerializer.equals(((NFASerializer) obj).eventSerializer)); - } - - @Override - public boolean canEqual(Object obj) { - return true; - } + private SharedBuffer eventSharedBuffer; - @Override - public int hashCode() { - return 37 * sharedBufferSerializer.hashCode() + eventSerializer.hashCode(); + public ConditionContext( + NFA nfa, + SharedBuffer eventSharedBuffer, + ComputationState computationState) { + this.computationState = computationState; + this.nfa = nfa; + this.eventSharedBuffer = eventSharedBuffer; + this.shouldUpdate = true; } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { - return new NFASerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer); - } + public Iterable getEventsForPattern(final String key) { + Preconditions.checkNotNull(key); - @Override - public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - if (configSnapshot instanceof NFASerializerConfigSnapshot) { - List, TypeSerializerConfigSnapshot>> serializersAndConfigs = - ((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); - - CompatibilityResult eventCompatResult = CompatibilityUtil.resolveCompatibilityResult( - serializersAndConfigs.get(0).f0, - UnloadableDummyTypeSerializer.class, - serializersAndConfigs.get(0).f1, - eventSerializer); - - CompatibilityResult> sharedBufCompatResult = - CompatibilityUtil.resolveCompatibilityResult( - serializersAndConfigs.get(1).f0, - UnloadableDummyTypeSerializer.class, - serializersAndConfigs.get(1).f1, - sharedBufferSerializer); - - if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) { - return CompatibilityResult.compatible(); - } else { - if (eventCompatResult.getConvertDeserializer() != null && - sharedBufCompatResult.getConvertDeserializer() != null) { - return CompatibilityResult.requiresMigration( - new NFASerializer<>( - new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()), - new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer()))); - } - } - } - - return CompatibilityResult.requiresMigration(); - } + // the (partially) matched pattern is computed lazily when this method is called. + // this is to avoid any overheads when using a simple, non-iterative condition. - private void serializeStates(Set> states, DataOutputView out) throws IOException { - TypeSerializer nameSerializer = StringSerializer.INSTANCE; - TypeSerializer stateTypeSerializer = new EnumSerializer<>(State.StateType.class); - TypeSerializer actionSerializer = new EnumSerializer<>(StateTransitionAction.class); - - out.writeInt(states.size()); - for (State state: states) { - nameSerializer.serialize(state.getName(), out); - stateTypeSerializer.serialize(state.getStateType(), out); + if (shouldUpdate) { + this.matchedEvents = nfa.extractCurrentMatches(eventSharedBuffer, computationState); + shouldUpdate = false; } - for (State state: states) { - nameSerializer.serialize(state.getName(), out); - - out.writeInt(state.getStateTransitions().size()); - for (StateTransition transition : state.getStateTransitions()) { - nameSerializer.serialize(transition.getSourceState().getName(), out); - nameSerializer.serialize(transition.getTargetState().getName(), out); - actionSerializer.serialize(transition.getAction(), out); - - serializeCondition(transition.getCondition(), out); + return new Iterable() { + @Override + public Iterator iterator() { + List elements = matchedEvents.get(key); + return elements == null + ? Collections.EMPTY_LIST.iterator() + : elements.iterator(); } - } - } - - private Set> deserializeStates(DataInputView in) throws IOException { - TypeSerializer nameSerializer = StringSerializer.INSTANCE; - TypeSerializer stateTypeSerializer = new EnumSerializer<>(State.StateType.class); - TypeSerializer actionSerializer = new EnumSerializer<>(StateTransitionAction.class); - - final int noOfStates = in.readInt(); - Map> states = new HashMap<>(noOfStates); - - for (int i = 0; i < noOfStates; i++) { - String stateName = nameSerializer.deserialize(in); - State.StateType stateType = stateTypeSerializer.deserialize(in); - - State state = new State<>(stateName, stateType); - states.put(stateName, state); - } - - for (int i = 0; i < noOfStates; i++) { - String srcName = nameSerializer.deserialize(in); - - int noOfTransitions = in.readInt(); - for (int j = 0; j < noOfTransitions; j++) { - String src = nameSerializer.deserialize(in); - Preconditions.checkState(src.equals(srcName), - "Source Edge names do not match (" + srcName + " - " + src + ")."); - - String trgt = nameSerializer.deserialize(in); - StateTransitionAction action = actionSerializer.deserialize(in); - - IterativeCondition condition = null; - try { - condition = deserializeCondition(in); - } catch (ClassNotFoundException e) { - e.printStackTrace(); - } - - State srcState = states.get(src); - State trgtState = states.get(trgt); - srcState.addStateTransition(action, trgtState, condition); - } - - } - return new HashSet<>(states.values()); - } - - private void serializeCondition(IterativeCondition condition, DataOutputView out) throws IOException { - out.writeBoolean(condition != null); - if (condition != null) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(baos); - - oos.writeObject(condition); - - oos.close(); - baos.close(); - - byte[] serCondition = baos.toByteArray(); - out.writeInt(serCondition.length); - out.write(serCondition); - } - } - - private IterativeCondition deserializeCondition(DataInputView in) throws IOException, ClassNotFoundException { - boolean hasCondition = in.readBoolean(); - if (hasCondition) { - int length = in.readInt(); - - byte[] serCondition = new byte[length]; - in.readFully(serCondition); - - ByteArrayInputStream bais = new ByteArrayInputStream(serCondition); - ObjectInputStream ois = new ObjectInputStream(bais); - - IterativeCondition condition = (IterativeCondition) ois.readObject(); - ois.close(); - bais.close(); - - return condition; - } - return null; + }; } } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java new file mode 100644 index 0000000000000..e9414c396655f --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java @@ -0,0 +1,144 @@ +/* + * 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.operator.AbstractKeyedCEPPatternOperator; + +import java.util.LinkedList; +import java.util.Queue; + +/** + * State kept for a {@link NFA}. + * + *

    The {@link 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 + * {@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. + *
    + * + *

    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 NFAState { + + /** + * Current set of {@link ComputationState computation states} within the state machine. + * These are the "active" intermediate states that are waiting for new matching + * events to transition to new valid states. + */ + private Queue> computationStates; + + /** + * Buffer used to store the matched events. + */ + private SharedBuffer eventSharedBuffer; + + /** + * Flag indicating whether the matching status of the state machine has changed. + */ + private boolean stateChanged; + + public NFAState( + Queue> computationStates, + SharedBuffer eventSharedBuffer, + boolean stateChanged) { + this.computationStates = computationStates; + this.eventSharedBuffer = eventSharedBuffer; + this.stateChanged = stateChanged; + } + + public NFAState() { + this(new LinkedList<>(), new SharedBuffer<>(), false); + } + + public NFAState(Iterable> startingStates) { + this(); + + for (ComputationState startingState : startingStates) { + computationStates.add(startingState); + } + } + + /** + * Check if the NFA has finished processing all incoming data so far. That is + * when the buffer keeping the matches is empty. + * + * @return {@code true} if there are no elements in the {@link SharedBuffer}, + * {@code false} otherwise. + */ + public boolean isEmpty() { + return eventSharedBuffer.isEmpty(); + } + + /** + * Check if the matching status of the NFA has changed so far. + * + * @return {@code true} if matching status has changed, {@code false} otherwise + */ + public boolean isStateChanged() { + return stateChanged; + } + + /** + * Reset the changed bit checked via {@link #isStateChanged()} to {@code false}. + */ + public void resetStateChanged() { + this.stateChanged = false; + } + + public void setStateChanged(boolean stateChanged) { + this.stateChanged = stateChanged; + } + + public Queue> getComputationStates() { + return computationStates; + } + + public SharedBuffer getEventSharedBuffer() { + return eventSharedBuffer; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof NFAState) { + @SuppressWarnings("unchecked") + NFAState other = (NFAState) obj; + + return eventSharedBuffer.equals(other.eventSharedBuffer); + } else { + return false; + } + } + + @Override + public int hashCode() { + return eventSharedBuffer.hashCode(); + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java new file mode 100644 index 0000000000000..4dae7986f438d --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java @@ -0,0 +1,267 @@ +/* + * 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.api.common.typeutils.CompatibilityResult; +import org.apache.flink.api.common.typeutils.CompatibilityUtil; +import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + +/** + * A {@link TypeSerializer} for {@link NFAState} that uses Java Serialization. + */ +public class NFAStateSerializer extends TypeSerializer> { + + private static final long serialVersionUID = 2098282423980597010L; + + private final TypeSerializer> sharedBufferSerializer; + + private final TypeSerializer eventSerializer; + + public NFAStateSerializer(TypeSerializer typeSerializer) { + this(typeSerializer, new SharedBuffer.SharedBufferSerializer<>(StringSerializer.INSTANCE, typeSerializer)); + } + + public NFAStateSerializer( + TypeSerializer typeSerializer, + TypeSerializer> sharedBufferSerializer) { + this.eventSerializer = typeSerializer; + this.sharedBufferSerializer = sharedBufferSerializer; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public NFAStateSerializer duplicate() { + return new NFAStateSerializer<>(eventSerializer.duplicate()); + } + + @Override + public NFAState createInstance() { + return null; + } + + @Override + public NFAState copy(NFAState from) { + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serialize(from, new DataOutputViewStreamWrapper(baos)); + baos.close(); + + byte[] data = baos.toByteArray(); + + ByteArrayInputStream bais = new ByteArrayInputStream(data); + NFAState copy = deserialize(new DataInputViewStreamWrapper(bais)); + bais.close(); + return copy; + } catch (IOException e) { + throw new RuntimeException("Could not copy NFA.", e); + } + } + + @Override + public NFAState copy(NFAState from, NFAState reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(NFAState record, DataOutputView target) throws IOException { + sharedBufferSerializer.serialize(record.getEventSharedBuffer(), target); + + target.writeInt(record.getComputationStates().size()); + + StringSerializer stateNameSerializer = StringSerializer.INSTANCE; + LongSerializer timestampSerializer = LongSerializer.INSTANCE; + DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer(); + + for (ComputationState computationState: record.getComputationStates()) { + stateNameSerializer.serialize(computationState.getState(), target); + stateNameSerializer.serialize(computationState.getPreviousState(), target); + + timestampSerializer.serialize(computationState.getTimestamp(), target); + versionSerializer.serialize(computationState.getVersion(), target); + timestampSerializer.serialize(computationState.getStartTimestamp(), target); + target.writeInt(computationState.getCounter()); + + if (computationState.getEvent() == null) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + eventSerializer.serialize(computationState.getEvent(), target); + } + } + } + + @Override + public NFAState deserialize(DataInputView source) throws IOException { + SharedBuffer sharedBuffer = sharedBufferSerializer.deserialize(source); + + Queue> computationStates = new LinkedList<>(); + StringSerializer stateNameSerializer = StringSerializer.INSTANCE; + LongSerializer timestampSerializer = LongSerializer.INSTANCE; + DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer(); + + int computationStateNo = source.readInt(); + for (int i = 0; i < computationStateNo; i++) { + String state = stateNameSerializer.deserialize(source); + String prevState = stateNameSerializer.deserialize(source); + long timestamp = timestampSerializer.deserialize(source); + DeweyNumber version = versionSerializer.deserialize(source); + long startTimestamp = timestampSerializer.deserialize(source); + int counter = source.readInt(); + + T event = null; + if (source.readBoolean()) { + event = eventSerializer.deserialize(source); + } + + computationStates.add(ComputationState.createState( + state, prevState, event, counter, timestamp, version, startTimestamp)); + } + + return new NFAState<>(computationStates, sharedBuffer, false); + } + + @Override + public NFAState deserialize(NFAState reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + SharedBuffer sharedBuffer = sharedBufferSerializer.deserialize(source); + sharedBufferSerializer.serialize(sharedBuffer, target); + + StringSerializer stateNameSerializer = StringSerializer.INSTANCE; + LongSerializer timestampSerializer = LongSerializer.INSTANCE; + DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer(); + + int computationStateNo = source.readInt(); + target.writeInt(computationStateNo); + + for (int i = 0; i < computationStateNo; i++) { + String stateName = stateNameSerializer.deserialize(source); + stateNameSerializer.serialize(stateName, target); + + String prevStateName = stateNameSerializer.deserialize(source); + stateNameSerializer.serialize(prevStateName, target); + + long timestamp = timestampSerializer.deserialize(source); + timestampSerializer.serialize(timestamp, target); + + DeweyNumber version = versionSerializer.deserialize(source); + versionSerializer.serialize(version, target); + + long startTimestamp = timestampSerializer.deserialize(source); + timestampSerializer.serialize(startTimestamp, target); + + int counter = source.readInt(); + target.writeInt(counter); + + boolean hasEvent = source.readBoolean(); + target.writeBoolean(hasEvent); + if (hasEvent) { + T event = eventSerializer.deserialize(source); + eventSerializer.serialize(event, target); + } + } + } + + @Override + public boolean equals(Object obj) { + return obj == this || + (obj != null && obj.getClass().equals(getClass()) && + sharedBufferSerializer.equals(((NFAStateSerializer) obj).sharedBufferSerializer) && + eventSerializer.equals(((NFAStateSerializer) obj).eventSerializer)); + } + + @Override + public boolean canEqual(Object obj) { + return true; + } + + @Override + public int hashCode() { + return 37 * sharedBufferSerializer.hashCode() + eventSerializer.hashCode(); + } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + return new NFAStateSerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer); + } + + @Override + public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof NFAStateSerializerConfigSnapshot) { + List, TypeSerializerConfigSnapshot>> serializersAndConfigs = + ((NFAStateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + + CompatibilityResult eventCompatResult = CompatibilityUtil.resolveCompatibilityResult( + serializersAndConfigs.get(0).f0, + UnloadableDummyTypeSerializer.class, + serializersAndConfigs.get(0).f1, + eventSerializer); + + CompatibilityResult> sharedBufCompatResult = + CompatibilityUtil.resolveCompatibilityResult( + serializersAndConfigs.get(1).f0, + UnloadableDummyTypeSerializer.class, + serializersAndConfigs.get(1).f1, + sharedBufferSerializer); + + if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) { + return CompatibilityResult.compatible(); + } else { + if (eventCompatResult.getConvertDeserializer() != null && + sharedBufCompatResult.getConvertDeserializer() != null) { + return CompatibilityResult.requiresMigration( + new NFAStateSerializer<>( + new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()), + new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer()))); + } + } + } + + return CompatibilityResult.requiresMigration(); + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java new file mode 100644 index 0000000000000..ab72348d527c7 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java @@ -0,0 +1,45 @@ +/* + * 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.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +/** + * {@link CompositeTypeSerializerConfigSnapshot} for {@link NFAStateSerializer}. + */ +public final class NFAStateSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty constructor is required for deserializing the configuration. */ + public NFAStateSerializerConfigSnapshot() {} + + public NFAStateSerializerConfigSnapshot( + TypeSerializer eventSerializer, + TypeSerializer> sharedBufferSerializer) { + + super(eventSerializer, sharedBufferSerializer); + } + + @Override + public int getVersion() { + return VERSION; + } +} 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 adfa737937604..a00a310392cd2 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 @@ -36,7 +36,6 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -67,11 +66,9 @@ * @param Type of the keys * @param Type of the values */ -public class SharedBuffer implements Serializable { +public class SharedBuffer { - private static final long serialVersionUID = 9213251042562206495L; - - private transient Map> pages; + private Map> pages; public SharedBuffer() { this.pages = new HashMap<>(4); @@ -807,7 +804,7 @@ public int getVersion() { /** * A {@link TypeSerializer} for the {@link SharedBuffer}. */ - public static class SharedBufferSerializer extends TypeSerializer> { + public static class SharedBufferSerializer extends TypeSerializer> { private static final long serialVersionUID = -3254176794680331560L; 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 39e8d34acef4d..5b9522b3e20ac 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 @@ -919,11 +919,8 @@ private NFAFactoryImpl( @Override public NFA createNFA() { - NFA result = new NFA<>(inputTypeSerializer.duplicate(), windowTime, timeoutHandling); - - result.addStates(states); - - return result; + return new NFA<>( + inputTypeSerializer.duplicate(), windowTime, timeoutHandling, states); } } } 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 2415a2e3a231a..448e36715357b 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 @@ -31,6 +31,8 @@ import org.apache.flink.cep.EventComparator; import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.NFA; +import org.apache.flink.cep.nfa.NFAState; +import org.apache.flink.cep.nfa.NFAStateSerializer; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.VoidNamespace; @@ -78,13 +80,15 @@ public abstract class AbstractKeyedCEPPatternOperator> nfaOperatorState; + private transient ValueState> nfaValueState; private transient MapState> elementQueueState; private final NFACompiler.NFAFactory nfaFactory; private transient InternalTimerService timerService; + private transient NFA nfa; + /** * The last seen watermark. This will be used to * decide if an incoming element is late or not. @@ -120,11 +124,11 @@ public AbstractKeyedCEPPatternOperator( public void initializeState(StateInitializationContext context) throws Exception { super.initializeState(context); - if (nfaOperatorState == null) { - nfaOperatorState = getRuntimeContext().getState( + if (nfaValueState == null) { + nfaValueState = getRuntimeContext().getState( new ValueStateDescriptor<>( NFA_OPERATOR_STATE_NAME, - new NFA.NFASerializer<>(inputSerializer))); + new NFAStateSerializer<>(inputSerializer))); } if (elementQueueState == null) { @@ -146,6 +150,8 @@ public void open() throws Exception { "watermark-callbacks", VoidNamespaceSerializer.INSTANCE, this); + + this.nfa = nfaFactory.createNFA(); } @Override @@ -153,9 +159,9 @@ public void processElement(StreamRecord element) throws Exception { if (isProcessingTime) { if (comparator == null) { // there can be no out of order elements in processing time - NFA nfa = getNFA(); - processEvent(nfa, element.getValue(), getProcessingTimeService().getCurrentProcessingTime()); - updateNFA(nfa); + NFAState nfaState = getNFAState(); + processEvent(nfaState, element.getValue(), getProcessingTimeService().getCurrentProcessingTime()); + updateNFA(nfaState); } else { long currentTime = timerService.currentProcessingTime(); bufferEvent(element.getValue(), currentTime); @@ -226,27 +232,27 @@ public void onEventTime(InternalTimer timer) throws Exceptio // STEP 1 PriorityQueue sortedTimestamps = getSortedTimestamps(); - NFA nfa = getNFA(); + NFAState nfaState = getNFAState(); // STEP 2 while (!sortedTimestamps.isEmpty() && sortedTimestamps.peek() <= timerService.currentWatermark()) { long timestamp = sortedTimestamps.poll(); sort(elementQueueState.get(timestamp)).forEachOrdered( - event -> processEvent(nfa, event, timestamp) + event -> processEvent(nfaState, event, timestamp) ); elementQueueState.remove(timestamp); } // STEP 3 - advanceTime(nfa, timerService.currentWatermark()); + advanceTime(nfaState, timerService.currentWatermark()); // STEP 4 if (sortedTimestamps.isEmpty()) { elementQueueState.clear(); } - updateNFA(nfa); + updateNFA(nfaState); - if (!sortedTimestamps.isEmpty() || !nfa.isEmpty()) { + if (!sortedTimestamps.isEmpty() || !nfaState.isEmpty()) { saveRegisterWatermarkTimer(); } @@ -264,7 +270,7 @@ public void onProcessingTime(InternalTimer timer) throws Exc // STEP 1 PriorityQueue sortedTimestamps = getSortedTimestamps(); - NFA nfa = getNFA(); + NFAState nfa = getNFAState(); // STEP 2 while (!sortedTimestamps.isEmpty()) { @@ -295,18 +301,18 @@ private void updateLastSeenWatermark(long timestamp) { this.lastWatermark = timestamp; } - private NFA getNFA() throws IOException { - NFA nfa = nfaOperatorState.value(); - return nfa != null ? nfa : nfaFactory.createNFA(); + private NFAState getNFAState() throws IOException { + NFAState nfaState = nfaValueState.value(); + return nfaState != null ? nfaState : nfa.createNFAState(); } - private void updateNFA(NFA nfa) throws IOException { - if (nfa.isNFAChanged()) { - if (nfa.isEmpty()) { - nfaOperatorState.clear(); + private void updateNFA(NFAState nfaState) throws IOException { + if (nfaState.isStateChanged()) { + if (nfaState.isEmpty()) { + nfaValueState.clear(); } else { - nfa.resetNFAChanged(); - nfaOperatorState.update(nfa); + nfaState.resetStateChanged(); + nfaValueState.update(nfaState); } } } @@ -323,13 +329,13 @@ private PriorityQueue getSortedTimestamps() throws Exception { * Process the given event by giving it to the NFA and outputting the produced set of matched * event sequences. * - * @param nfa NFA to be used for the event detection + * @param nfaState Our NFAState object * @param event The current event to be processed * @param timestamp The timestamp of the event */ - private void processEvent(NFA nfa, IN event, long timestamp) { + private void processEvent(NFAState nfaState, IN event, long timestamp) { Tuple2>>, Collection>, Long>>> patterns = - nfa.process(event, timestamp, afterMatchSkipStrategy); + nfa.process(nfaState, event, timestamp, afterMatchSkipStrategy); try { processMatchedSequences(patterns.f0, timestamp); @@ -343,12 +349,9 @@ private void processEvent(NFA nfa, IN event, long timestamp) { /** * Advances the time for the given NFA to the given timestamp. This can lead to pruning and * timeouts. - * - * @param nfa to advance the time for - * @param timestamp to advance the time to */ - private void advanceTime(NFA nfa, long timestamp) throws Exception { - processEvent(nfa, null, timestamp); + private void advanceTime(NFAState nfaState, long timestamp) throws Exception { + processEvent(nfaState, null, timestamp); } protected abstract void processMatchedSequences(Iterable>> matchingSequences, long timestamp) throws Exception; @@ -361,9 +364,9 @@ protected void processTimedOutSequences( ////////////////////// Testing Methods ////////////////////// @VisibleForTesting - public boolean hasNonEmptyNFA(KEY key) throws IOException { + public boolean hasNonEmptyNFAState(KEY key) throws IOException { setCurrentKey(key); - return nfaOperatorState.value() != null; + return nfaValueState.value() != null; } @VisibleForTesting 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 e7c814f1a4718..0386d0e949485 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 @@ -84,7 +84,7 @@ public abstract class IterativeCondition implements Function, Serializable { /** * The context used when evaluating the {@link IterativeCondition condition}. */ - public interface Context extends Serializable { + public interface Context { /** * @return An {@link Iterable} over the already accepted elements diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java index c2c7cdacf3eb8..f9b9aa2bfa44e 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java @@ -1075,14 +1075,16 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(c, a1, b1, d), Lists.newArrayList(c, a1, b1, a2, b2, d) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } } 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 84278b15fd462..76bc2d04a181a 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 @@ -386,9 +386,11 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), true); + NFAState nfaState = nfa.createNFAState(); + for (StreamRecord event: events) { Tuple2>>, Collection>, Long>>> patterns = - nfa.process(event.getValue(), event.getTimestamp()); + nfa.process(nfaState, event.getValue(), event.getTimestamp()); Collection>> matchedPatterns = patterns.f0; Collection>, Long>> timeoutPatterns = patterns.f1; @@ -2317,16 +2319,18 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - nfa.process(startEvent, 1); - nfa.process(middleEvent1, 2); - nfa.process(middleEvent2, 3); - nfa.process(middleEvent3, 4); - nfa.process(end1, 6); + NFAState nfaState = nfa.createNFAState(); + + nfa.process(nfaState, startEvent, 1); + nfa.process(nfaState, middleEvent1, 2); + nfa.process(nfaState, middleEvent2, 3); + nfa.process(nfaState, middleEvent3, 4); + nfa.process(nfaState, end1, 6); //pruning element - nfa.process(null, 10); + nfa.process(nfaState, null, 10); - assertEquals(true, nfa.isEmpty()); + assertEquals(true, nfaState.isEmpty()); } @Test @@ -2360,14 +2364,16 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - nfa.process(startEvent, 1); - nfa.process(middleEvent, 5); - nfa.process(end1, 6); + NFAState nfaState = nfa.createNFAState(); + + nfa.process(nfaState, startEvent, 1); + nfa.process(nfaState, middleEvent, 5); + nfa.process(nfaState, end1, 6); //pruning element - nfa.process(null, 10); + nfa.process(nfaState, null, 10); - assertEquals(true, nfa.isEmpty()); + assertEquals(true, nfaState.isEmpty()); } @Test @@ -2402,15 +2408,17 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - nfa.process(startEvent, 1); - nfa.process(middleEvent1, 3); - nfa.process(middleEvent2, 4); - nfa.process(end1, 6); + NFAState nfaState = nfa.createNFAState(); + + nfa.process(nfaState, startEvent, 1); + nfa.process(nfaState, middleEvent1, 3); + nfa.process(nfaState, middleEvent2, 4); + nfa.process(nfaState, end1, 6); //pruning element - nfa.process(null, 10); + nfa.process(nfaState, null, 10); - assertEquals(true, nfa.isEmpty()); + assertEquals(true, nfaState.isEmpty()); } @Test @@ -2445,15 +2453,17 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - nfa.process(startEvent, 1); - nfa.process(middleEvent1, 3); - nfa.process(middleEvent2, 4); - nfa.process(end1, 6); + NFAState nfaState = nfa.createNFAState(); + + nfa.process(nfaState, startEvent, 1); + nfa.process(nfaState, middleEvent1, 3); + nfa.process(nfaState, middleEvent2, 4); + nfa.process(nfaState, end1, 6); //pruning element - nfa.process(null, 10); + nfa.process(nfaState, null, 10); - assertEquals(true, nfa.isEmpty()); + assertEquals(true, nfaState.isEmpty()); } /////////////////////////////////////// Skip till next ///////////////////////////// @@ -2703,8 +2713,11 @@ public boolean filter(Event s) throws Exception { List>> resultingPatterns = new ArrayList<>(); + NFAState nfaState = nfa.createNFAState(); + for (StreamRecord inputEvent : inputEvents) { Collection>> patterns = nfa.process( + nfaState, inputEvent.getValue(), inputEvent.getTimestamp()).f0; @@ -2774,8 +2787,11 @@ public boolean filter(Event s) throws Exception { List>> resultingPatterns = new ArrayList<>(); + NFAState nfaState = nfa.createNFAState(); + for (StreamRecord inputEvent : inputEvents) { Collection>> patterns = nfa.process( + nfaState, inputEvent.getValue(), inputEvent.getTimestamp()).f0; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java index 37ad0064bbe75..38796992bb37a 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java @@ -36,8 +36,7 @@ import static org.junit.Assert.assertTrue; /** - * Tests if the {@link NFA} status ({@link NFA#computationStates} or {@link NFA#eventSharedBuffer}) - * is changed after processing events. + * Tests if the {@link NFAState} status is changed after processing events. */ public class NFAStatusChangeITCase { @@ -76,45 +75,47 @@ public boolean filter(Event value, Context ctx) throws Exception { NFACompiler.NFAFactory nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true); NFA nfa = nfaFactory.createNFA(); - nfa.process(new Event(1, "b", 1.0), 1L); - assertFalse("NFA status should not change as the event does not match the take condition of the 'start' state", nfa.isNFAChanged()); + NFAState nfaState = nfa.createNFAState(); - nfa.resetNFAChanged(); - nfa.process(new Event(2, "a", 1.0), 2L); - assertTrue("NFA status should change as the event matches the take condition of the 'start' state", nfa.isNFAChanged()); + nfa.process(nfaState, new Event(1, "b", 1.0), 1L); + assertFalse("NFA status should not change as the event does not match the take condition of the 'start' state", nfaState.isStateChanged()); + + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(2, "a", 1.0), 2L); + assertTrue("NFA status should change as the event matches the take condition of the 'start' state", nfaState.isStateChanged()); // the status of the queue of ComputationStatus changed, // more than one ComputationStatus is generated by the event from some ComputationStatus - nfa.resetNFAChanged(); - nfa.process(new Event(3, "f", 1.0), 3L); - assertTrue("NFA status should change as the event matches the ignore condition and proceed condition of the 'middle:1' state", nfa.isNFAChanged()); + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(3, "f", 1.0), 3L); + assertTrue("NFA status should change as the event matches the ignore condition and proceed condition of the 'middle:1' state", nfaState.isStateChanged()); // both the queue of ComputationStatus and eventSharedBuffer have not changed - nfa.resetNFAChanged(); - nfa.process(new Event(4, "f", 1.0), 4L); - assertFalse("NFA status should not change as the event only matches the ignore condition of the 'middle:2' state and the target state is still 'middle:2'", nfa.isNFAChanged()); + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(4, "f", 1.0), 4L); + assertFalse("NFA status should not change as the event only matches the ignore condition of the 'middle:2' state and the target state is still 'middle:2'", nfaState.isStateChanged()); // both the queue of ComputationStatus and eventSharedBuffer have changed - nfa.resetNFAChanged(); - nfa.process(new Event(5, "b", 1.0), 5L); - assertTrue("NFA status should change as the event matches the take condition of 'middle:2' state", nfa.isNFAChanged()); + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(5, "b", 1.0), 5L); + assertTrue("NFA status should change as the event matches the take condition of 'middle:2' state", nfaState.isStateChanged()); // both the queue of ComputationStatus and eventSharedBuffer have changed - nfa.resetNFAChanged(); - nfa.process(new Event(6, "d", 1.0), 6L); - assertTrue("NFA status should change as the event matches the take condition of 'middle2' state", nfa.isNFAChanged()); + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(6, "d", 1.0), 6L); + assertTrue("NFA status should change as the event matches the take condition of 'middle2' state", nfaState.isStateChanged()); // both the queue of ComputationStatus and eventSharedBuffer have not changed // as the timestamp is within the window - nfa.resetNFAChanged(); - nfa.process(null, 8L); - assertFalse("NFA status should not change as the timestamp is within the window", nfa.isNFAChanged()); + nfaState.resetStateChanged(); + nfa.process(nfaState, null, 8L); + assertFalse("NFA status should not change as the timestamp is within the window", nfaState.isStateChanged()); // timeout ComputationStatus will be removed from the queue of ComputationStatus and timeout event will // be removed from eventSharedBuffer as the timeout happens - nfa.resetNFAChanged(); - Collection>, Long>> timeoutResults = nfa.process(null, 12L).f1; - assertTrue("NFA status should change as timeout happens", nfa.isNFAChanged() && !timeoutResults.isEmpty()); + nfaState.resetStateChanged(); + Collection>, Long>> timeoutResults = nfa.process(nfaState, null, 12L).f1; + assertTrue("NFA status should change as timeout happens", nfaState.isStateChanged() && !timeoutResults.isEmpty()); } @Test @@ -143,12 +144,14 @@ public boolean filter(Event value, Context ctx) throws Exception { NFACompiler.NFAFactory nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true); NFA nfa = nfaFactory.createNFA(); - nfa.resetNFAChanged(); - nfa.process(new Event(6, "start", 1.0), 6L); + NFAState nfaState = nfa.createNFAState(); + + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(6, "start", 1.0), 6L); - nfa.resetNFAChanged(); - nfa.process(new Event(6, "a", 1.0), 7L); - assertTrue(nfa.isNFAChanged()); + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(6, "a", 1.0), 7L); + assertTrue(nfaState.isStateChanged()); } @Test @@ -170,11 +173,13 @@ public boolean filter(Event value, Context ctx) throws Exception { NFACompiler.NFAFactory nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true); NFA nfa = nfaFactory.createNFA(); - nfa.resetNFAChanged(); - nfa.process(new Event(6, "start", 1.0), 6L); + NFAState nfaState = nfa.createNFAState(); + + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(6, "start", 1.0), 6L); - nfa.resetNFAChanged(); - nfa.process(new Event(6, "end", 1.0), 17L); - assertTrue(nfa.isNFAChanged()); + nfaState.resetStateChanged(); + nfa.process(nfaState, new Event(6, "end", 1.0), 17L); + assertTrue(nfaState.isStateChanged()); } } 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 7721653914f57..5d43111ba0e6a 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 @@ -52,7 +52,6 @@ public class NFATest extends TestLogger { @Test public void testSimpleNFA() { - NFA nfa = new NFA<>(Event.createTypeSerializer(), 0, false); List> streamEvents = new ArrayList<>(); streamEvents.add(new StreamRecord<>(new Event(1, "start", 1.0), 1L)); @@ -86,9 +85,12 @@ public boolean filter(Event value) throws Exception { }); endState.addIgnore(BooleanConditions.trueFunction()); - nfa.addState(startState); - nfa.addState(endState); - nfa.addState(endingState); + List> states = new ArrayList<>(); + states.add(startState); + states.add(endState); + states.add(endingState); + + NFA nfa = new NFA<>(Event.createTypeSerializer(), 0, false, states); Set>> expectedPatterns = new HashSet<>(); @@ -103,7 +105,7 @@ public boolean filter(Event value) throws Exception { expectedPatterns.add(firstPattern); expectedPatterns.add(secondPattern); - Collection>> actualPatterns = runNFA(nfa, streamEvents); + Collection>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents); assertEquals(expectedPatterns, actualPatterns); } @@ -126,7 +128,7 @@ public void testTimeoutWindowPruning() { expectedPatterns.add(secondPattern); - Collection>> actualPatterns = runNFA(nfa, streamEvents); + Collection>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents); assertEquals(expectedPatterns, actualPatterns); } @@ -145,7 +147,7 @@ public void testWindowBorders() { Set>> expectedPatterns = Collections.emptySet(); - Collection>> actualPatterns = runNFA(nfa, streamEvents); + Collection>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents); assertEquals(expectedPatterns, actualPatterns); } @@ -172,7 +174,7 @@ public void testTimeoutWindowPruningWindowBorders() { expectedPatterns.add(secondPattern); - Collection>> actualPatterns = runNFA(nfa, streamEvents); + Collection>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents); assertEquals(expectedPatterns, actualPatterns); } @@ -187,21 +189,25 @@ public void testTimeoutWindowPruning2() throws IOException { streamEvents.add(new StreamRecord<>(new Event(3, "loop", 3.0), 103L)); streamEvents.add(new StreamRecord<>(new Event(4, "loop", 4.0), 104L)); streamEvents.add(new StreamRecord<>(new Event(5, "loop", 5.0), 105L)); - runNFA(nfa, streamEvents); - NFA.NFASerializer serializer = new NFA.NFASerializer<>(Event.createTypeSerializer()); + NFAState nfaState = nfa.createNFAState(); + runNFA(nfa, nfaState, streamEvents); + + NFAStateSerializer serializer = new NFAStateSerializer<>(Event.createTypeSerializer()); //serialize ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serializer.serialize(nfa, new DataOutputViewStreamWrapper(baos)); + serializer.serialize(nfaState, new DataOutputViewStreamWrapper(baos)); baos.close(); } - public Collection>> runNFA(NFA nfa, List> inputs) { + public Collection>> runNFA( + NFA nfa, NFAState nfaState, List> inputs) { Set>> actualPatterns = new HashSet<>(); for (StreamRecord streamEvent : inputs) { Collection>> matchedPatterns = nfa.process( + nfaState, streamEvent.getValue(), streamEvent.getTimestamp()).f0; @@ -311,24 +317,26 @@ public boolean filter(Event value) throws Exception { Event b3 = new Event(41, "b", 5.0); Event d = new Event(43, "d", 4.0); - nfa.process(a, 1); - nfa.process(b, 2); - nfa.process(c, 3); - nfa.process(b1, 4); - nfa.process(b2, 5); - nfa.process(b3, 6); - nfa.process(d, 7); - nfa.process(a, 8); + NFAState nfaState = nfa.createNFAState(); + + nfa.process(nfaState, a, 1); + nfa.process(nfaState, b, 2); + nfa.process(nfaState, c, 3); + nfa.process(nfaState, b1, 4); + nfa.process(nfaState, b2, 5); + nfa.process(nfaState, b3, 6); + nfa.process(nfaState, d, 7); + nfa.process(nfaState, a, 8); - NFA.NFASerializer serializer = new NFA.NFASerializer<>(Event.createTypeSerializer()); + NFAStateSerializer serializer = new NFAStateSerializer<>(Event.createTypeSerializer()); //serialize ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serializer.serialize(nfa, new DataOutputViewStreamWrapper(baos)); + serializer.serialize(nfaState, new DataOutputViewStreamWrapper(baos)); baos.close(); // copy - NFA.NFASerializer copySerializer = new NFA.NFASerializer<>(Event.createTypeSerializer()); + NFAStateSerializer copySerializer = new NFAStateSerializer<>(Event.createTypeSerializer()); ByteArrayInputStream in = new ByteArrayInputStream(baos.toByteArray()); ByteArrayOutputStream out = new ByteArrayOutputStream(); copySerializer.duplicate().copy(new DataInputViewStreamWrapper(in), new DataOutputViewStreamWrapper(out)); @@ -337,15 +345,14 @@ public boolean filter(Event value) throws Exception { // deserialize ByteArrayInputStream bais = new ByteArrayInputStream(out.toByteArray()); - NFA copy = serializer.duplicate().deserialize(new DataInputViewStreamWrapper(bais)); + NFAState copy = serializer.duplicate().deserialize(new DataInputViewStreamWrapper(bais)); bais.close(); - assertEquals(nfa, copy); + assertEquals(nfaState, copy); } } private NFA createStartEndNFA(long windowLength) { - NFA nfa = new NFA<>(Event.createTypeSerializer(), windowLength, false); State startState = new State<>("start", State.StateType.Start); State endState = new State<>("end", State.StateType.Normal); @@ -373,11 +380,12 @@ public boolean filter(Event value) throws Exception { }); endState.addIgnore(BooleanConditions.trueFunction()); - nfa.addState(startState); - nfa.addState(endState); - nfa.addState(endingState); + List> states = new ArrayList<>(); + states.add(startState); + states.add(endState); + states.add(endingState); - return nfa; + return new NFA<>(Event.createTypeSerializer(), windowLength, false, states); } private NFA createLoopingNFA(long windowLength) { 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 index a9e17955ac8ba..b5312cbfbf710 100644 --- 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 @@ -35,16 +35,36 @@ */ public class NFATestUtilities { - public static List> feedNFA(List> inputEvents, NFA nfa) { - return feedNFA(inputEvents, nfa, AfterMatchSkipStrategy.noSkip()); + public static List> feedNFA( + List> inputEvents, + NFA nfa) { + return feedNFA(inputEvents, nfa, nfa.createNFAState(), AfterMatchSkipStrategy.noSkip()); } - public static List> feedNFA(List> inputEvents, NFA nfa, - AfterMatchSkipStrategy afterMatchSkipStrategy) { + public static List> feedNFA( + List> inputEvents, + NFA nfa, + NFAState nfaState) { + return feedNFA(inputEvents, nfa, nfaState, AfterMatchSkipStrategy.noSkip()); + } + + public static List> feedNFA( + List> inputEvents, + NFA nfa, + AfterMatchSkipStrategy afterMatchSkipStrategy) { + return feedNFA(inputEvents, nfa, nfa.createNFAState(), afterMatchSkipStrategy); + } + + public static List> feedNFA( + List> inputEvents, + NFA nfa, + NFAState nfaState, + AfterMatchSkipStrategy afterMatchSkipStrategy) { List> resultingPatterns = new ArrayList<>(); for (StreamRecord inputEvent : inputEvents) { Collection>> patterns = nfa.process( + nfaState, inputEvent.getValue(), inputEvent.getTimestamp(), afterMatchSkipStrategy).f0; 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 index 357107fce9cc8..c94d739d2aa67 100644 --- 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 @@ -140,11 +140,13 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(a1, b1, c1, d) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -182,13 +184,15 @@ public boolean filter(Event value, Context ctx) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(a1, d1, d2, d3), Lists.newArrayList(a1, d1, d2), Lists.newArrayList(a1, d1) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java index f88e5b21ab401..b60317436123c 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java @@ -91,13 +91,15 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking), Lists.newArrayList(startEvent, middleEvent1, breaking) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -137,7 +139,9 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, breaking), @@ -145,7 +149,7 @@ public boolean filter(Event value) throws Exception { Lists.newArrayList(startEvent, middleEvent1, middleEvent3, breaking), Lists.newArrayList(startEvent, middleEvent1, breaking) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -185,13 +189,15 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking), Lists.newArrayList(startEvent, middleEvent1, breaking) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -233,12 +239,14 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, breaking) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -278,14 +286,16 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking), Lists.newArrayList(startEvent, middleEvent1, breaking), Lists.newArrayList(startEvent, breaking) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -325,7 +335,9 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, breaking), @@ -334,7 +346,7 @@ public boolean filter(Event value) throws Exception { Lists.newArrayList(startEvent, middleEvent1, breaking), Lists.newArrayList(startEvent, breaking) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -374,14 +386,16 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking), Lists.newArrayList(startEvent, middleEvent1, breaking), Lists.newArrayList(startEvent, breaking) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -502,7 +516,9 @@ public boolean filter(Event value) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3), @@ -510,7 +526,7 @@ public boolean filter(Event value) throws Exception { Lists.newArrayList(startEvent, middleEvent1) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -553,7 +569,9 @@ public boolean filter(Event value, Context ctx) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3), @@ -561,7 +579,7 @@ public boolean filter(Event value, Context ctx) throws Exception { Lists.newArrayList(startEvent, middleEvent1) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } @Test @@ -604,7 +622,9 @@ public boolean filter(Event value, Context ctx) throws Exception { NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); - final List> resultingPatterns = feedNFA(inputEvents, nfa); + NFAState nfaState = nfa.createNFAState(); + + final List> resultingPatterns = feedNFA(inputEvents, nfa, nfaState); compareMaps(resultingPatterns, Lists.>newArrayList( Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3), @@ -613,6 +633,6 @@ public boolean filter(Event value, Context ctx) throws Exception { Lists.newArrayList(startEvent) )); - assertTrue(nfa.isEmpty()); + assertTrue(nfaState.isEmpty()); } } 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 ec2cf47578cf6..d1b6d5901bc86 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 @@ -40,6 +40,7 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -132,7 +133,7 @@ public void testNFACompilerWithSimplePattern() { NFA nfa = NFACompiler.compile(pattern, serializer, false); - Set> states = nfa.getStates(); + Collection> states = nfa.getStates(); assertEquals(4, states.size()); Map> stateMap = new HashMap<>(); diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java index 2fda47f9d3cbe..3151498feed7b 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java @@ -71,7 +71,7 @@ public class CEPMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters () { - return Arrays.asList(MigrationVersion.v1_3, MigrationVersion.v1_4); + return Arrays.asList(MigrationVersion.v1_5); } public CEPMigrationTest(MigrationVersion migrateVersion) { 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 322ee99fbfc0e..33a7e1156bf00 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 @@ -461,9 +461,9 @@ public void testKeyedCEPOperatorNFAUpdateTimes() throws Exception { try { harness.open(); - final ValueState nfaOperatorState = (ValueState) Whitebox.getInternalState(operator, "nfaOperatorState"); + final ValueState nfaOperatorState = (ValueState) Whitebox.getInternalState(operator, "nfaValueState"); final ValueState nfaOperatorStateSpy = Mockito.spy(nfaOperatorState); - Whitebox.setInternalState(operator, "nfaOperatorState", nfaOperatorStateSpy); + Whitebox.setInternalState(operator, "nfaValueState", nfaOperatorStateSpy); Event startEvent = new Event(42, "c", 1.0); SubEvent middleEvent = new SubEvent(42, "a", 1.0, 10.0); @@ -506,9 +506,9 @@ public void testKeyedCEPOperatorNFAUpdateTimesWithRocksDB() throws Exception { harness.open(); - final ValueState nfaOperatorState = (ValueState) Whitebox.getInternalState(operator, "nfaOperatorState"); + final ValueState nfaOperatorState = (ValueState) Whitebox.getInternalState(operator, "nfaValueState"); final ValueState nfaOperatorStateSpy = Mockito.spy(nfaOperatorState); - Whitebox.setInternalState(operator, "nfaOperatorState", nfaOperatorStateSpy); + Whitebox.setInternalState(operator, "nfaValueState", nfaOperatorStateSpy); Event startEvent = new Event(42, "c", 1.0); SubEvent middleEvent = new SubEvent(42, "a", 1.0, 10.0); @@ -567,8 +567,8 @@ public void testCEPOperatorCleanupEventTime() throws Exception { assertEquals(2L, harness.numEventTimeTimers()); assertEquals(4L, operator.getPQSize(42)); assertEquals(1L, operator.getPQSize(43)); - assertTrue(!operator.hasNonEmptyNFA(42)); - assertTrue(!operator.hasNonEmptyNFA(43)); + assertTrue(!operator.hasNonEmptyNFAState(42)); + assertTrue(!operator.hasNonEmptyNFAState(43)); harness.processWatermark(new Watermark(2L)); @@ -580,9 +580,9 @@ public void testCEPOperatorCleanupEventTime() throws Exception { // for 43 the element entered the NFA and the PQ is empty assertEquals(2L, harness.numEventTimeTimers()); - assertTrue(operator.hasNonEmptyNFA(42)); + assertTrue(operator.hasNonEmptyNFAState(42)); assertEquals(1L, operator.getPQSize(42)); - assertTrue(operator.hasNonEmptyNFA(43)); + assertTrue(operator.hasNonEmptyNFAState(43)); assertTrue(!operator.hasNonEmptyPQ(43)); harness.processElement(new StreamRecord<>(startEvent2, 4L)); @@ -604,9 +604,9 @@ public void testCEPOperatorCleanupEventTime() throws Exception { // now we have 1 key because the 43 expired and was removed. // 42 is still there due to startEvent2 assertEquals(1L, harness.numEventTimeTimers()); - assertTrue(operator2.hasNonEmptyNFA(42)); + assertTrue(operator2.hasNonEmptyNFAState(42)); assertTrue(!operator2.hasNonEmptyPQ(42)); - assertTrue(!operator2.hasNonEmptyNFA(43)); + assertTrue(!operator2.hasNonEmptyNFAState(43)); assertTrue(!operator2.hasNonEmptyPQ(43)); verifyPattern(harness.getOutput().poll(), startEvent1, middleEvent1, endEvent1); @@ -621,7 +621,7 @@ public void testCEPOperatorCleanupEventTime() throws Exception { harness.processWatermark(20L); harness.processWatermark(21L); - assertTrue(!operator2.hasNonEmptyNFA(42)); + assertTrue(!operator2.hasNonEmptyNFAState(42)); assertTrue(!operator2.hasNonEmptyPQ(42)); assertEquals(0L, harness.numEventTimeTimers()); @@ -664,7 +664,7 @@ public void testCEPOperatorCleanupEventTimeWithSameElements() throws Exception { assertEquals(1L, harness.numEventTimeTimers()); assertEquals(7L, operator.getPQSize(41)); - assertTrue(!operator.hasNonEmptyNFA(41)); + assertTrue(!operator.hasNonEmptyNFAState(41)); harness.processWatermark(new Watermark(2L)); @@ -673,7 +673,7 @@ public void testCEPOperatorCleanupEventTimeWithSameElements() throws Exception { assertEquals(1L, harness.numEventTimeTimers()); assertEquals(6L, operator.getPQSize(41)); - assertTrue(operator.hasNonEmptyNFA(41)); // processed the first element + assertTrue(operator.hasNonEmptyNFAState(41)); // processed the first element harness.processWatermark(new Watermark(8L)); @@ -713,12 +713,12 @@ public void testCEPOperatorCleanupEventTimeWithSameElements() throws Exception { assertEquals(1L, harness.numEventTimeTimers()); assertEquals(0L, operator.getPQSize(41)); - assertTrue(operator.hasNonEmptyNFA(41)); + assertTrue(operator.hasNonEmptyNFAState(41)); harness.processWatermark(new Watermark(17L)); verifyWatermark(harness.getOutput().poll(), 17L); - assertTrue(!operator.hasNonEmptyNFA(41)); + assertTrue(!operator.hasNonEmptyNFAState(41)); assertTrue(!operator.hasNonEmptyPQ(41)); assertEquals(0L, harness.numEventTimeTimers()); } finally { @@ -756,8 +756,8 @@ public void testCEPOperatorCleanupProcessingTime() throws Exception { assertTrue(!operator.hasNonEmptyPQ(42)); assertTrue(!operator.hasNonEmptyPQ(43)); - assertTrue(operator.hasNonEmptyNFA(42)); - assertTrue(operator.hasNonEmptyNFA(43)); + assertTrue(operator.hasNonEmptyNFAState(42)); + assertTrue(operator.hasNonEmptyNFAState(43)); harness.setProcessingTime(3L); @@ -790,10 +790,10 @@ public void testCEPOperatorCleanupProcessingTime() throws Exception { harness.setProcessingTime(21L); - assertTrue(operator2.hasNonEmptyNFA(42)); + assertTrue(operator2.hasNonEmptyNFAState(42)); harness.processElement(new StreamRecord<>(startEvent1, 21L)); - assertTrue(operator2.hasNonEmptyNFA(42)); + assertTrue(operator2.hasNonEmptyNFAState(42)); harness.setProcessingTime(49L); @@ -801,7 +801,7 @@ public void testCEPOperatorCleanupProcessingTime() throws Exception { harness.processElement(new StreamRecord<>(new Event(42, "foobar", 1.0), 2L)); // the pattern expired - assertTrue(!operator2.hasNonEmptyNFA(42)); + assertTrue(!operator2.hasNonEmptyNFAState(42)); assertEquals(0L, harness.numEventTimeTimers()); assertTrue(!operator2.hasNonEmptyPQ(42)); @@ -944,12 +944,12 @@ public void testCEPOperatorComparatorProcessTime() throws Exception { harness .processElement(new StreamRecord<>(new SubEvent(42, "barfoo", 1.0, 5.0), 0L)); - assertTrue(!operator.hasNonEmptyNFA(42)); - assertTrue(!operator.hasNonEmptyNFA(43)); + assertTrue(!operator.hasNonEmptyNFAState(42)); + assertTrue(!operator.hasNonEmptyNFAState(43)); harness.setProcessingTime(3L); - assertTrue(operator.hasNonEmptyNFA(42)); - assertTrue(operator.hasNonEmptyNFA(43)); + assertTrue(operator.hasNonEmptyNFAState(42)); + assertTrue(operator.hasNonEmptyNFAState(43)); harness.processElement(new StreamRecord<>(middleEvent2, 3L)); harness.processElement(new StreamRecord<>(middleEvent1, 3L)); @@ -1003,14 +1003,14 @@ public void testCEPOperatorComparatorEventTime() throws Exception { assertTrue(operator.hasNonEmptyPQ(42)); assertTrue(operator.hasNonEmptyPQ(43)); - assertTrue(!operator.hasNonEmptyNFA(42)); - assertTrue(!operator.hasNonEmptyNFA(43)); + assertTrue(!operator.hasNonEmptyNFAState(42)); + assertTrue(!operator.hasNonEmptyNFAState(43)); harness.processWatermark(3L); assertTrue(!operator.hasNonEmptyPQ(42)); assertTrue(!operator.hasNonEmptyPQ(43)); - assertTrue(operator.hasNonEmptyNFA(42)); - assertTrue(operator.hasNonEmptyNFA(43)); + assertTrue(operator.hasNonEmptyNFAState(42)); + assertTrue(operator.hasNonEmptyNFAState(43)); harness.processElement(new StreamRecord<>(startEvent2, 4L)); harness.processElement(new StreamRecord(middleEvent2, 5L)); diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot similarity index 80% rename from flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot rename to flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot index 660a21a77b04b798bf7db8423d071ca99a0a36ba..73934e565f52e86f176b8bec3d80c9c4e26ca80e 100644 GIT binary patch delta 368 zcmdlqnQ>+Z;{;WeR8wOULsOGv-IO#-BV7|iGZWp!#8gw=w4`LyWMfm)v24-m{rn*LnDWpdDJBMH7G`M%sSFH`^EOvA)k?8G zV_;CZwE3@0IU{G@nzszB#YGI-lM59^C;wGrXALe%EJ@uwSI&!r(Gpc~a-yB=WN%*1 z&3fwFrTCcGb2C#?a)8D%FflTKRT_B83jw(xzyzTf7~xWUlA9c$f(|NB283ds+~r~9 z$PDCwFbIHjgD@kA0|X!%Sq#ko4+Ky-5T9l8OOG^pmejlygu{TsU^7AHg48fh=I}IQ R2D)Riqo*Dt)8qnACjcvBOSAw0 delta 1556 zcmc&!O-vI(6n?uDN;Mu-Frg8{VU3ZvY+JY8Etn`>6X)YJ^G9adw!8jXeH;Yh}aXQcbd8um~h16(*B(+yRR z@p@bh^P((8cug0>d`OIuNI9a#+B?PshVOiX>YNMB_$k-m z@9AvkNXy^c*Hz2N8-HPeZ^S2e1oEK+kAkOt{$W$svjzaL+d0G1`;m$ajtm(e8PVT0 zIi$LXc(7ru=Xz^^A?xZ20Pu%Bb$CADn{ZTw4A*K&+m>!3)3$Og?UtTc8@-#`T|2jO z_c04>3Vc~3gAg1^+3ujAI%;~z5HeZQIwzzJNANNUrA$JSq*jqRYOLpLOF^}34i2I7 za0QrgmxWfsx}GlsKT7DQ);>vhPdR6X7QdObYzD z5e?5aHB`AE(QT!23Cv+9TZ%t^<3Z4u{QjB+7^j%oQ5PC%EbdkOy9)KDH77Ri5Lw&k zRFCaY-nQG4=;-Y>2Wbs#Yrx1MuDc*(rt1cu{1naV(F~V~?5|AIEox$xiK?s#-Ol;*Xu{gjr2Su0 zl-OJED$sLZAE{ik4ofcWMXz+e=z{?7 zJsAoYQLHQjd2(D-6yns_L=!V}U6V9pV_g%=lw@7Yl$1o>6pJ(qBTEzWG?OHgjh<{w zIw6b&gYNc48 zF)*lH+Wc3hoRKqc&07Z6;vxp^$%TrdlmDu*vj&$WmZWZ;E9b?*Xo)H~InhpbvNtd1 OWdcsiHYV644zvyS2NX0u|8#B zP`R-AuS_{3XYQJ}46MaP3|f;9sz|c?xjCu;@#cAQUL1@T$U>9#b!8`e^Kx!BP~R>U z$jnh(l2}w?WxxOg+$yPgDZwR)C8;Vv9wQrA3dmwYQp5;j!(|y5fhtZxZ8rv5C0bvQ zSW=Q&l&6=RpO=zZl9``Ztmj|k4C5#lpH(<2Zd1g>z~sZgl9QTNQo>;4lV6mspIDHX zoRO-ZmXn#6t)HA)pbs@jA7PNbC(z`?lFYJHxOvt-46H?&=@})340Q~Q6-5k;Rv=dx zV{rwh<|NT~AyyZUUT0!ps3<6&{7_6}@&Qg3Uo3hZ^HShe-rXe3vFXFwod_?0?M8J7 zNQem-BcR|HHNj$HaA{IWWdXt?*BXOfcxo4x-tre`;ROqj(ejXRdB_DT4_zrpoLtb-nTuia6UUzbqg>YJ diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot similarity index 90% rename from flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot rename to flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot index c87f307a574aa5a132d26d7caab9b242bfeaf9c1..63b1b4922493210fa42af54dd64f329945301c20 100644 GIT binary patch delta 164 zcmcc6%y_Apae}H!s%ctEqPdx+u7!oMsjf+qp@FWYS(>44im|b=p+%Bql5twE#+Jzj#=0q{7D>7$Mk$867MA9zx@jqf2FZy@X_m&RmJAH`DVy_{YNc48 zF)*lH+WbMLoRKqc&07Z6;vxp^$&A{flRv1jvj&$WmZWZ;Am_!wXo)H~S&>(5v!eQX FDF9wWFpvNM delta 162 zcmccA%y^-hae}Ifg@u`cp`}Hlu0>jsk*-NnQi^VpK~l1AQnI;es=0-wrI}IMM$ey& zIzfyKDrU)sscB{=M!Km6Nfx>$iKZsH7Rf0Fx@iVxrY0$gDaJ;Ii3|+(Nt^SSYNc48 zGBBuI*!)4JoRKqk&07Z6;vxpE$rDW_+5OxcRe*T&L^&^x$&;9bCSS1UfC)}k4w+yVsMGV@L3l&8t|5amW4K7J6N!>hG&WnT55>;?=qMhtyZ(h#L zdg|MyIGEUTGgDG>QYYtoT5&Ky0Mq0P9!9!MKn@6l03#5C*dQ7N82 Plli?A7#SxUcsT(8;vzrf delta 1644 zcmc&!PiWIn7=KB-byEbvO=)o|m?4U3(l%|n&BL0y>1H=|8y>u+O}YiUB&KhFL{x_0 zWtU+D5r!u(f}*a72p%TGlNUXx2)mFfc=#5 z`?}i`3>k%+yE<#A_}V3#V;k&aTWSmZzDI$RUf-amC^;1&L~mwPUFpR#RvDsFUZTNr zbsozWCNs3=tTT1aAc~x;D<)un++Jr-*Lp9RGRCT<2Q!AQXjn7!d@!La$rbJW)V7+b z#k-GbM5mB9r)Dumhf;>s&&j5o8Bn=wPSekD8P((*Pn_$C>maUWtPQKH{#sI~%F_A= z@L6;bQKJ@(nhEN9woDrT8mH?Q`kG%pdt)!Q1|WH%j8*SY8lN-WfUn1T<~yHoKWIdy zkS{-!wkP-aJr)untXR&0HfZKt`@`2C2D}~LUeO4RLeBKCg&M%tPa?Gqt9r5IP<8!>xVKJdY_udOjnV~b}dJWb4q%Cd1icea>aYKLzqh;ML0vV zS4BSnj+V(+GK`!m>#G&cGa=k@>wb-AdK|nuNJC{Tdn+TgO0;NYtSZw2;mkj~6RmV7 z{QuelVDFsU0Igpiu3WP=T`3hzUbw9+FHL;fFtp_Qp#cU5&wpj*^!ig@eH^IL%fmD$_t-9SfI;I6g8q_vJ=(Zo@=$t{)A_7-SQCXcr1?PfIa-zqHl zwZb-$R(K{|@z^0LobZqZ7$5{faX=+N;J Date: Fri, 4 May 2018 15:41:27 +0200 Subject: [PATCH 2/2] Reverted backward compatibility with <=1.5 --- .../java/org/apache/flink/cep/nfa/NFA.java | 270 +++++++++++++++++- .../flink/cep/nfa/NFASerializationUtils.java | 70 +++++ .../flink/cep/nfa/NFAStateSerializer.java | 47 +-- .../AbstractKeyedCEPPatternOperator.java | 29 +- .../flink/cep/operator/CEPMigrationTest.java | 2 +- ...igration-after-branching-flink1.4-snapshot | Bin 0 -> 19058 bytes ...igration-after-branching-flink1.5-snapshot | Bin 17433 -> 19390 bytes ...cep-migration-conditions-flink1.4-snapshot | Bin 0 -> 19503 bytes ...cep-migration-conditions-flink1.5-snapshot | Bin 17117 -> 19835 bytes ...ingle-pattern-afterwards-flink1.4-snapshot | Bin 0 -> 16848 bytes ...ingle-pattern-afterwards-flink1.5-snapshot | Bin 16850 -> 17180 bytes ...ion-starting-new-pattern-flink1.4-snapshot | Bin 0 -> 18866 bytes ...ion-starting-new-pattern-flink1.5-snapshot | Bin 17241 -> 19198 bytes .../api/operators/AbstractStreamOperator.java | 2 + 14 files changed, 378 insertions(+), 42 deletions(-) create mode 100644 flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java create mode 100644 flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot create mode 100644 flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot create mode 100644 flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot create mode 100644 flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot 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 7324db31cc7c1..bc28cc5809e18 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 @@ -19,15 +19,28 @@ package org.apache.flink.cep.nfa; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.CompatibilityResult; +import org.apache.flink.api.common.typeutils.CompatibilityUtil; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter; import org.apache.flink.api.common.typeutils.TypeSerializer; +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.StringSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.cep.nfa.compiler.NFAStateNameHandler; import org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator; import org.apache.flink.cep.pattern.conditions.IterativeCondition; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.util.Preconditions; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.ObjectInputStream; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -121,13 +134,11 @@ public NFAState createNFAState() { } private State getState(String state) { - State result = states.get(state); - return result; + return states.get(state); } private State getState(ComputationState state) { - State result = states.get(state.getState()); - return result; + return states.get(state.getState()); } private boolean isStartState(ComputationState state) { @@ -801,4 +812,255 @@ public Iterator iterator() { }; } } + + /** + * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state. + */ + @Deprecated + public static final class NFASerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty constructor is required for deserializing the configuration. */ + public NFASerializerConfigSnapshot() {} + + public NFASerializerConfigSnapshot( + TypeSerializer eventSerializer, + TypeSerializer> sharedBufferSerializer) { + + super(eventSerializer, sharedBufferSerializer); + } + + @Override + public int getVersion() { + return VERSION; + } + } + + /** + * Only for backward compatibility with <1.5. + */ + @Deprecated + public static class NFASerializer extends TypeSerializer> { + + private static final long serialVersionUID = 2098282423980597010L; + + private final TypeSerializer> sharedBufferSerializer; + + private final TypeSerializer eventSerializer; + + public NFASerializer(TypeSerializer typeSerializer) { + this(typeSerializer, new SharedBuffer.SharedBufferSerializer<>(StringSerializer.INSTANCE, typeSerializer)); + } + + NFASerializer( + TypeSerializer typeSerializer, + TypeSerializer> sharedBufferSerializer) { + this.eventSerializer = typeSerializer; + this.sharedBufferSerializer = sharedBufferSerializer; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public NFASerializer duplicate() { + return new NFASerializer<>(eventSerializer.duplicate()); + } + + @Override + public NFA createInstance() { + return null; + } + + @Override + public NFA copy(NFA from) { + throw new UnsupportedOperationException(); + } + + @Override + public NFA copy(NFA from, NFA reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(NFA record, DataOutputView target) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public NFA deserialize(DataInputView source) throws IOException { + deserializeStates(source); + source.readLong(); + source.readBoolean(); + + SharedBuffer sharedBuffer = sharedBufferSerializer.deserialize(source); + Queue> computationStates = NFASerializationUtils.deserializeComputationStates(eventSerializer, source); + + return new DummyNFA<>(eventSerializer, computationStates, sharedBuffer); + } + + /** + * Dummy nfa just for backwards compatibility. + */ + public static class DummyNFA extends NFA { + + Queue> computationStates; + SharedBuffer sharedBuffer; + + public SharedBuffer getSharedBuffer() { + return sharedBuffer; + } + + public Queue> getComputationStates() { + return computationStates; + } + + DummyNFA(TypeSerializer eventSerializer, Queue> computationStates, SharedBuffer sharedBuffer) { + super(eventSerializer, 0, false, Collections.emptyList()); + this.sharedBuffer = sharedBuffer; + this.computationStates = computationStates; + } + } + + @Override + public NFA deserialize(NFA reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean equals(Object obj) { + return obj == this || + (obj != null && obj.getClass().equals(getClass()) && + sharedBufferSerializer.equals(((NFASerializer) obj).sharedBufferSerializer) && + eventSerializer.equals(((NFASerializer) obj).eventSerializer)); + } + + @Override + public boolean canEqual(Object obj) { + return true; + } + + @Override + public int hashCode() { + return 37 * sharedBufferSerializer.hashCode() + eventSerializer.hashCode(); + } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + return new NFASerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer); + } + + @Override + public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof NFASerializerConfigSnapshot) { + List, TypeSerializerConfigSnapshot>> serializersAndConfigs = + ((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + + CompatibilityResult eventCompatResult = CompatibilityUtil.resolveCompatibilityResult( + serializersAndConfigs.get(0).f0, + UnloadableDummyTypeSerializer.class, + serializersAndConfigs.get(0).f1, + eventSerializer); + + CompatibilityResult> sharedBufCompatResult = + CompatibilityUtil.resolveCompatibilityResult( + serializersAndConfigs.get(1).f0, + UnloadableDummyTypeSerializer.class, + serializersAndConfigs.get(1).f1, + sharedBufferSerializer); + + if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) { + return CompatibilityResult.compatible(); + } else { + if (eventCompatResult.getConvertDeserializer() != null && + sharedBufCompatResult.getConvertDeserializer() != null) { + return CompatibilityResult.requiresMigration( + new NFASerializer<>( + new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()), + new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer()))); + } + } + } + + return CompatibilityResult.requiresMigration(); + } + + private Set> deserializeStates(DataInputView in) throws IOException { + TypeSerializer nameSerializer = StringSerializer.INSTANCE; + TypeSerializer stateTypeSerializer = new EnumSerializer<>(State.StateType.class); + TypeSerializer actionSerializer = new EnumSerializer<>(StateTransitionAction.class); + + final int noOfStates = in.readInt(); + Map> states = new HashMap<>(noOfStates); + + for (int i = 0; i < noOfStates; i++) { + String stateName = nameSerializer.deserialize(in); + State.StateType stateType = stateTypeSerializer.deserialize(in); + + State state = new State<>(stateName, stateType); + states.put(stateName, state); + } + + for (int i = 0; i < noOfStates; i++) { + String srcName = nameSerializer.deserialize(in); + + int noOfTransitions = in.readInt(); + for (int j = 0; j < noOfTransitions; j++) { + String src = nameSerializer.deserialize(in); + Preconditions.checkState(src.equals(srcName), + "Source Edge names do not match (" + srcName + " - " + src + ")."); + + String trgt = nameSerializer.deserialize(in); + StateTransitionAction action = actionSerializer.deserialize(in); + + IterativeCondition condition = null; + try { + condition = deserializeCondition(in); + } catch (ClassNotFoundException e) { + e.printStackTrace(); + } + + State srcState = states.get(src); + State trgtState = states.get(trgt); + srcState.addStateTransition(action, trgtState, condition); + } + + } + return new HashSet<>(states.values()); + } + + private IterativeCondition deserializeCondition(DataInputView in) throws IOException, ClassNotFoundException { + boolean hasCondition = in.readBoolean(); + if (hasCondition) { + int length = in.readInt(); + + byte[] serCondition = new byte[length]; + in.read(serCondition); + + ByteArrayInputStream bais = new ByteArrayInputStream(serCondition); + ObjectInputStream ois = new ObjectInputStream(bais); + + IterativeCondition condition = (IterativeCondition) ois.readObject(); + ois.close(); + bais.close(); + + return condition; + } + return null; + } + } + } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java new file mode 100644 index 0000000000000..612eb250615bd --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java @@ -0,0 +1,70 @@ +/* + * 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.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.core.memory.DataInputView; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.Queue; + +class NFASerializationUtils { + + /** + * Deserializes {@link Queue} of {@link ComputationState}s. The queue is represented as count of states then follows + * n instances of the computational state. + * + * @param eventSerializer event serializer for deserializing accepted events + * @param source view on the serialized data + * @param type of processed events + * @return queue of computation states + */ + static Queue> deserializeComputationStates(TypeSerializer eventSerializer, + DataInputView source) throws IOException { + Queue> computationStates = new LinkedList<>(); + StringSerializer stateNameSerializer = StringSerializer.INSTANCE; + LongSerializer timestampSerializer = LongSerializer.INSTANCE; + DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer(); + + int computationStateNo = source.readInt(); + for (int i = 0; i < computationStateNo; i++) { + String state = stateNameSerializer.deserialize(source); + String prevState = stateNameSerializer.deserialize(source); + long timestamp = timestampSerializer.deserialize(source); + DeweyNumber version = versionSerializer.deserialize(source); + long startTimestamp = timestampSerializer.deserialize(source); + int counter = source.readInt(); + + T event = null; + if (source.readBoolean()) { + event = eventSerializer.deserialize(source); + } + + computationStates.add(ComputationState.createState( + state, prevState, event, counter, timestamp, version, startTimestamp)); + } + return computationStates; + } + + private NFASerializationUtils() { + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java index 4dae7986f438d..5b057e17b399c 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java @@ -35,7 +35,6 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.LinkedList; import java.util.List; import java.util.Queue; @@ -135,30 +134,8 @@ public void serialize(NFAState record, DataOutputView target) throws IOExcept @Override public NFAState deserialize(DataInputView source) throws IOException { SharedBuffer sharedBuffer = sharedBufferSerializer.deserialize(source); - - Queue> computationStates = new LinkedList<>(); - StringSerializer stateNameSerializer = StringSerializer.INSTANCE; - LongSerializer timestampSerializer = LongSerializer.INSTANCE; - DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer(); - - int computationStateNo = source.readInt(); - for (int i = 0; i < computationStateNo; i++) { - String state = stateNameSerializer.deserialize(source); - String prevState = stateNameSerializer.deserialize(source); - long timestamp = timestampSerializer.deserialize(source); - DeweyNumber version = versionSerializer.deserialize(source); - long startTimestamp = timestampSerializer.deserialize(source); - int counter = source.readInt(); - - T event = null; - if (source.readBoolean()) { - event = eventSerializer.deserialize(source); - } - - computationStates.add(ComputationState.createState( - state, prevState, event, counter, timestamp, version, startTimestamp)); - } - + Queue> computationStates = NFASerializationUtils.deserializeComputationStates( + eventSerializer, source); return new NFAState<>(computationStates, sharedBuffer, false); } @@ -234,20 +211,20 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof NFAStateSerializerConfigSnapshot) { List, TypeSerializerConfigSnapshot>> serializersAndConfigs = - ((NFAStateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + ((NFAStateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult eventCompatResult = CompatibilityUtil.resolveCompatibilityResult( - serializersAndConfigs.get(0).f0, - UnloadableDummyTypeSerializer.class, - serializersAndConfigs.get(0).f1, - eventSerializer); + serializersAndConfigs.get(0).f0, + UnloadableDummyTypeSerializer.class, + serializersAndConfigs.get(0).f1, + eventSerializer); CompatibilityResult> sharedBufCompatResult = - CompatibilityUtil.resolveCompatibilityResult( - serializersAndConfigs.get(1).f0, - UnloadableDummyTypeSerializer.class, - serializersAndConfigs.get(1).f1, - sharedBufferSerializer); + CompatibilityUtil.resolveCompatibilityResult( + serializersAndConfigs.get(1).f0, + UnloadableDummyTypeSerializer.class, + serializersAndConfigs.get(1).f1, + sharedBufferSerializer); if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) { return CompatibilityResult.compatible(); 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 448e36715357b..601a78b6c0a3e 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 @@ -34,6 +34,7 @@ import org.apache.flink.cep.nfa.NFAState; import org.apache.flink.cep.nfa.NFAStateSerializer; import org.apache.flink.cep.nfa.compiler.NFACompiler; +import org.apache.flink.runtime.state.KeyedStateFunction; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; @@ -77,7 +78,7 @@ public abstract class AbstractKeyedCEPPatternOperator> nfaValueState; @@ -127,7 +128,7 @@ public void initializeState(StateInitializationContext context) throws Exception if (nfaValueState == null) { nfaValueState = getRuntimeContext().getState( new ValueStateDescriptor<>( - NFA_OPERATOR_STATE_NAME, + NFA_STATE_NAME, new NFAStateSerializer<>(inputSerializer))); } @@ -140,6 +141,30 @@ public void initializeState(StateInitializationContext context) throws Exception ) ); } + + migrateOldState(); + } + + private void migrateOldState() throws Exception { + getKeyedStateBackend().applyToAllKeys( + VoidNamespace.INSTANCE, + VoidNamespaceSerializer.INSTANCE, + new ValueStateDescriptor<>( + "nfaOperatorStateName", + new NFA.NFASerializer<>(inputSerializer) + ), + new KeyedStateFunction>>() { + @Override + public void process(Object key, ValueState> state) throws Exception { + NFA oldState = state.value(); + if (oldState instanceof NFA.NFASerializer.DummyNFA) { + NFA.NFASerializer.DummyNFA dummyNFA = (NFA.NFASerializer.DummyNFA) oldState; + nfaValueState.update(new NFAState<>(dummyNFA.getComputationStates(), dummyNFA.getSharedBuffer(), false)); + state.clear(); + } + } + } + ); } @Override diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java index 3151498feed7b..9b120cac92529 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java @@ -71,7 +71,7 @@ public class CEPMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters () { - return Arrays.asList(MigrationVersion.v1_5); + return Arrays.asList(MigrationVersion.v1_3, MigrationVersion.v1_4, MigrationVersion.v1_5); } public CEPMigrationTest(MigrationVersion migrateVersion) { diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..34c5110607d8956fc2cbaf83e83da13b2f9feae1 GIT binary patch literal 19058 zcmeGkTWlOx_3ru^C&Z2GG-*VG5c|^#&)V@TaVjN_AHiYcgsjs5>IbvCch`ewXSR1{ z?RAkXRR{s<4~P$nga9EVR0L9qghVBzhCT#RKcGl72udrW@)U#$1@Xa0Ip@y3JF}jd z?XJD649<>Z&*RRVd+s^sp69(w2psQ<11@7-_yCTWbHa;U9-Ob57J{Xa?)f4MJ=h> zRjTB5%~*pQP054hW{H+D01nrEtzP@>+od z+xhH*SpX~@rt8$OKUSt?YNb(JQBt7=Tk-fa@TWvb|BHki{R-S8-`=_x1iz(R+PR7V zdiPxr^d9vIPOYN(h-$4~R7cXZ(RKy`*xhfQ{;VXbkl9J$y5r!B^||_tQEb}DKfaO>Sd22dxC6{7-;C-D^#!+r+i}!oE+R*q)emWjtHd{j`o-f z9Ams$^fb$3%?7{tnK#~i^~iH^5=)UoRHsFlcP{o05oI@L0;%w6bzSvXOv(H;wrMIK zg?}zIhZLkMPWn>hXpZL9vhL|@PFF39S~htwg)oz7fs-jP%`_)u2-rO2`^-Z01^|2P z@d_=#_?T*IrXd>!8GlF_&7stUc95nvbdcEg_BG93b%!8u6kQ&#`>yQ28>QAJhn!LA zs!>RCFsCb$utzuzalA^+cqeWwKYpKn48tZ)TQt7M}4h0VM0U&rNqm)3(w4a!T8*k87JK^uU#5*sv*q|q{wkgcw}lu z_ssew1H2m}3*?@!t4J0qGz;`L4aaCDB!_FEp<0psQ0DFasQY-H<~2h@SSVdKxfa|p z7ip9P?pP+pG)ZnBdleJPY5||Fl-5~2syl4u|0|_ZX^WAoyJ12oBC?+$?f@U;JVl{a z7L0Py+gP&6{;SLB%L{W$i%SdhHhI{CkGjrGr9chfVpcitoaa?rP0TW;ZgPPfhVeEv z1XKWuf-F(7>nHvp+h--K^$V|6?e^*;5yvJ

    JYr-q5QpH1q{f~)BAhnV5=?RFwMtLvZvc87to4rx8Ps?`gxnIA zJEB&t6*ilXw586{u80NxQt!-8sNRgR-m&x4DlaL*{y|Tee*g_03)4#f03bKuAHZAx zC;q{{Q2#)-M8-b=_wsJ~2k6~){DZdp2ZMXlKRE334~DrlyV!((FlXvI&7x20aYuN1 zW}BNJ5?(#u#em*%4|={S3{`~HIR>lfhIyj~?lVF|o76c#Za|>jA=w6^;$WR*{Gn zdqb^(Y>A9D=(@Ys0J`s31JZVDaA0p*gCU8 zUN;p{|69^TCZK&8fLME@{umiH^6I4$g=D13>J%~d{gIHwFCE+4DPnjs$&Hnoc2&ga zFA87h5#MyeY$h;JSzeq?z@O_!J#_q;7q0!{14F!gF~|#5ZsU$)@DzE#T2&!Ib*`Mx zQ|3a%1X0WC7!h{qIuW{B+GPe`#^I!zr3N>$J^aQId<45XLsvt`WKm0wp-$=sBt#VYWrD`4SKPR^v0GkNdL5i&E<$(d9)!iH>)=qz^uE!Tecqmwh~48&zZ!wgbx$RYEC3Kv$wLkggbs>q@~_fLyS}c%W|j$#er*#`8s3Vv)|oU)N(nU zZ|2^gphk|bjN+BCZn%Rt;=QnikKsNh;(1qGigyrRd{~H^e}^xo=t`;$yVMQ%Ne3gx zcjsBktdRr%+`0Sx_51s;zw?9V`6`}no#t(uJnAnN0;HsXlyo5_mjE>EFEFHz{(FRbNKek%xkaQ1ddipfg%zzz;ro-I}EExz5djdzg+l9^548R z+GjbffjdzdnGOtl0`N_1#ggv!G5^&c9qV5D=dTf2o-u;B_@O{pfF0i+ikuY(AD-Wb z$Kjb`<(x9NaQUKEaJKZW!0z3|MJ_;#nhs2O?A)uLed+19zAk(36>{QAP7Fm){x(2x zR{?`OXPP=yjVdLr#F)MGo#*pM<%(-$UpDW!r>)Pb-Nd{_}*$%faZ9H}BPhXq7 z6-90fJu-mjxI6TCjMP&>cVa8CylMXM%sbBaXVLz+2Mm-SAU_R2xqeo>z72}Z`Ut+j zOR;OvFB-QyvSaZjw~la)pUBS;3H+07c=kTy_xNK;;vKy2M!|Qf08jWQ2vUNS literal 0 HcmV?d00001 diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot index 73934e565f52e86f176b8bec3d80c9c4e26ca80e..4a2b607368bbc8ddcf314b91761f97b1b493b6e3 100644 GIT binary patch delta 2011 zcmc&!Uq}=|7@ytyvua)sso{x6()I^+@4R#Gj~b|F8kSmW_+Y*4-L86c-d%gEokbj! zNjMCAeH274fXiTaSvg(-owB1x8Y#gBb2c2v<>b7SOBAJIU_)+N>e^HCK(0+EXJw3% z8m^6-xvnXV>fvx`IYizPWFL+$xB32Ex4>1g%UC zJZVan3lDp?5n7L4zHS-ODp%@eIX$<(r~bVT{l;G05Tz-3NYU@2g&^M~agsn|yD=hh zB4P>JL3Sias>9Fo0RgLjp9L#tRaD}n;(taha7jpPvv6U~*DhFE1Ah35rr1ze5IHF% zS}Lnf*ZJF!?hgge2oZ@!p@>qoKOhO>sqVhH&m)dGMBu0hT&Uz3wlF85ouI`R3i5so ziddTb9pQjb88;jozMrM9`SvD;D#5504O9(sgKq6g0KFNDXG&f_dxwGHc=lMXNgLPh z{R-oed;7-OR0a81F5KcBe)-4X{El;}++HqfJ_HVa_J+$0r;_y!4OgB}C?Igbm1;(4 zn^bfAUXpIAf5)LWYuvX*8xoX?^H5C$GmK1Ex&S}!O~>iA}W$VAseF+ z#(Hp^Y;dVO3kT0?pqe_pQcpNR(6@b!3nv~~f0BTx1yrQd3&-}Gb?VD-X*t;VW#E{} zE)nG5_W@6Ps6EP~N9@2O8b-f)Dc-8znA>&AzYwE$v**Q?r~R{Im?PM=QEgnJfB%(v zWg5lVxF^a75Gw>o!IFf%o_x4aG0dWc4qKMdKmxy}_{3&}RMv4_k@AeElrUlyfIsoJ BR?Ywb delta 545 zcmdltopELdBQFC8{D*>xyb_M7rp6|QrY6a{DQT8Qx+aEZCc24*-SuLDVWS=L7QJJlV#*ISf4R4s9XYCS;ol4kq5O?dvYO< z=;V9y?5x2hKre27DZd7!8=Kj`Sr{!*RZm_hCcn8@!xP=U-z=MN=+9)H9BvGE1=!&t zCU-y%hYJFo#>*}b_8%~eLE#JpT0qRio|~DHk^>Ab2ByjK-l~)Dc?nMb>>0+!R$P); oR5H2VThs-l76`z4fHX4{gT$Di7$ygzk;TA#P*T`j>t)Rd0KScj+yDRo diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..d4d3405ae45dbbb457e844546ccf5c5c2df1b732 GIT binary patch literal 19503 zcmeHPZ)_Y#6@PcOW5)?`<0g$8(E!SQX@&Qi*tz7KNQvV=#o^+lIi~^C56s<-&x`kV zx3hbWT_j5gAwYZq@qvCofcS!nKtdHrL_(r%T7jq#6p02=prR_`k04YieBwiSGrM!U z=iR-Y?K@@b>`M0jncX*UX5PH_d+*I003PqS2VSOn_yasr&WT@S2&s&Xmrjh0l3b3C zjuT2p^R$#7&6je=kB^fvnm=(0U%V|oyZ<_|DXS5-JX$82RwCtP>wEu#K8wLAgH;t` zlJXK&7By90#v4^B8?~CDEB2a6>$a*{N{LuhnbYkeWh&9srzvxJNu>c+8Jyn)nFXZk z(J88WRin0{uRZ(u@BU1=KI0G^Z;xOzEk94D+rV6a5UcC9TBC|(W3MPr7;0sn)To7h z;A6{ADwfR?0}Jqo%P90qjyFil*()e5l^i{B{CGJ(N^^OXmQ%-0j^=Z@{OGYUN>G4~ zjS+GZVD5)VqnFc>lJjSES6^q}PD8o??^~v8UXcyC0+P!*ia9G3Gv26K zM=@s`iaFaXdZ!C8v_iDHPmw(Z>ySbXy?up>)*>HhW9Z}%)}mnORj)-{TJh+Px!`R~ zZ#Q?FMj$De|qK657Lk-z<#RH8qPb9d#4j+HD^P6;ZtOV_$;Oz{AIRgC?Cat z9yO;cpeGIe1sJK&MN-#%l}%~HvZ!Ulg9T1A%PnxWfTo#NTp7Y)9u9nFarOq0dtT!e zTE+G;VylKO3x=3@NExkgsfnzhNNubjv*GP)s=eej!R67d@<_{fW&OQeYHir>v`Qm- zH7m%RUX>ktNc16ZuhP`s@#p8Cyia@ne>ggyfIXyKMxA?!RVYKb-tT?m_D)~_?_aq2 zz$ahrf%Gg4Y6hvylCo{EIoNG4F={OtT4mKdd(Ih#>jQYck6*8X;B?=+mh5eOleXb- zKp4u@RA#u|Mu|KVDCH9-LR7ev_y_L7GZSCXKeukAp%>@1*T-*hfMwR5CfSW`Z*W3cGvU*#<_c@o>&~!>|Es2Ht~0p0ok56XMAkDb0PsFBrd+6% z8NFWf7n(NgxjJ9GJTo;nJ2x|J!^1v)$O1hCjR6>b`&gatFsUsYq}&ERz_;Eb z_+WP=K9D7`!3PL0?w)yCiTG)5uVuq z6WoAT-*>n}@4$n;Z@Pvm#_F6qs~m>;y#|39v7k%p94WV<&hf2ib>0bJgT7TPyv1FS z)5CzesPYQ8?1LBJ0c(k1fa-jGagj2Q zA`cLCERPXmNSAo%#N1{CU;5$Dh||m1zAT9iCQCeYZ{bNJZ-O>J9y=p~fA4tcp(!tE zLW(}cp7Gy!TK+e$-C_$bvfrfv-G%0yxG(_d)`sP`(G6a~L`RdK?-&gJpQnaT2Mk+u z=DKJzT$JQ7!UZtew`kc4ic*T(QD_l9Lu zN$95Qp&RJG;|2zT3z?*~_5-=iqq=Pb3O0!ZJCt>ZH;_b_7US5)KvcR43%Zo9LdtDm z9(*f0T_s(JSmc}bM&_GjNo+t1u0_3@s}HdHB;VBieADpG%r}h$@=b^0?4NgRD;HOG zeZq^r5E+YT!{Uc}>~d$)Khgbv8W&o82zRc$1TcsTE(UKoo;$qd;*Hpv*-pR+(bkTx z%G_pm;qsRxW|G8A-hFd~jEp2Plg38ah|Lknau?Kc-G?7Z%p{4KC?YuIsqdB>Ub-JM zNv*pWCYITpSZ-$TWOj+~_(?%W(N$Xmsf4EGxDs>^zQ$OrGv<5tdj@cW9ipr&ptkwhd>3i9!tYWnDs9k5G0FDdYaq722SOmavbh)g_xd5qzyqc)x`( z6{J-u)t7JKXs((l9>ML)JpLF$YErK>82aOL_YXYx_V;gi4HTR;vnj(n!SrTC z4aIti?_X@x^4e#v{O!U|vi}vC*M7?>4c+I;;AFeQR8(uG=G8I%wI3bso%{E1d3Okg z(d&LFlooO)HlK32au3mgK@Wj9&v4h}DN{3-FRE2%!|(#`4bEItD;15-su~XS6X##~ z?9DH|@pV}{PgS1_tv#Bai2!A5{O}|^i z#aRlOlzcqDM}*&z@rKN&P`rDNy1Kn>-8XO84XyO%jRos<<h#XotQIQ0?T$-~M)!@zU^k4$pgYsWIpv=|q2|fe4|h@vDUPx3oPFE*{@jR9V_+=I zz2j(&|A}AYazd~c?e8C^!At>Qw-V!$r`u-mSL!xQlJ;46~+f$bq literal 0 HcmV?d00001 diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot index 3126ff4867b53b37715c131e27d8785f6430f533..f5e00beece1030947efab0b0166567fbfd8341bc 100644 GIT binary patch literal 19835 zcmeHPZ)_Y#6`wuZvEzieag)Z4XaMED5aB&1@xOB-C650Thl>+(P6MSMn7f@jFW%d| z&h9zqB3VMH0>l^iL?sYKd_hI6geoBssS<@yfm&5iBpO74iYkadf>KcU#E0@`cII}^ zyL+5{cjTJvO6R-1xqb8I&71dr@4Xo#RDUTw;$D&8D=3*ho}L;%o}L^{=d_8@@rj%^ zil-;XMyJyg$4{P^DxA>pBtktW{`u;YPs3yEQ3c(94LgLEFkKtXW5dYd{F)672|@QS z@EZqiKLxK_;-No0`*+l5Q#5VSqKeBnze?1Cp_^-PqpNwVRI*If*(j5WqZ_uG!!}Wu zOeagIjt%_@LIrIFI;m20ZX0A)fT~9)tD8lGIF`Bb%oo1{+x0Z(EVG~&Ik8q`#U|=aA@j)qQ?z0_dWopE0~4t}V(Hp4@MFWY z_$1{Y-8~}lolAX)M1vf|D>3M=je!IOn?SZrhxr$@(6PE3vF3KL_aQ|U2{ zoP;$uJ(fmj=>?$C%jn3}DpnQ0k5`l5SafGsy>hNXTC{&>_}QJ|g#>d&ANjTCUj@O3t4VB03PgC5K9e;88vHOe{{{yr0DB6Sbc|bB3 zsYWR1^?vUeA)UVd-~8yt1D}4a2PGEKpkZNc5$7F?E}`AdDkb)+WoXs%nX~RRTpNJ* z``G6y;+*aiEedbp9&t-c4{u=(GvT1a0cFUOvbwr!}Zjd zS&{t^>hFHU`+A-fbW>-vn07hnYIw(7rI{vh@SPINB(-(yHBJbvCw!_}UZ;(u&Zbga z)pEJKNzs*7CLxj$*~u^u687;qWlF6sn3a;hvFxBdSC+Gv7Uq@~mloz7^kE-Ac%52m zk(e-xdG%rUeI7eFIZG+t;0k#F#=ET{Faw~|wbl7kSB0}C zF2fRsn2@7aiJ1Nhx9bS?{{W#OU9`O=Jb0ISwRZSyKGL=ZiY*BXtm&?qok)E<&U(k- zsa+_ki}68U6h43kpNDBDJ^;#1-~;&8djucsj>HGDrEBm3z{@*{4;Z?Q@j=J&!O-1_ z4-N+K!2#~gE^GrI%vpv(@(h#u;0RB8Zh;9R@zwVofC%SZc+mH>a8zAv&RMX^V3JoUq?KmWim zFJBDtf|b{};}|@H9e~0m{ExK}@btr7y@|Y6J*umO0<8lDFoC$C#Wj9Z*k&Wd_ zDMx71_t|6e`s%@t{OtWlKfx>5`&b3rYje;{crh-poq+9yOV#a#bE~sSV6K%dVoZcw zer%*kb`wHQjx_)^LQr{aFN{pt3@svb_^X^Uic>e<-1F)ikx8LAbpskA%K0)u6NCw} zrE6e9oVvLeU_zX_iQXH-vfNv?QN^j7j;C%=|80>N2ySGO-r5i3wvXysNEB=n3AU-~ z(A_{1U|JXFHU^@~RnX9(aurZ+0`tJHqRUk>gy@QT)85E>lWgf4&;ol=@8s?S=svDD zb-vy-e0SEHMgsMwL){#ox9zwaGj@GEiasBii)g~*hequ3gwo$2{C@&AT6_pju>1s} zLD+CHc+-vC(Jg^Dy4KCM14i(%wslu#i{gUqFLBN!&YAr0!x1tw66Z{6hlL{!N5tD* zz{+(Ve#AMGIA@~rO--a4T9#u* zkRL?iJB*LPP0s)v&7;oD7;D*+B&NnUMe(B8F5U#+CwxCryU$|ICo0?Kf* zbe+`bfe-AXdU4fZ;RD;H6E3k3rh>F`2j5@@}JMYkoq^Tx%S&`Yv>7Ail&;K zrmS8n8(tstU;F;y-lc#1iiv|W4Dt3up|pTIe(aR#mAJS7gRub{o+WnVsdEdLF6c$~ zsPGD$3Qk_oHO(N4x&hPt$hlWOf8)zMWH^YCgQbN0i^y@tV#npx!-4g>7%z_AOgZ%__Zh zeZ_u5oxZ`X(trhaSvRrKqJeL(O>d9SYGBfv7F_H!I`1?FJK;12#Y*Szihucpx+h^D zZYwJa!&tY^yyHH9cEo2fAPXk$7+Pa5^e$ZDGrJcaNg^`#3%J-*;LO}7*cWhdO0bxB W#7&Jp>z-XekazFi@q6A2{P{1~^ZHT% delta 437 zcmex8i}7wNBQFC8{D*>xyb_LP$%%%ECT8ZkCTYgTx+az>$-0&)DT%r%7HJknmL}$D zCP^j?42piHS)f1A{}-=Kah&7+DW7Fi1?^T*F$< z$mzA_Edy(D5re_x9#(l4kHq4P&FbuHm?uA%5aQ>vcK|t@fv21UND53Av}xL0#$C$} zlAPSD#KDd%VWE71X>*#`Y$l+r6ijBbpv^Cq$ue>ptj`!2R4xInEMw&2$b;IcJ-Lua zbn-oUcGln$pcgm4lwSkVjm_-eER2?@swXcLliysd;fZeFZ7%Q6n=J^CT&C0gtSd7C_?xG!s|4osgoduG(RQ9O-tQUKs_*CkCP?qU3Ye! zI7P}00Y}88CqP^fhyzDBQ3?`B9QZj<1PD|FLLj(t;{b1FceCs4+Bk`=F6pjhuh+Bt z=FOWo?|tvRSpYo#u@C$iYvX_55wlMGMVOnrmn5L^}^eq zNXyd}!J)?6d=nyD12WSA%w=?PuQ&ygZEPzD~hyre=oJexs&TCX7E z=TJe0<)gGPJT#c31G#i^h#VVArse!VvM?}wY$#9i4-FnA0OL7)eup5!zwr1C>6v)u z%641C#i=O)bY2JOe+94kJpH)+bv*`6ou$oN1(bKkDO=tHKE{bMVB7KFPEAn3TTT%M+W-yv_Q(Lr?W|w z7zQ;=IF#WuQ{3sMGU$zI-qsTjF^h9ISt)R{7D(jbFHb_ zL6(}>L1NwA7Zr2C9fHlH-Q|(G_d5OW;96@!k2NZlX!9vSW~|}}_MngIsP8uk!bMw@$v7rPJQ=?S~&7c;?GCh|j=oRVVowBAYs!gb0r838)9ylV>GwWaiPn_V<{w5FT@dsYFzh*nC5Sw!z zta2FUjT!`I#D*4`bEMpWImf$doAYJ>8;q?&5iRbl=?$Eg(0GHkTk8$5`^XzW%e_I@ z_Vfn5K5x(?%=D!|Z%{8{tqx8T)t3P=WSs;LauBbZim3lAY#IWA;zo%D}FS#|SZ`iz0MV+F}OZiNj$n zPw(XRotDsGvM54#_nNHLP0$B8$M(qJ-#!s~V9CpIAz6=MPk1j}EC1EkH`w{>?3*}X zxKLda7Y+c!+MxP2hQUi%=%|YGEr-Fsd1-je=h$jjuCpe?eMl}-wlcP`cFno$L@q~t zTspFwqRVt;u3XGfmhk*`nS6g?|AU|2b>EYsg1whlu-!HjMuUrSzU>6Oa-XW&+GSN| z6Uf}=H?vp*6!POEP2hDbfSfZ{AJhmyZ)09a#}(IT5vDwt=xTp z-ADDN*4LYQw`aYn-&b!s80PrAX>V_FW7jUC=ub?pxf|F7Uii+l0S zm5%{-}7Vezx&OXx(fxV1yWJQ+H)ms)Q(L^+cvXQElwZ0${ehF*?{ zw!5H}Yd!pkawbvEL=wp%x7<l(H8Qp^HnTKOGe}EI zWnf@d{dl%@K2R^?L|;h>Ak73d=Os|61jvmB;wPJ%nMxTqf0QX=bcI>N$P8sMLTLLa zMh2B+gQR2&Lvv%@lr*yxT@wRC6Wt_Z!xY_Av*ctmL&KEBBujG!2K%hdc}zPPSx+%A zNX*{+L8gq6i9Kj?Hivon@+0F7mw+{h`+&J1R=Opa4*+APae%MO=Z zCyqQ;YHj^S$E67z948?^Ebqs9w4j>)_vOzZ16)ckv%V@AZWnfUb0CYq- zBWLcKw+yVsMGRV#Cz?vK`?)!)0P*IDa%-4ySXs+Dc@mS*dkXC1y+(J~iS|=B8#gpMK>h^)jJIu} delta 432 zcmbQ!#(1fjk(U7k{zJh;UJ1uk)3lUCb2Cd_3kzdYU6Uk316@nAG(+7KV`F1OizLe= zv2Kc~MUt+GQHr6ig{66_Zd!_=L2_bJnx%27B?E(f%H~bXI~Z9HF)&C>-ki@` z&dBMt<}CwjaS?;T^wq zorS~C%`vzH=*7)<{BIUUOAHN@H_FIvPSNm2w{;2!*pwJ!R7bL#+yS{B V*?qrRHve}JVw#-K%+3OeBLGE>foT8$ diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..660a21a77b04b798bf7db8423d071ca99a0a36ba GIT binary patch literal 18866 zcmeGkTWlOx_3ru^C&Z2GG-*VG5c|`LosEOz*iNO?@gq2FoRD=IK>gtE&fWFk*_r9g ztRF?PR3QYYKOjCR5(0#fP!UL_5)ze=8u}1O{eU9TASkVf%2N<36u$T<=iIq>XVx>b z^pkTsi}+Xk3Lfa=33 zXhub6u4$}4^M&vKTv?s<0FG`AU?Z#`PL>$pSO}0^GhD686vqXsD36+&x66)_Iy#a)F**jKGk&r- zHhzMT^B^{tSv`x{)k;{K0&USjw_&xhnCazb3DHV6&ODxEn_qZCe?Mp>(HU^|~( zFbjaC!)%2a?#FAa#+(9*D@rP~U@IPf0sdHo^uI{R(XYZi^6jm=LGWA3rJbt?pm*N| zLGKeG!D&=9AEnOHMQZt)@7s^pzIpZc_IJ`GohR>GX6t^FOUCj5F6Sxc99PV6qvbrs zoU1G5T%+in$&~XR|QlO!CuTa5SOoYZ5IXSqs$eTvR9}!9`9PKd|IL7pL z(bJrOH5>fWXWw}9)g#ZQNh(hcF`ZRm-uc*jM3mi}jikb7=n4&3Ov(IZd)-t%2LF6$ z9w|syn)Kz#QI(ZxO%HT7ty9Ngj!W*(Bg`CH;9MR|GgE~O0h@S&MVH5$zAO9hMyYklA#YSVH7Yp{ z=FFNT>=8~w9Iw(a-qGs|kKLnR|1WUoQ)E9al|biSvQ=h-Tp#kk5!C7F``yo8y6@Ak zb&>QO8PH9t&e4)<+Vf+B;Q8&><_d=Fl)5svA>_iQ;(|0eB{VOY=gWzEnK7D|^(u0?muMH(f6 zJC->yO>&#ZUdM#ljeyUrSu1uks(WnZ|JN+b+OWyh-7p~*5!ufWcYqIao}y4Ivqr5N zY_wdm|LQ{F^6d2d-2Ci}OCAj1Lsx85sW1b$SW?b==NanK%#>}@buN&@Fy3tq0TqBE zImv7%GgGTp*QcN-Pr6V8a3)GD;S8jLZax{n%#Q|Q#5KMfWTxMLxT_5m-eWH5na)oY z3Tp(|gir1VJW^yX5{KxBq`{tgB3v-21yfvltVe}=zi{oB9~k20i$Pwf@*8&?gXhV8&Ju+L)wx=^%xoVb zCWu;A$4IbC*NM=nwaW~?jKfJiPcL)(vL!J-StmmG7oODXCSU{Pu{SdKcTa>KS@M!5 zq!3W-nc$7D<$wM94SVrL`?qNV;X-3goErcLYoqGh5C*S6p`#(r_Y4O9$I|ebkYS6j zTo+s$7A5&e31Mtw?HaM{2P`K;SW4Q>uvNCcP^%W1oe5m~NIt)G_`|<=|A#)oE7%9I zg6+4tWHP!K7g|n$mHS-7Ixnv}n*ro@zL>=$Ag3RWGz0gk2yk+&Ay*^9E3dDCktv(O zIYN$og+nHt)Q!)3UVURSDbz{bfP{!bzf91C-2~Z^7&oDly15(Ngih)ver*h9xi>AN z>ZESkp1L7@w?$$gypTy+>!2gIeXnj8iGpoQ9m*j%wFP)r8Cuj2Bnt1U57i7d;Z5)=onG3M)xanF8tUsY38op0ve zpJ9f|S4Q#5SU23k8|hxy!e`??CK7m8T#9!PUV2!FTX=^rrsyo{!Y*|Ke$v5E`R+VN znKD%H&%L`pJa=#ZbMO4{dA^FLTW4k0B@c&-g#alhAmw~Wxp@E$drVhY9UfT29$Tv{ zxfn>`dYx$6MPL<%RXluaaq_iSZU9HutVj`w8DODS#2topq+WmW%3m-1H1{7~8|`zv z*2tZxHaQ&`_C(-Y(5ja1_c8OeA0O+U|JQF2S)MV1xcGrcSb!bh9*Ue52OnP8hsWWW zV&$ANJ$w10R`IspL^-4x4t2J?-g?5OHK+!PyRMQbXO6B zJZG9Zqeh*QPG-cN|L(K>UDuz&u3AH`)(J`DOd@h7;dxUNQsWIVal(b|=7gU$RFCTP z)-&q6|NipZNhAf1S&aX#V0~R%ro1a(^10G|v#CvvpJbqxZP(-WrPU{I{`u>tZYGi2 zLXQmKIqeS}INwYK-A*m0%BK0Fv+sD{KZ5qhKVYEz5EO;$)HV15&C+f8iF`$z@ss*z z;+oq$xK80u8sS;|b)k6rxWZHZDQ1b}C-@8xe!s)Ph#ukpK5*7I@Vu0ei;O{ZP K+v&hPA^!s~6Py_U literal 0 HcmV?d00001 diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot index 1b1fa754c9ee99d838fa38e3a2feb71764d55522..53819f22637aa94105828b1864ed32f470076518 100644 GIT binary patch delta 2011 zcmc&!O=uHA6rSDuY+D-CXI+n z5xulX>mX8E6v2y7L0kj{4~lqEL_Meo{-CJTgTLU>pP6Kn)Vi@)4-O3T=Dm4uzW3fY z^W+%u{Ul)lGsl>vwwdKPzlZg649~7*8IFyLjGq++BypU_UVpZfNRL*YjzTV_KQBj!wPQ3%L!hFwPs8Hs*{9Tz2A4stvrdpY zq0N5^-PxKo!u$5srZ)i4_YvwrqH;h3-3V{cHfqa;4jN7M^s*^(E)cs-gm0!3bTUif zW=o~oyecpcX#0)oamTn`xvKw^({1)C_4jocHXp_f$u*@B3C0DqC`Eb|UJ+@mH>VU{ zMl3-Vq(=)$4a6dmxQNAH&w}Dv&8u`W7yr3d;Fg5k@8FZXut#)6<1u)Trnp2>lzAl~ zJA$EY-LZZo#}ZPXm{Mp2MOHS%;)*D59UOY~cHI365jZOYFDkj!mE;vCLAPc~j$!MEzLp2K9%IL3A$z>eN*mYi zX_4_LJX=1SkQB-1!hbj)`SE4+^*`qloL??lJ_H8;;)K@*uU8t~I-vrIL|o*hVl`8= zOX)m!vBGeD_@ANg(7A7pHk2q;;GvcXW*AwM>jHeaqz0#RSSNgk6e*_i?9$?jOP?-W zB>~w5jBzolC}3R|BBwJ?PA4L#wAU$!Nv9UcSz?*>-BO<3-MOzJ&+r|AJ4bE~znH`{!O_iH?@BHF zujC8q)Tew~a(_U+5TFXLWd5^*^+7A_3oL@u9X+eD8ry!LffoF<3KOfb4etRQ3EVYf G0f3)9Pg)HC delta 509 zcmex2mGNd9BQFC8{D*>xyb_M7=7}arMn*-SuLDVWS=L7QJJlV#*ISf4R4s9XYCS;ol4kq5O?dvYO< z=;V9y?5x2hKre27DZd7!8=Kj`Sr{!*RZm_hCcn8@!xP=U-z=MN=+9)H9BvGE1=!&t zCU-y%hYJFo#>*}M_8%~eLE#JpT9em#hO)60mn0UIOy2J$Y5|dlNCMeRPz(|VX#nxT Q8bLHDexZEE$=Y6L0i_Ctng9R* diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index e447cbeec05b0..7d7ff003b58cf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -390,6 +390,8 @@ public final OperatorSnapshotFutures snapshotState(long checkpointId, long times return snapshotInProgress; } + + /** * Stream operators with state, which want to participate in a snapshot need to override this hook method. *