You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by yestinchen <gi...@git.apache.org> on 2017/08/31 21:15:46 UTC

[GitHub] flink pull request #4632: [FLINK-7563] [cep] Fix watermark semantics in cep ...

GitHub user yestinchen opened a pull request:

    https://github.com/apache/flink/pull/4632

    [FLINK-7563] [cep] Fix watermark semantics in cep and related tests.

    ## What is the purpose of the change
    
    Correct the watermark semantics in cep.
    
    ## Brief change log
    
    -  use the logic that (timestamp <= watermark) is considered late.
    
    ## Verifying this change
    
    *(Please pick either of the following options)*
    
    This change is already covered by existing tests, such as `testCEPOperatorCleanupEventTime()`, and `testCEPOperatorSerializationWRocksDB()` in `CEPOperatorTest.java`
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (no)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (not applicable)
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/yestinchen/flink FLINK-7563

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4632.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4632
    
----
commit 419921c81be20b0c00dfeaba287f603af2600fad
Author: Yestin <87...@qq.com>
Date:   2017-08-31T21:09:30Z

    [FLINK-7563] [cep] Fix watermark semantics in cep and related tests.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4632: [FLINK-7563] [cep] Fix watermark semantics in cep and rel...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on the issue:

    https://github.com/apache/flink/pull/4632
  
    In fact you were right @yestinchen  in the first commit. I was mistaken with my comment :( Sorry for that. I am merging your first version.


---

[GitHub] flink issue #4632: [FLINK-7563] [cep] Fix watermark semantics in cep and rel...

Posted by yestinchen <gi...@git.apache.org>.
Github user yestinchen commented on the issue:

    https://github.com/apache/flink/pull/4632
  
    I added the test case to the `CEP Operator` and addressed the line. Thanks so much.


---

[GitHub] flink pull request #4632: [FLINK-7563] [cep] Fix watermark semantics in cep ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4632


---

[GitHub] flink issue #4632: [FLINK-7563] [cep] Fix watermark semantics in cep and rel...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on the issue:

    https://github.com/apache/flink/pull/4632
  
    Hi @yestinchen,
    Thanks for working on this issue. Unfortunately your changes do not fix the semantics. Have a look at this failing test:
    
    	@Test
    	public void testCEPWatermarkSemantic() throws Exception {
    		Event startEvent = new Event(42, "start", 1.0);
    		SubEvent middleEvent = new SubEvent(42, "foo2", 2.0, 10.0);
    		Event endEvent = new Event(42, "end", 1.0);
    
    		SelectCepOperator<Event, Integer, Map<String, List<Event>>> operator = getKeyedCepOperatorWithComparator(false);
    
    		try (OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness = CepOperatorTestUtilities.getCepTestHarness(operator)) {
    			harness.open();
    
    			harness.processWatermark(0L);
    			verifyWatermark(harness.getOutput().poll(), 0L);
    
    			harness.processElement(new StreamRecord<>(startEvent, 1L));
    			harness.processElement(new StreamRecord<>(middleEvent, 2L));
    			harness.processElement(new StreamRecord<>(endEvent, 3L));
    
    			assertTrue(operator.hasNonEmptyPQ(42));
    			assertTrue(!operator.hasNonEmptyNFA(42));
    
    			harness.processWatermark(3L);
    			verifyWatermark(harness.getOutput().poll(), 3L);
    
    			assertTrue(operator.hasNonEmptyPQ(42));
    			assertTrue(operator.hasNonEmptyNFA(42));
    
    			harness.processWatermark(4L);
    			assertTrue(!operator.hasNonEmptyPQ(42));
    
    			verifyPattern(harness.getOutput().poll(), startEvent, middleEvent, endEvent);
    			verifyWatermark(harness.getOutput().poll(), 4L);
    		}
    	}
    
    To fix it you also have to fix the condition in `AbstractKeyedCEPPatternOperator::onEventTime:232`. Instead of:
    
    	while (!sortedTimestamps.isEmpty() && sortedTimestamps.peek() <= timerService.currentWatermark()) {
    
    it should be 
    
    	while (!sortedTimestamps.isEmpty() && sortedTimestamps.peek() < timerService.currentWatermark()) {


---