Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4195#discussion_r124550121
  
    --- Diff: 
flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
 ---
    @@ -297,6 +297,85 @@ public void testKeyedAdvancingTimeWithoutElements() 
throws Exception {
        }
     
        @Test
    +   public void testKeyedCEPOperatorNFAChanged() throws Exception {
    --- End diff --
    
    I meant rather checking if the resetting for the flag works correct. We 
could do it by veryfying correct number of invocations of `update` method. Also 
we should test it with both RocksDB and InMemory state backends.
    
    We can do it with Mockito. Just a suggestion for the code:
    
        @Test
        public void testKeyedCEPOperatorNFAChanged() throws Exception {
    
                String rocksDbPath = tempFolder.newFolder().getAbsolutePath();
                RocksDBStateBackend rocksDBStateBackend = new 
RocksDBStateBackend(new MemoryStateBackend());
                rocksDBStateBackend.setDbStoragePath(rocksDbPath);
    
                KeyedCEPPatternOperator<Event, Integer> operator = new 
KeyedCEPPatternOperator<>(
                        Event.createTypeSerializer(),
                        true,
                        IntSerializer.INSTANCE,
                        new SimpleNFAFactory(),
                        true);
                OneInputStreamOperatorTestHarness<Event, Map<String, 
List<Event>>> harness = getCepTestHarness(operator);
    
                try {
                        harness.setStateBackend(rocksDBStateBackend);
    
                        harness.open();
    
                        final ValueState nfaOperatorState = 
Whitebox.<ValueState>getInternalState(operator, "nfaOperatorState");
                        final ValueState nfaOperatorStateSpy = 
Mockito.spy(nfaOperatorState);
                        Whitebox.setInternalState(operator, "nfaOperatorState", 
nfaOperatorStateSpy);
    
                        Event startEvent = new Event(42, "c", 1.0);
                        SubEvent middleEvent = new SubEvent(42, "a", 1.0, 10.0);
                        Event endEvent = new Event(42, "b", 1.0);
    
                        harness.processElement(new StreamRecord<>(startEvent, 
1L));
                        harness.processElement(new StreamRecord<>(new Event(42, 
"d", 1.0), 4L));
                        harness.processElement(new 
StreamRecord<Event>(middleEvent, 4L));
                        harness.processElement(new StreamRecord<>(endEvent, 
4L));
    
                        Mockito.verify(nfaOperatorStateSpy, 
Mockito.times(3)).update(Mockito.any());
                        // get and verify the output
    
                        Queue<Object> result = harness.getOutput();
    
                        assertEquals(1, result.size());
    
                        verifyPattern(result.poll(), startEvent, middleEvent, 
endEvent);
                } finally {
                        harness.close();
                }
        }


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to