private static <IN1, IN2, OUT> TwoInputStreamOperatorTestHarness<IN1, IN2, OUT> getInitializedTestHarness( final BroadcastProcessFunction<IN1, IN2, OUT> function, final MapStateDescriptor<?, ?>... descriptors) throws Exception { return getInitializedTestHarness( function, 1, 1, 0, descriptors); }
private static <IN1, IN2, OUT> TwoInputStreamOperatorTestHarness<IN1, IN2, OUT> getInitializedTestHarness( final BroadcastProcessFunction<IN1, IN2, OUT> function, final int maxParallelism, final int numTasks, final int taskIdx, final MapStateDescriptor<?, ?>... descriptors) throws Exception { return getInitializedTestHarness( function, maxParallelism, numTasks, taskIdx, null, descriptors); }
@Test public void testMultiStateSupport() throws Exception { try ( TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness = getInitializedTestHarness(new FunctionWithMultipleStates(), STATE_DESCRIPTOR, STATE_DESCRIPTOR_A) ) { testHarness.processElement2(new StreamRecord<>(5, 12L)); testHarness.processElement2(new StreamRecord<>(6, 13L)); testHarness.processElement1(new StreamRecord<>("9", 15L)); Queue<Object> expectedBr = new ConcurrentLinkedQueue<>(); expectedBr.add(new StreamRecord<>("9:key.6->6", 15L)); expectedBr.add(new StreamRecord<>("9:key.5->5", 15L)); expectedBr.add(new StreamRecord<>("9:5->value.5", 15L)); expectedBr.add(new StreamRecord<>("9:6->value.6", 15L)); TestHarnessUtil.assertOutputEquals("Wrong Side Output", expectedBr, testHarness.getOutput()); } }
TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness1 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10, STATE_DESCRIPTOR); TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness2 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10, STATE_DESCRIPTOR); TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness3 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10, TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness1 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10, STATE_DESCRIPTOR); TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness2 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10,
TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness1 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10, STATE_DESCRIPTOR); TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness2 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10, TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness1 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10, STATE_DESCRIPTOR); TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness2 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10, STATE_DESCRIPTOR); TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness3 = getInitializedTestHarness( new TestFunctionWithOutput(keysToRegister), 10,
@Test public void testBroadcastState() throws Exception { final Set<String> keysToRegister = new HashSet<>(); keysToRegister.add("test1"); keysToRegister.add("test2"); keysToRegister.add("test3"); try ( TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness = getInitializedTestHarness( new TestFunction(keysToRegister), STATE_DESCRIPTOR) ) { testHarness.processWatermark1(new Watermark(10L)); testHarness.processWatermark2(new Watermark(10L)); testHarness.processElement2(new StreamRecord<>(5, 12L)); testHarness.processWatermark1(new Watermark(40L)); testHarness.processWatermark2(new Watermark(40L)); testHarness.processElement1(new StreamRecord<>("6", 13L)); testHarness.processElement1(new StreamRecord<>("6", 15L)); testHarness.processWatermark1(new Watermark(50L)); testHarness.processWatermark2(new Watermark(50L)); Queue<Object> expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new Watermark(10L)); expectedOutput.add(new StreamRecord<>("5WM:10 TS:12", 12L)); expectedOutput.add(new Watermark(40L)); expectedOutput.add(new StreamRecord<>("6WM:40 TS:13", 13L)); expectedOutput.add(new StreamRecord<>("6WM:40 TS:15", 15L)); expectedOutput.add(new Watermark(50L)); TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); } }
@Test public void testNoKeyedStateOnBroadcastSide() throws Exception { boolean exceptionThrown = false; try ( TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness = getInitializedTestHarness( new BroadcastProcessFunction<String, Integer, String>() { private static final long serialVersionUID = -1725365436500098384L; private final ValueStateDescriptor<String> valueState = new ValueStateDescriptor<>("any", BasicTypeInfo.STRING_TYPE_INFO); @Override public void processBroadcastElement(Integer value, Context ctx, Collector<String> out) throws Exception { getRuntimeContext().getState(valueState).value(); // this should fail } @Override public void processElement(String value, ReadOnlyContext ctx, Collector<String> out) throws Exception { // do nothing } }) ) { testHarness.processWatermark1(new Watermark(10L)); testHarness.processWatermark2(new Watermark(10L)); testHarness.processElement2(new StreamRecord<>(5, 12L)); } catch (NullPointerException e) { Assert.assertEquals("Keyed state can only be used on a 'keyed stream', i.e., after a 'keyBy()' operation.", e.getMessage()); exceptionThrown = true; } if (!exceptionThrown) { Assert.fail("No exception thrown"); } }
@Test public void testNoKeyedStateOnNonBroadcastSide() throws Exception { boolean exceptionThrown = false; try ( TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness = getInitializedTestHarness( new BroadcastProcessFunction<String, Integer, String>() { private static final long serialVersionUID = -1725365436500098384L; private final ValueStateDescriptor<String> valueState = new ValueStateDescriptor<>("any", BasicTypeInfo.STRING_TYPE_INFO); @Override public void processBroadcastElement(Integer value, Context ctx, Collector<String> out) throws Exception { // do nothing } @Override public void processElement(String value, ReadOnlyContext ctx, Collector<String> out) throws Exception { getRuntimeContext().getState(valueState).value(); // this should fail } }) ) { testHarness.processWatermark1(new Watermark(10L)); testHarness.processWatermark2(new Watermark(10L)); testHarness.processElement1(new StreamRecord<>("5", 12L)); } catch (NullPointerException e) { Assert.assertEquals("Keyed state can only be used on a 'keyed stream', i.e., after a 'keyBy()' operation.", e.getMessage()); exceptionThrown = true; } if (!exceptionThrown) { Assert.fail("No exception thrown"); } }
@Test public void testSideOutput() throws Exception { try ( TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness = getInitializedTestHarness( new FunctionWithSideOutput(), STATE_DESCRIPTOR) ) { testHarness.processWatermark1(new Watermark(10L)); testHarness.processWatermark2(new Watermark(10L)); testHarness.processElement2(new StreamRecord<>(5, 12L)); testHarness.processWatermark1(new Watermark(40L)); testHarness.processWatermark2(new Watermark(40L)); testHarness.processElement1(new StreamRecord<>("6", 13L)); testHarness.processElement1(new StreamRecord<>("6", 15L)); testHarness.processWatermark1(new Watermark(50L)); testHarness.processWatermark2(new Watermark(50L)); ConcurrentLinkedQueue<StreamRecord<String>> expectedBr = new ConcurrentLinkedQueue<>(); expectedBr.add(new StreamRecord<>("BR:5 WM:10 TS:12", 12L)); ConcurrentLinkedQueue<StreamRecord<String>> expectedNonBr = new ConcurrentLinkedQueue<>(); expectedNonBr.add(new StreamRecord<>("NON-BR:6 WM:40 TS:13", 13L)); expectedNonBr.add(new StreamRecord<>("NON-BR:6 WM:40 TS:15", 15L)); ConcurrentLinkedQueue<StreamRecord<String>> brSideOutput = testHarness.getSideOutput(FunctionWithSideOutput.BROADCAST_TAG); ConcurrentLinkedQueue<StreamRecord<String>> nonBrSideOutput = testHarness.getSideOutput(FunctionWithSideOutput.NON_BROADCAST_TAG); TestHarnessUtil.assertOutputEquals("Wrong Side Output", expectedBr, brSideOutput); TestHarnessUtil.assertOutputEquals("Wrong Side Output", expectedNonBr, nonBrSideOutput); } }