int key2 = getKeyInKeyGroupRange(subKeyGroupRange2, maxParallelism); TestOperator testOperator1 = new TestOperator(); testHarness1.setProcessingTime(0L); TestOperator testOperator2 = new TestOperator(); TestOperator testOperator3 = new TestOperator();
@Override public void processElement(StreamRecord<Tuple2<Integer, String>> element) throws Exception { String[] command = element.getValue().f1.split(":"); switch (command[0]) { case "SET_STATE": getPartitionedState(stateDescriptor).update(command[1]); break; case "DELETE_STATE": getPartitionedState(stateDescriptor).clear(); break; case "SET_EVENT_TIME_TIMER": timerService.registerEventTimeTimer(VoidNamespace.INSTANCE, Long.parseLong(command[1])); break; case "SET_PROC_TIME_TIMER": timerService.registerProcessingTimeTimer(VoidNamespace.INSTANCE, Long.parseLong(command[1])); break; case "EMIT_STATE": String stateValue = getPartitionedState(stateDescriptor).value(); output.collect(new StreamRecord<>("ON_ELEMENT:" + element.getValue().f0 + ":" + stateValue)); break; default: throw new IllegalArgumentException(); } }
int key2 = getKeyInKeyGroupRange(subKeyGroupRange2, maxParallelism); TestOperator testOperator = new TestOperator(); TestOperator testOperator1 = new TestOperator(); TestOperator testOperator2 = new TestOperator();
TestOperator testOperator = new TestOperator(); TestOperator testOperator1 = new TestOperator();
/** * Verify that firing processing-time timers see the state of the key that was active * when the timer was set. */ @Test public void testProcessingTimeTimersDontInterfere() throws Exception { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, String> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); testHarness.setProcessingTime(0L); testHarness.processElement(new Tuple2<>(1, "SET_PROC_TIME_TIMER:20"), 0); testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); testHarness.processElement(new Tuple2<>(0, "SET_PROC_TIME_TIMER:10"), 0); testHarness.setProcessingTime(10L); assertThat( extractResult(testHarness), contains("ON_PROC_TIME:HELLO")); testHarness.setProcessingTime(20L); assertThat( extractResult(testHarness), contains("ON_PROC_TIME:CIAO")); }
/** * Verify that firing event-time timers see the state of the key that was active * when the timer was set. */ @Test public void testEventTimeTimersDontInterfere() throws Exception { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, String> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); testHarness.processWatermark(0L); testHarness.processElement(new Tuple2<>(1, "SET_EVENT_TIME_TIMER:20"), 0); testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); testHarness.processElement(new Tuple2<>(0, "SET_EVENT_TIME_TIMER:10"), 0); testHarness.processWatermark(10L); assertThat( extractResult(testHarness), contains("ON_EVENT_TIME:HELLO")); testHarness.processWatermark(20L); assertThat( extractResult(testHarness), contains("ON_EVENT_TIME:CIAO")); }
/** * Verify that timers for the different time domains don't clash. */ @Test public void testProcessingTimeAndEventTimeDontInterfere() throws Exception { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, String> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); testHarness.setProcessingTime(0L); testHarness.processWatermark(0L); testHarness.processElement(new Tuple2<>(0, "SET_PROC_TIME_TIMER:10"), 0); testHarness.processElement(new Tuple2<>(0, "SET_EVENT_TIME_TIMER:20"), 0); testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); testHarness.processWatermark(20L); assertThat( extractResult(testHarness), contains("ON_EVENT_TIME:HELLO")); testHarness.setProcessingTime(10L); assertThat( extractResult(testHarness), contains("ON_PROC_TIME:HELLO")); }
@Test public void testStateDoesNotInterfere() throws Exception { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, String> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); testHarness.processElement(new Tuple2<>(1, "EMIT_STATE"), 0); testHarness.processElement(new Tuple2<>(0, "EMIT_STATE"), 0); assertThat( extractResult(testHarness), contains("ON_ELEMENT:1:CIAO", "ON_ELEMENT:0:HELLO")); }
@Override public void onProcessingTime(InternalTimer<Integer, VoidNamespace> timer) throws Exception { String stateValue = getPartitionedState(stateDescriptor).value(); output.collect(new StreamRecord<>("ON_PROC_TIME:" + stateValue)); } }
@Override public void onEventTime(InternalTimer<Integer, VoidNamespace> timer) throws Exception { String stateValue = getPartitionedState(stateDescriptor).value(); output.collect(new StreamRecord<>("ON_EVENT_TIME:" + stateValue)); }
@Override public void open() throws Exception { super.open(); this.timerService = getInternalTimerService( "test-timers", VoidNamespaceSerializer.INSTANCE, this); }