@Test public void testEmitMaxWatermarkForFiniteSource() throws Exception { // regular stream source operator StreamSource<String, FiniteSource<String>> operator = new StreamSource<>(new FiniteSource<String>()); final List<StreamElement> output = new ArrayList<>(); setupSourceOperator(operator, TimeCharacteristic.EventTime, 0); operator.run(new Object(), mock(StreamStatusMaintainer.class), new CollectorOutput<String>(output)); assertEquals(1, output.size()); assertEquals(Watermark.MAX_WATERMARK, output.get(0)); }
@Test public void testNoMaxWatermarkOnImmediateStop() throws Exception { final List<StreamElement> output = new ArrayList<>(); // regular stream source operator final StoppableStreamSource<String, InfiniteSource<String>> operator = new StoppableStreamSource<>(new InfiniteSource<String>()); setupSourceOperator(operator, TimeCharacteristic.EventTime, 0); operator.stop(); // run and stop operator.run(new Object(), mock(StreamStatusMaintainer.class), new CollectorOutput<String>(output)); assertTrue(output.isEmpty()); }
@Test public void testNoMaxWatermarkOnImmediateCancel() throws Exception { final List<StreamElement> output = new ArrayList<>(); // regular stream source operator final StreamSource<String, InfiniteSource<String>> operator = new StreamSource<>(new InfiniteSource<String>()); setupSourceOperator(operator, TimeCharacteristic.EventTime, 0); operator.cancel(); // run and exit operator.run(new Object(), mock(StreamStatusMaintainer.class), new CollectorOutput<String>(output)); assertTrue(output.isEmpty()); }
@SuppressWarnings("unchecked") private static <T> void setupSourceOperator(StreamSource<T, ?> operator, TimeCharacteristic timeChar, long watermarkInterval, final ProcessingTimeService timeProvider) { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.setAutoWatermarkInterval(watermarkInterval); StreamConfig cfg = new StreamConfig(new Configuration()); cfg.setStateBackend(new MemoryStateBackend()); cfg.setTimeCharacteristic(timeChar); cfg.setOperatorID(new OperatorID()); Environment env = new DummyEnvironment("MockTwoInputTask", 1, 0); StreamStatusMaintainer streamStatusMaintainer = mock(StreamStatusMaintainer.class); when(streamStatusMaintainer.getStreamStatus()).thenReturn(StreamStatus.ACTIVE); StreamTask<?, ?> mockTask = mock(StreamTask.class); when(mockTask.getName()).thenReturn("Mock Task"); when(mockTask.getCheckpointLock()).thenReturn(new Object()); when(mockTask.getConfiguration()).thenReturn(cfg); when(mockTask.getEnvironment()).thenReturn(env); when(mockTask.getExecutionConfig()).thenReturn(executionConfig); when(mockTask.getAccumulatorMap()).thenReturn(Collections.<String, Accumulator<?, ?>>emptyMap()); when(mockTask.getStreamStatusMaintainer()).thenReturn(streamStatusMaintainer); doAnswer(new Answer<ProcessingTimeService>() {
@Test public void testNoMaxWatermarkOnAsyncCancel() throws Exception { final List<StreamElement> output = new ArrayList<>(); final Thread runner = Thread.currentThread(); // regular stream source operator final StreamSource<String, InfiniteSource<String>> operator = new StreamSource<>(new InfiniteSource<String>()); setupSourceOperator(operator, TimeCharacteristic.EventTime, 0); // trigger an async cancel in a bit new Thread("canceler") { @Override public void run() { try { Thread.sleep(200); } catch (InterruptedException ignored) {} operator.cancel(); runner.interrupt(); } }.start(); // run and wait to be canceled try { operator.run(new Object(), mock(StreamStatusMaintainer.class), new CollectorOutput<String>(output)); } catch (InterruptedException ignored) {} assertTrue(output.isEmpty()); }
@Test public void testNoMaxWatermarkOnAsyncStop() throws Exception { final List<StreamElement> output = new ArrayList<>(); // regular stream source operator final StoppableStreamSource<String, InfiniteSource<String>> operator = new StoppableStreamSource<>(new InfiniteSource<String>()); setupSourceOperator(operator, TimeCharacteristic.EventTime, 0); // trigger an async cancel in a bit new Thread("canceler") { @Override public void run() { try { Thread.sleep(200); } catch (InterruptedException ignored) {} operator.stop(); } }.start(); // run and wait to be stopped operator.run(new Object(), mock(StreamStatusMaintainer.class), new CollectorOutput<String>(output)); assertTrue(output.isEmpty()); }
processingTimeService.setCurrentTime(0); setupSourceOperator(operator, TimeCharacteristic.IngestionTime, watermarkInterval, processingTimeService);