DataflowPipelineJob run(Pipeline pipeline, DataflowRunner runner) { updatePAssertCount(pipeline); jobSuccess = waitForStreamingJobTermination(job, messageHandler); jobSuccess = waitForBatchJobTermination(job, messageHandler); allAssertionsPassed = checkForPAssertSuccess(job); LOG.warn("Dataflow job {} did not output a success or failure metric.", job.getJobId()); } else if (!allAssertionsPassed.get()) { throw new AssertionError(errorMessage(job, messageHandler)); throw new RuntimeException(errorMessage(job, messageHandler));
@Override public DataflowPipelineJob run(Pipeline pipeline) { return run(pipeline, runner); }
@VisibleForTesting static TestDataflowRunner fromOptionsAndClient( TestDataflowPipelineOptions options, DataflowClient client) { return new TestDataflowRunner(options, client); }
@Test public void testToString() { assertEquals( "TestDataflowRunner#TestAppName", TestDataflowRunner.fromOptions(options).toString()); }
@Test public void testRunStreamingJobNotUsingPAssertThatSucceeds() throws Exception { options.setStreaming(true); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); when(mockJob.getState()).thenReturn(State.DONE); when(mockJob.waitUntilFinish(any(Duration.class), any(JobMessagesHandler.class))) .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); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockStreamingMetricResponse(ImmutableMap.of())); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); runner.run(p, mockRunner); }
@Test public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception { DataflowPipelineJob job = spy(new DataflowPipelineJob(mockClient, "test-job", options, null)); Pipeline p = TestPipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); when(mockClient.getJobMetrics(anyString())) .thenReturn( buildJobMetrics(generateMockMetrics(true /* success */, false /* tentative */))); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); runner.updatePAssertCount(p); doReturn(State.RUNNING).when(job).getState(); assertThat(runner.checkForPAssertSuccess(job), equalTo(Optional.<Boolean>absent())); }
@Test public void testGetJobMetricsThatFailsForException() throws Exception { DataflowPipelineJob job = spy(new DataflowPipelineJob(mockClient, "test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); when(mockClient.getJobMetrics(anyString())).thenThrow(new IOException()); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); assertNull(runner.getJobMetrics(job)); }
/** * Tests that a tentative {@code true} from metrics indicates that every {@link PAssert} has * succeeded. */ @Test public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception { DataflowPipelineJob job = spy(new DataflowPipelineJob(mockClient, "test-job", options, null)); Pipeline p = TestPipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); when(mockClient.getJobMetrics(anyString())) .thenReturn(buildJobMetrics(generateMockMetrics(true /* success */, true /* tentative */))); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); doReturn(State.DONE).when(job).getState(); assertThat(runner.checkForPAssertSuccess(job), equalTo(Optional.of(true))); }
Optional<Boolean> checkForPAssertSuccess(DataflowPipelineJob job) { JobMetrics metrics = getJobMetrics(job); if (metrics == null || metrics.getMetrics() == null) { LOG.warn("Metrics not present for Dataflow job {}.", job.getJobId());
@Test public void testRunBatchJobThatSucceeds() throws Exception { Pipeline p = Pipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); 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); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, true /* tentative */)); assertEquals(mockJob, runner.run(p, mockRunner)); }
@Test public void testGetJobMetricsThatSucceeds() throws Exception { DataflowPipelineJob job = spy(new DataflowPipelineJob(mockClient, "test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, true /* tentative */)); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); JobMetrics metrics = runner.getJobMetrics(job); assertEquals(1, metrics.getMetrics().size()); assertEquals( generateMockMetrics(true /* success */, true /* tentative */), metrics.getMetrics()); }
/** * Tests that when we just see a tentative failure for a {@link PAssert} it is considered a * conclusive failure. */ @Test public void testCheckingForSuccessWhenPAssertFails() throws Exception { DataflowPipelineJob job = spy(new DataflowPipelineJob(mockClient, "test-job", options, null)); Pipeline p = TestPipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); when(mockClient.getJobMetrics(anyString())) .thenReturn( buildJobMetrics(generateMockMetrics(false /* success */, true /* tentative */))); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); doReturn(State.DONE).when(job).getState(); assertThat(runner.checkForPAssertSuccess(job), equalTo(Optional.of(false))); }
@Test public void testBatchOnCreateMatcher() 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).setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0)); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, true /* tentative */)); runner.run(p, mockRunner); }
/** Constructs a runner from the provided options. */ public static TestDataflowRunner fromOptions(PipelineOptions options) { TestDataflowPipelineOptions dataflowOptions = options.as(TestDataflowPipelineOptions.class); String tempLocation = Joiner.on("/") .join(dataflowOptions.getTempRoot(), dataflowOptions.getJobName(), "output", "results"); dataflowOptions.setTempLocation(tempLocation); return new TestDataflowRunner( dataflowOptions, DataflowClient.create(options.as(DataflowPipelineOptions.class))); }
@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); }
/** A streaming job that terminates with no error messages is a success. */ @Test public void testRunStreamingJobUsingPAssertThatSucceeds() 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); DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); when(mockJob.getState()).thenReturn(State.DONE); when(mockJob.waitUntilFinish(any(Duration.class), any(JobMessagesHandler.class))) .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); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, true /* tentative */)); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); runner.run(p, mockRunner); }
@Test public void testStreamingOnCreateMatcher() 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).setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0)); 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 batch job terminates in a failure state even if all assertions passed, it * throws an error to that effect. */ @Test public void testRunBatchJobThatFails() throws Exception { Pipeline p = TestPipeline.create(options); PCollection<Integer> pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); 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); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, false /* tentative */)); expectedException.expect(RuntimeException.class); runner.run(p, mockRunner); // Note that fail throws an AssertionError which is why it is placed out here // instead of inside the try-catch block. fail("AssertionError expected"); }
@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); }