@Test public void testBatchOnSuccessMatcherWhenPipelineSucceeds() throws Exception { Pipeline p = TestPipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); when(mockJob.getState()).thenReturn(State.DONE); when(mockJob.getProjectId()).thenReturn("test-project"); when(mockJob.getJobId()).thenReturn("test-job"); DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class); when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); options.as(TestPipelineOptions.class).setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1)); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, true /* tentative */)); runner.run(p, mockRunner); }
@Test public void testBatchOnSuccessMatcherWhenPipelineFails() throws Exception { Pipeline p = TestPipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); when(mockJob.getState()).thenReturn(State.FAILED); when(mockJob.getProjectId()).thenReturn("test-project"); when(mockJob.getJobId()).thenReturn("test-job"); DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class); when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); options.as(TestPipelineOptions.class).setOnSuccessMatcher(new TestFailureMatcher()); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(false /* success */, true /* tentative */)); try { runner.run(p, mockRunner); } catch (AssertionError expected) { verify(mockJob, Mockito.times(1)) .waitUntilFinish(any(Duration.class), any(JobMessagesHandler.class)); return; } fail("Expected an exception on pipeline failure."); }
/** * Tests that when a streaming pipeline terminates and doesn't fail due to {@link PAssert} that * the {@link TestPipelineOptions#setOnSuccessMatcher(SerializableMatcher) on success matcher} is * invoked. */ @Test public void testStreamingOnSuccessMatcherWhenPipelineSucceeds() throws Exception { options.setStreaming(true); Pipeline p = TestPipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); when(mockJob.getState()).thenReturn(State.DONE); when(mockJob.getProjectId()).thenReturn("test-project"); when(mockJob.getJobId()).thenReturn("test-job"); DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class); when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); options.as(TestPipelineOptions.class).setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1)); when(mockJob.waitUntilFinish(any(Duration.class), any(JobMessagesHandler.class))) .thenReturn(State.DONE); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, true /* tentative */)); runner.run(p, mockRunner); }
/** * Tests that when a streaming pipeline terminates in FAIL that the {@link * TestPipelineOptions#setOnSuccessMatcher(SerializableMatcher) on success matcher} is not * invoked. */ @Test public void testStreamingOnSuccessMatcherWhenPipelineFails() throws Exception { options.setStreaming(true); Pipeline p = TestPipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); when(mockJob.getState()).thenReturn(State.FAILED); when(mockJob.getProjectId()).thenReturn("test-project"); when(mockJob.getJobId()).thenReturn("test-job"); DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class); when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); options.as(TestPipelineOptions.class).setOnSuccessMatcher(new TestFailureMatcher()); when(mockJob.waitUntilFinish(any(Duration.class), any(JobMessagesHandler.class))) .thenReturn(State.FAILED); expectedException.expect(RuntimeException.class); runner.run(p, mockRunner); // If the onSuccessMatcher were invoked, it would have crashed here with AssertionError }
.toString(); options.setOnSuccessMatcher( SerializableMatchers.allOf( new FileChecksumMatcher(