forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-7008] [cep] Update NFA state only when the NFA changes
This closes apache#4195.
- Loading branch information
Showing
6 changed files
with
535 additions
and
8 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
180 changes: 180 additions & 0 deletions
180
flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,180 @@ | ||
/* | ||
* 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.java.tuple.Tuple2; | ||
import org.apache.flink.cep.Event; | ||
import org.apache.flink.cep.nfa.compiler.NFACompiler; | ||
import org.apache.flink.cep.pattern.Pattern; | ||
import org.apache.flink.cep.pattern.conditions.IterativeCondition; | ||
import org.apache.flink.cep.pattern.conditions.SimpleCondition; | ||
import org.apache.flink.streaming.api.windowing.time.Time; | ||
|
||
import org.junit.Test; | ||
|
||
import java.util.Collection; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
import static org.junit.Assert.assertFalse; | ||
import static org.junit.Assert.assertTrue; | ||
|
||
/** | ||
* Tests if the {@link NFA} status ({@link NFA#computationStates} or {@link NFA#eventSharedBuffer}) | ||
* is changed after processing events. | ||
*/ | ||
public class NFAStatusChangeITCase { | ||
|
||
@Test | ||
public void testNFAChange() { | ||
Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() { | ||
private static final long serialVersionUID = 1858562682635302605L; | ||
|
||
@Override | ||
public boolean filter(Event value) throws Exception { | ||
return value.getName().equals("a"); | ||
} | ||
}).followedByAny("middle").where(new IterativeCondition<Event>() { | ||
private static final long serialVersionUID = 8061969839441121955L; | ||
|
||
@Override | ||
public boolean filter(Event value, Context<Event> ctx) throws Exception { | ||
return value.getName().equals("b"); | ||
} | ||
}).oneOrMore().optional().allowCombinations().followedBy("middle2").where(new IterativeCondition<Event>() { | ||
private static final long serialVersionUID = 8061969839441121955L; | ||
|
||
@Override | ||
public boolean filter(Event value, Context<Event> ctx) throws Exception { | ||
return value.getName().equals("d"); | ||
} | ||
}).followedBy("end").where(new IterativeCondition<Event>() { | ||
private static final long serialVersionUID = 8061969839441121955L; | ||
|
||
@Override | ||
public boolean filter(Event value, Context<Event> ctx) throws Exception { | ||
return value.getName().equals("e"); | ||
} | ||
}).within(Time.milliseconds(10)); | ||
|
||
NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true); | ||
NFA<Event> 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()); | ||
|
||
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()); | ||
|
||
// 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()); | ||
|
||
// 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()); | ||
|
||
// 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()); | ||
|
||
// 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()); | ||
|
||
// 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()); | ||
|
||
// 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<Tuple2<Map<String, List<Event>>, Long>> timeoutResults = nfa.process(null, 12L).f1; | ||
assertTrue("NFA status should change as timeout happens", nfa.isNFAChanged() && !timeoutResults.isEmpty()); | ||
} | ||
|
||
@Test | ||
public void testNFAChangedOnOneNewComputationState() { | ||
Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() { | ||
@Override | ||
public boolean filter(Event value) throws Exception { | ||
return value.getName().equals("start"); | ||
} | ||
}).followedBy("a*").where(new SimpleCondition<Event>() { | ||
private static final long serialVersionUID = 1858562682635302605L; | ||
|
||
@Override | ||
public boolean filter(Event value) throws Exception { | ||
return value.getName().equals("a"); | ||
} | ||
}).oneOrMore().optional().next("end").where(new IterativeCondition<Event>() { | ||
private static final long serialVersionUID = 8061969839441121955L; | ||
|
||
@Override | ||
public boolean filter(Event value, Context<Event> ctx) throws Exception { | ||
return value.getName().equals("b"); | ||
} | ||
}).within(Time.milliseconds(10)); | ||
|
||
NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true); | ||
NFA<Event> nfa = nfaFactory.createNFA(); | ||
|
||
nfa.resetNFAChanged(); | ||
nfa.process(new Event(6, "start", 1.0), 6L); | ||
|
||
nfa.resetNFAChanged(); | ||
nfa.process(new Event(6, "a", 1.0), 7L); | ||
assertTrue(nfa.isNFAChanged()); | ||
} | ||
|
||
@Test | ||
public void testNFAChangedOnTimeoutWithoutPrune() { | ||
Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new IterativeCondition<Event>() { | ||
@Override | ||
public boolean filter(Event value, Context<Event> ctx) throws Exception { | ||
return value.getName().equals("start"); | ||
} | ||
}).followedBy("end").where(new IterativeCondition<Event>() { | ||
private static final long serialVersionUID = 8061969839441121955L; | ||
|
||
@Override | ||
public boolean filter(Event value, Context<Event> ctx) throws Exception { | ||
return value.getName().equals("end"); | ||
} | ||
}).within(Time.milliseconds(10)); | ||
|
||
NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true); | ||
NFA<Event> nfa = nfaFactory.createNFA(); | ||
|
||
nfa.resetNFAChanged(); | ||
nfa.process(new Event(6, "start", 1.0), 6L); | ||
|
||
nfa.resetNFAChanged(); | ||
nfa.process(new Event(6, "end", 1.0), 17L); | ||
assertTrue(nfa.isNFAChanged()); | ||
} | ||
} |
Oops, something went wrong.