@Test public void testSimplePatternWithSingleState() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream<Tuple2<Integer, Integer>> input = env.fromElements(new Tuple2<>(0, 1), new Tuple2<>(0, 2)); Pattern<Tuple2<Integer, Integer>, ?> pattern = Pattern.<Tuple2<Integer, Integer>>begin("start") .where( new FilterFunction<Tuple2<Integer, Integer>>() { @Override public boolean filter(Tuple2<Integer, Integer> rec) throws Exception { return rec.f1 == 1; } }); PatternStream<Tuple2<Integer, Integer>> pStream = CEP.pattern(input, pattern); DataStream<Tuple2<Integer, Integer>> result = pStream.select( new PatternSelectFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>>() { @Override public Tuple2<Integer, Integer> select(Map<String, Tuple2<Integer, Integer>> pattern) throws Exception { return pattern.get("start"); } }); result.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE); expected = "(0,1)"; env.execute(); }
/** * Checks that a certain event sequence is recognized * * @throws Exception */ @Test public void testSimplePatternCEP() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream<Event> input = env.fromElements( new Event(1, "barfoo", 1.0), new Event(2, "start", 2.0), new Event(3, "foobar", 3.0), new SubEvent(4, "foo", 4.0, 1.0), new Event(5, "middle", 5.0), new SubEvent(6, "middle", 6.0, 2.0), new SubEvent(7, "bar", 3.0, 3.0), new Event(42, "42", 42.0), new Event(8, "end", 1.0)); Pattern<Event, ?> pattern = Pattern.<Event>begin("start") .where( new FilterFunction<Event>() { @Override public boolean filter(Event value) throws Exception { return value.getName().equals("start"); } }) .followedBy("middle") .subtype(SubEvent.class) .where( new FilterFunction<SubEvent>() { @Override public boolean filter(SubEvent value) throws Exception { return value.getName().equals("middle"); } }) .followedBy("end") .where( new FilterFunction<Event>() { @Override public boolean filter(Event value) throws Exception { return value.getName().equals("end"); } }); DataStream<String> result = CEP.pattern(input, pattern) .select( new PatternSelectFunction<Event, String>() { @Override public String select(Map<String, Event> pattern) { StringBuilder builder = new StringBuilder(); builder .append(pattern.get("start").getId()) .append(",") .append(pattern.get("middle").getId()) .append(",") .append(pattern.get("end").getId()); return builder.toString(); } }); result.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE); // expected sequence of matching event ids expected = "2,6,8"; env.execute(); }
@Test public void testSimpleKeyedPatternEventTime() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.setParallelism(2); // (Event, timestamp) DataStream<Event> input = env.fromElements( Tuple2.of(new Event(1, "start", 1.0), 5L), Tuple2.of(new Event(1, "middle", 2.0), 1L), Tuple2.of(new Event(2, "middle", 2.0), 4L), Tuple2.of(new Event(2, "start", 2.0), 3L), Tuple2.of(new Event(1, "end", 3.0), 3L), Tuple2.of(new Event(3, "start", 4.1), 5L), Tuple2.of(new Event(1, "end", 4.0), 10L), Tuple2.of(new Event(2, "end", 2.0), 8L), Tuple2.of(new Event(1, "middle", 5.0), 7L), Tuple2.of(new Event(3, "middle", 6.0), 9L), Tuple2.of(new Event(3, "end", 7.0), 7L), // last element for high final watermark Tuple2.of(new Event(3, "end", 7.0), 100L)) .assignTimestampsAndWatermarks( new AssignerWithPunctuatedWatermarks<Tuple2<Event, Long>>() { @Override public long extractTimestamp(Tuple2<Event, Long> element, long currentTimestamp) { return element.f1; } @Override public Watermark checkAndGetNextWatermark( Tuple2<Event, Long> lastElement, long extractedTimestamp) { return new Watermark(lastElement.f1 - 5); } }) .map( new MapFunction<Tuple2<Event, Long>, Event>() { @Override public Event map(Tuple2<Event, Long> value) throws Exception { return value.f0; } }) .keyBy( new KeySelector<Event, Integer>() { @Override public Integer getKey(Event value) throws Exception { return value.getId(); } }); Pattern<Event, ?> pattern = Pattern.<Event>begin("start") .where( new FilterFunction<Event>() { @Override public boolean filter(Event value) throws Exception { return value.getName().equals("start"); } }) .followedBy("middle") .where( new FilterFunction<Event>() { @Override public boolean filter(Event value) throws Exception { return value.getName().equals("middle"); } }) .followedBy("end") .where( new FilterFunction<Event>() { @Override public boolean filter(Event value) throws Exception { return value.getName().equals("end"); } }); DataStream<String> result = CEP.pattern(input, pattern) .select( new PatternSelectFunction<Event, String>() { @Override public String select(Map<String, Event> pattern) { StringBuilder builder = new StringBuilder(); builder .append(pattern.get("start").getId()) .append(",") .append(pattern.get("middle").getId()) .append(",") .append(pattern.get("end").getId()); return builder.toString(); } }); result.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE); // the expected sequences of matching event ids expected = "1,1,1\n2,2,2"; env.execute(); }