[ 
https://issues.apache.org/jira/browse/FLINK-10570?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16662266#comment-16662266
 ] 

ASF GitHub Bot commented on FLINK-10570:
----------------------------------------

dawidwys closed pull request #6896: [FLINK-10570] Fixed clearing shared buffer 
nodes when using After match skip strategy
URL: https://github.com/apache/flink/pull/6896
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 f5e89d705b8..01dcfd97eb4 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
@@ -389,6 +389,7 @@ private void processMatchesAccordingToSkipStrategy(
                                        sharedBufferAccessor);
 
                                
result.add(sharedBufferAccessor.materializeMatch(matchedResult.get(0)));
+                               
sharedBufferAccessor.releaseNode(earliestMatch.getPreviousBufferEntry());
                                earliestMatch = 
nfaState.getCompletedMatches().peek();
                        }
 
diff --git 
a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java
 
b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java
index d42839e3071..239dd4f4857 100644
--- 
a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java
+++ 
b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java
@@ -20,15 +20,20 @@
 
 import org.apache.flink.cep.Event;
 import org.apache.flink.cep.nfa.aftermatch.AfterMatchSkipStrategy;
+import org.apache.flink.cep.nfa.aftermatch.SkipPastLastStrategy;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBufferAccessor;
 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.cep.utils.TestSharedBuffer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.TestLogger;
 
 import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
 
+import org.hamcrest.Matchers;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -38,6 +43,7 @@
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
 import static org.apache.flink.cep.utils.NFAUtils.compile;
+import static org.junit.Assert.assertThat;
 
 /**
  * IT tests covering {@link AfterMatchSkipStrategy}.
@@ -936,4 +942,43 @@ public boolean filter(Event value, Context<Event> ctx) 
throws Exception {
                        Lists.newArrayList(a2, c2, b1)
                ));
        }
+
+       @Test
+       public void testSharedBufferIsProperlyCleared() throws Exception {
+               List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+               for (int i = 0; i < 4; i++) {
+                       inputEvents.add(new StreamRecord<>(new Event(1, "a", 
1.0), i));
+               }
+
+               SkipPastLastStrategy matchSkipStrategy = 
AfterMatchSkipStrategy.skipPastLastEvent();
+               Pattern<Event, ?> pattern = Pattern.<Event>begin("start", 
matchSkipStrategy)
+                       .where(new SimpleCondition<Event>() {
+                               private static final long serialVersionUID = 
5726188262756267490L;
+
+                               @Override
+                               public boolean filter(Event value) throws 
Exception {
+                                       return true;
+                               }
+                       }).times(2);
+
+               NFA<Event> nfa = compile(pattern, false);
+
+               SharedBuffer<Event> sharedBuffer = 
TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+               NFAState nfaState = nfa.createInitialNFAState();
+
+               for (StreamRecord<Event> inputEvent : inputEvents) {
+                       try (SharedBufferAccessor<Event> sharedBufferAccessor = 
sharedBuffer.getAccessor()) {
+                               nfa.advanceTime(sharedBufferAccessor, nfaState, 
inputEvent.getTimestamp());
+                               nfa.process(
+                                       sharedBufferAccessor,
+                                       nfaState,
+                                       inputEvent.getValue(),
+                                       inputEvent.getTimestamp(),
+                                       matchSkipStrategy);
+                       }
+               }
+
+               assertThat(sharedBuffer.isEmpty(), Matchers.is(true));
+       }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> State grows unbounded when "within" constraint not applied
> ----------------------------------------------------------
>
>                 Key: FLINK-10570
>                 URL: https://issues.apache.org/jira/browse/FLINK-10570
>             Project: Flink
>          Issue Type: Bug
>          Components: CEP
>    Affects Versions: 1.6.1
>            Reporter: Thomas Wozniakowski
>            Assignee: Dawid Wysakowicz
>            Priority: Major
>              Labels: pull-request-available
>
> We have been running some failure monitoring using the CEP library. Simple 
> stuff that should probably have been implemented with a window, rather than 
> CEP, but we had already set the project up to use CEP elsewhere and it was 
> trivial to add this.
> We ran the following pattern (on 1.4.2):
> {code:java}
> begin(PURCHASE_SEQUENCE, AfterMatchSkipStrategy.skipPastLastEvent())
>         .subtype(PurchaseEvent.class)
>         .times(100)
> {code}
> and then flat selected the responses if the failure ratio was over a certain 
> threshold.
> With 1.6.1, the state size of the CEP operator for this pattern grows 
> unbounded, and eventually destroys the job with an OOM exception. We have 
> many CEP operators in this job but all the rest use a "within" call.
> In 1.4.2, it seems events would be discarded once they were no longer in the 
> 100 most recent, now it seems they are held onto indefinitely. 
> We have a workaround (we're just going to add a "within" call to force the 
> CEP operator to discard old events), but it would be useful if we could have 
> the old behaviour back.
> Please let me know if I can provide any more information.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to