[ https://issues.apache.org/jira/browse/FLINK-12319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16826610#comment-16826610 ]
Liya Fan commented on FLINK-12319: ---------------------------------- Hi [~mpf], thanks again for the additional information. It seems like a real bug of Flink. Would you please provide more information to reproduce the error? Can we reproduce the problem in local host? What is the implementation of class LoginEvent? > StackOverFlowError in cep.nfa.sharedbuffer.SharedBuffer > ------------------------------------------------------- > > Key: FLINK-12319 > URL: https://issues.apache.org/jira/browse/FLINK-12319 > Project: Flink > Issue Type: Bug > Components: Library / CEP > Affects Versions: 1.6.4, 1.7.2, 1.8.0 > Environment: Ubuntu 18.04 > openjdk version "1.8.0_191" > OpenJDK Runtime Environment (build 1.8.0_191-8u191-b12-2ubuntu0.18.04.1-b12) > OpenJDK 64-Bit Server VM (build 25.191-b12, mixed mode) > Reporter: Marco Pfatschbacher > Priority: Major > > > I wrote a simple SourceFunction that creats Events in a loop. > The CEP pattern is very simple: > {code:java} > final Pattern<Event, ?> failurePattern = > Pattern.<Event>begin("5 or more failures", > AfterMatchSkipStrategy.skipPastLastEvent()) > .subtype(LoginEvent.class) > .where( > new IterativeCondition<LoginEvent>() { > @Override > public boolean filter(LoginEvent value, > Context<LoginEvent> ctx) throws Exception { > return > value.get("type").equals("failed"); > } > }) > .times(5) > .next("1 or more successes") > .subtype(LoginEvent.class) > .where( > new IterativeCondition<LoginEvent>() { > @Override > public boolean filter(LoginEvent value, > Context<LoginEvent> ctx) throws Exception { > return > value.get("type").equals("success"); > } > }) > .times(1) > .within(Time.milliseconds(20)); > {code} > > After about 100k Events, Flink aborts with this stacktrace: > > {noformat} > Exception in thread "main" > org.apache.flink.runtime.client.JobExecutionException: Job execution failed. > at > org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:146) > at > org.apache.flink.runtime.minicluster.MiniCluster.executeJobBlocking(MiniCluster.java:630) > at > org.apache.flink.streaming.api.environment.LocalStreamEnvironment.execute(LocalStreamEnvironment.java:123) > at org.classdump.alerts.FlinkCep.brute_force_login(FlinkCep.java:263) > at org.classdump.alerts.FlinkCep.main(FlinkCep.java:41) > Caused by: java.lang.StackOverflowError > at > org.apache.flink.runtime.state.heap.HeapMapState.get(HeapMapState.java:85) > at > org.apache.flink.runtime.state.UserFacingMapState.get(UserFacingMapState.java:47) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:339) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.releaseNode(SharedBuffer.java:342) > at > org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer.removeNode(SharedBuffer.java:355) > [...]{noformat} > > This happens with version 1.8.0, 1.7.2, 1.6.4 > Version 1.5.6 does not have this issue. > Seems to be related to FLINK-9418 > > -- This message was sent by Atlassian JIRA (v7.6.3#76005)