/** * Applies the given {@link KeyedProcessFunction} on the input stream, thereby creating a transformed output stream. * * <p>The function will be called for every element in the input streams and can produce zero * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} * function, this function can also query the time and set timers. When reacting to the firing * of set timers the function can directly emit elements and/or register yet more timers. * * @param keyedProcessFunction The {@link KeyedProcessFunction} that is called for each element in the stream. * * @param outputType {@link TypeInformation} for the result type of the function. * * @param <R> The type of elements emitted by the {@code KeyedProcessFunction}. * * @return The transformed {@link DataStream}. */ @Internal public <R> SingleOutputStreamOperator<R> process( KeyedProcessFunction<KEY, T, R> keyedProcessFunction, TypeInformation<R> outputType) { KeyedProcessOperator<KEY, T, R> operator = new KeyedProcessOperator<>(clean(keyedProcessFunction)); return transform("KeyedProcess", outputType, operator); }
@Override public void open() throws Exception { super.open(); collector = new TimestampedCollector<>(output); InternalTimerService<VoidNamespace> internalTimerService = getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this); TimerService timerService = new SimpleTimerService(internalTimerService); context = new ContextImpl(userFunction, timerService); onTimerContext = new OnTimerContextImpl(userFunction, timerService); }
@Override public void onProcessingTime(InternalTimer<K, VoidNamespace> timer) throws Exception { collector.eraseTimestamp(); invokeUserFunction(TimeDomain.PROCESSING_TIME, timer); }
@Test public void testEventTimeTimers() throws Exception { final int expectedKey = 17; KeyedProcessOperator<Integer, Integer, Integer> operator = new KeyedProcessOperator<>(new TriggeringFlatMapFunction(TimeDomain.EVENT_TIME, expectedKey)); OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector<Integer>(), BasicTypeInfo.INT_TYPE_INFO); testHarness.setup(); testHarness.open(); testHarness.processWatermark(new Watermark(0)); testHarness.processElement(new StreamRecord<>(expectedKey, 42L)); testHarness.processWatermark(new Watermark(5)); ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new Watermark(0L)); expectedOutput.add(new StreamRecord<>(expectedKey, 42L)); expectedOutput.add(new StreamRecord<>(1777, 5L)); expectedOutput.add(new Watermark(5L)); TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); }
@Override public void open() throws Exception { super.open(); collector = new TimestampedCollector<>(output); InternalTimerService<VoidNamespace> internalTimerService = getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this); TimerService timerService = new SimpleTimerService(internalTimerService); context = new ContextImpl(userFunction, timerService); onTimerContext = new OnTimerContextImpl(userFunction, timerService); }
@Override public void onEventTime(InternalTimer<K, VoidNamespace> timer) throws Exception { collector.setAbsoluteTimestamp(timer.getTimestamp()); invokeUserFunction(TimeDomain.EVENT_TIME, timer); }
KeyedProcessOperator<Integer, Integer, String> operator = new KeyedProcessOperator<>(new SideOutputProcessFunction());
@Override public void open() throws Exception { super.open(); collector = new TimestampedCollector<>(output); InternalTimerService<VoidNamespace> internalTimerService = getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this); TimerService timerService = new SimpleTimerService(internalTimerService); context = new ContextImpl(userFunction, timerService); onTimerContext = new OnTimerContextImpl(userFunction, timerService); }
@Override public void onProcessingTime(InternalTimer<K, VoidNamespace> timer) throws Exception { collector.eraseTimestamp(); invokeUserFunction(TimeDomain.PROCESSING_TIME, timer); }
@Test public void testProcessingTimeTimers() throws Exception { final int expectedKey = 17; KeyedProcessOperator<Integer, Integer, Integer> operator = new KeyedProcessOperator<>(new TriggeringFlatMapFunction(TimeDomain.PROCESSING_TIME, expectedKey)); OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector<Integer>(), BasicTypeInfo.INT_TYPE_INFO); testHarness.setup(); testHarness.open(); testHarness.processElement(new StreamRecord<>(expectedKey)); testHarness.setProcessingTime(5); ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new StreamRecord<>(expectedKey)); expectedOutput.add(new StreamRecord<>(1777)); TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); }
@Override public void open() throws Exception { super.open(); collector = new TimestampedCollector<>(output); InternalTimerService<VoidNamespace> internalTimerService = getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this); TimerService timerService = new SimpleTimerService(internalTimerService); context = new ContextImpl(userFunction, timerService); onTimerContext = new OnTimerContextImpl(userFunction, timerService); }
@Override public void onProcessingTime(InternalTimer<K, VoidNamespace> timer) throws Exception { collector.eraseTimestamp(); invokeUserFunction(TimeDomain.PROCESSING_TIME, timer); }
@Test public void testTimestampAndWatermarkQuerying() throws Exception { KeyedProcessOperator<Integer, Integer, String> operator = new KeyedProcessOperator<>(new QueryingFlatMapFunction(TimeDomain.EVENT_TIME)); OneInputStreamOperatorTestHarness<Integer, String> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector<Integer>(), BasicTypeInfo.INT_TYPE_INFO); testHarness.setup(); testHarness.open(); testHarness.processWatermark(new Watermark(17)); testHarness.processElement(new StreamRecord<>(5, 12L)); testHarness.processWatermark(new Watermark(42)); testHarness.processElement(new StreamRecord<>(6, 13L)); ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new Watermark(17L)); expectedOutput.add(new StreamRecord<>("5TIME:17 TS:12", 12L)); expectedOutput.add(new Watermark(42L)); expectedOutput.add(new StreamRecord<>("6TIME:42 TS:13", 13L)); TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); }
@Override public void onEventTime(InternalTimer<K, VoidNamespace> timer) throws Exception { collector.setAbsoluteTimestamp(timer.getTimestamp()); invokeUserFunction(TimeDomain.EVENT_TIME, timer); }
new KeyedProcessOperator<>(new KeyQueryingProcessFunction());
@Override public void onEventTime(InternalTimer<K, VoidNamespace> timer) throws Exception { collector.setAbsoluteTimestamp(timer.getTimestamp()); invokeUserFunction(TimeDomain.EVENT_TIME, timer); }
@Test public void testTimestampAndProcessingTimeQuerying() throws Exception { KeyedProcessOperator<Integer, Integer, String> operator = new KeyedProcessOperator<>(new QueryingFlatMapFunction(TimeDomain.PROCESSING_TIME)); OneInputStreamOperatorTestHarness<Integer, String> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector<Integer>(), BasicTypeInfo.INT_TYPE_INFO); testHarness.setup(); testHarness.open(); testHarness.setProcessingTime(17); testHarness.processElement(new StreamRecord<>(5)); testHarness.setProcessingTime(42); testHarness.processElement(new StreamRecord<>(6)); ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new StreamRecord<>("5TIME:17 TS:null")); expectedOutput.add(new StreamRecord<>("6TIME:42 TS:null")); TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); }
new KeyedProcessOperator<>(new TriggeringStatefulFlatMapFunction(TimeDomain.EVENT_TIME));
@Test public void testSnapshotAndRestore() throws Exception { final int expectedKey = 5; KeyedProcessOperator<Integer, Integer, String> operator = new KeyedProcessOperator<>(new BothTriggeringFlatMapFunction(expectedKey)); OneInputStreamOperatorTestHarness<Integer, String> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector<Integer>(), BasicTypeInfo.INT_TYPE_INFO); testHarness.setup(); testHarness.open(); testHarness.processElement(new StreamRecord<>(expectedKey, 12L)); // snapshot and restore from scratch OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); testHarness.close(); operator = new KeyedProcessOperator<>(new BothTriggeringFlatMapFunction(expectedKey)); testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector<Integer>(), BasicTypeInfo.INT_TYPE_INFO); testHarness.setup(); testHarness.initializeState(snapshot); testHarness.open(); testHarness.setProcessingTime(5); testHarness.processWatermark(new Watermark(6)); ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new StreamRecord<>("PROC:1777")); expectedOutput.add(new StreamRecord<>("EVENT:1777", 6L)); expectedOutput.add(new Watermark(6)); TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); }
/** * Verifies that we don't have leakage between different keys. */ @Test public void testProcessingTimeTimerWithState() throws Exception { KeyedProcessOperator<Integer, Integer, String> operator = new KeyedProcessOperator<>(new TriggeringStatefulFlatMapFunction(TimeDomain.PROCESSING_TIME)); OneInputStreamOperatorTestHarness<Integer, String> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector<Integer>(), BasicTypeInfo.INT_TYPE_INFO); testHarness.setup(); testHarness.open(); testHarness.setProcessingTime(1); testHarness.processElement(new StreamRecord<>(17)); // should set timer for 6 testHarness.processElement(new StreamRecord<>(13)); // should set timer for 6 testHarness.setProcessingTime(2); testHarness.processElement(new StreamRecord<>(13)); // should delete timer testHarness.processElement(new StreamRecord<>(42)); // should set timer for 7 testHarness.setProcessingTime(6); testHarness.setProcessingTime(7); ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new StreamRecord<>("INPUT:17")); expectedOutput.add(new StreamRecord<>("INPUT:13")); expectedOutput.add(new StreamRecord<>("INPUT:42")); expectedOutput.add(new StreamRecord<>("STATE:17")); expectedOutput.add(new StreamRecord<>("STATE:42")); TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); }