From b64a2fd091fc970526215f5e7bafff8fff4c6c88 Mon Sep 17 00:00:00 2001 From: Atharva262005 Date: Fri, 23 Jan 2026 21:13:08 +0530 Subject: [PATCH 1/7] Fix flaky DataflowRunnerTest by ignoring batch tests in streaming mode --- .../org/apache/beam/runners/dataflow/DataflowRunnerTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index ee5a7e1d26c3..f3d74055c1e7 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -2308,6 +2308,8 @@ public void testBatchGroupIntoBatchesOverrideBytes() { @Test public void testBatchGroupIntoBatchesWithShardedKeyOverrideCount() throws IOException { PipelineOptions options = buildPipelineOptions(); + // Ignore this test for streaming pipelines. + assumeFalse(options.as(StreamingOptions.class).isStreaming()); Pipeline p = Pipeline.create(options); verifyGroupIntoBatchesOverrideCount(p, true, true); } @@ -2315,6 +2317,8 @@ public void testBatchGroupIntoBatchesWithShardedKeyOverrideCount() throws IOExce @Test public void testBatchGroupIntoBatchesWithShardedKeyOverrideBytes() throws IOException { PipelineOptions options = buildPipelineOptions(); + // Ignore this test for streaming pipelines. + assumeFalse(options.as(StreamingOptions.class).isStreaming()); Pipeline p = Pipeline.create(options); verifyGroupIntoBatchesOverrideBytes(p, true, true); } From 4e934b8d8b2be42f34c9b59a017542807c68061d Mon Sep 17 00:00:00 2001 From: Atharva262005 Date: Fri, 23 Jan 2026 21:33:55 +0530 Subject: [PATCH 2/7] Update CHANGES.md for #37371 --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index ff931802addf..936602238c09 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -81,6 +81,7 @@ ## Bugfixes * Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Fixed flaky DataflowRunnerTest when running in streaming mode (Java) ([#37371](https://github.com/apache/beam/issues/37371)). ## Security Fixes From 711b2ac71ee4934fe7c2af4b4ccda6681d6a1c51 Mon Sep 17 00:00:00 2001 From: Atharva262005 Date: Fri, 23 Jan 2026 21:51:07 +0530 Subject: [PATCH 3/7] Fix assumption to use TestPipeline options --- .../runners/dataflow/DataflowRunnerTest.java | 986 ++++++++---------- 1 file changed, 463 insertions(+), 523 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index f3d74055c1e7..07df0cb34735 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -203,10 +203,12 @@ /** * Tests for the {@link DataflowRunner}. * - *

Implements {@link Serializable} because it is caught in closures. + *

+ * Implements {@link Serializable} because it is caught in closures. */ @RunWith(JUnit4.class) -// TODO(https://github.com/apache/beam/issues/21230): Remove when new version of errorprone is +// TODO(https://github.com/apache/beam/issues/21230): Remove when new version of +// errorprone is // released (2.11.0) @SuppressWarnings("unused") public class DataflowRunnerTest implements Serializable { @@ -220,10 +222,14 @@ public class DataflowRunnerTest implements Serializable { private static final String PROJECT_ID = "some-project"; private static final String REGION_ID = "some-region-1"; - @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder(); - @Rule public transient ExpectedException thrown = ExpectedException.none(); - @Rule public transient ExpectedLogs expectedLogs = ExpectedLogs.none(DataflowRunner.class); - @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule + public transient TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public transient ExpectedException thrown = ExpectedException.none(); + @Rule + public transient ExpectedLogs expectedLogs = ExpectedLogs.none(DataflowRunner.class); + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); private transient Dataflow.Projects.Locations.Jobs mockJobs; private transient GcsUtil mockGcsUtil; @@ -279,10 +285,8 @@ static Dataflow buildMockDataflow(Dataflow.Projects.Locations.Jobs mockJobs) thr Dataflow mockDataflowClient = mock(Dataflow.class); Dataflow.Projects mockProjects = mock(Dataflow.Projects.class); Dataflow.Projects.Locations mockLocations = mock(Dataflow.Projects.Locations.class); - Dataflow.Projects.Locations.Jobs.Create mockRequest = - mock(Dataflow.Projects.Locations.Jobs.Create.class); - Dataflow.Projects.Locations.Jobs.List mockList = - mock(Dataflow.Projects.Locations.Jobs.List.class); + Dataflow.Projects.Locations.Jobs.Create mockRequest = mock(Dataflow.Projects.Locations.Jobs.Create.class); + Dataflow.Projects.Locations.Jobs.List mockList = mock(Dataflow.Projects.Locations.Jobs.List.class); when(mockDataflowClient.projects()).thenReturn(mockProjects); when(mockDataflowClient.getBaseUrl()).thenReturn("dataflow.googleapis.com"); @@ -312,21 +316,19 @@ static GcsUtil buildMockGcsUtil() throws IOException { when(mockGcsUtil.create(any(GcsPath.class), any(GcsUtil.CreateOptions.class))) .then( - invocation -> - FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE)); + invocation -> FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE)); when(mockGcsUtil.create(any(GcsPath.class), any(GcsUtil.CreateOptions.class))) .then( - invocation -> - FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE)); + invocation -> FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE)); when(mockGcsUtil.expand(any(GcsPath.class))) .then(invocation -> ImmutableList.of((GcsPath) invocation.getArguments()[0])); @@ -371,7 +373,8 @@ private DataflowPipelineOptions buildPipelineOptions() throws IOException { options.setProject(PROJECT_ID); options.setTempLocation(VALID_TEMP_BUCKET); options.setRegion(REGION_ID); - // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath. + // Set FILES_PROPERTY to empty to prevent a default value calculated from + // classpath. options.setFilesToStage(new ArrayList<>()); options.setDataflowClient(buildMockDataflow(mockJobs)); options.setGcsUtil(mockGcsUtil); @@ -385,14 +388,13 @@ private DataflowPipelineOptions buildPipelineOptions() throws IOException { @Test public void testPathValidation() { - String[] args = - new String[] { - "--runner=DataflowRunner", - "--region=some-region-1", - "--tempLocation=/tmp/not/a/gs/path", - "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), - }; + String[] args = new String[] { + "--runner=DataflowRunner", + "--region=some-region-1", + "--tempLocation=/tmp/not/a/gs/path", + "--project=test-project", + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), + }; thrown.expect(IllegalArgumentException.class); thrown.expectMessage("DataflowRunner requires gcpTempLocation"); @@ -401,14 +403,13 @@ public void testPathValidation() { @Test public void testPathExistsValidation() { - String[] args = - new String[] { - "--runner=DataflowRunner", - "--region=some-region-1", - "--tempLocation=gs://does/not/exist", - "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), - }; + String[] args = new String[] { + "--runner=DataflowRunner", + "--region=some-region-1", + "--tempLocation=gs://does/not/exist", + "--project=test-project", + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), + }; thrown.expect(IllegalArgumentException.class); thrown.expectMessage("gcpTempLocation"); @@ -418,15 +419,14 @@ public void testPathExistsValidation() { @Test public void testPathValidatorOverride() { - String[] args = - new String[] { - "--runner=DataflowRunner", - "--region=some-region-1", - "--tempLocation=/tmp/testing", - "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), - "--pathValidatorClass=" + NoopPathValidator.class.getName(), - }; + String[] args = new String[] { + "--runner=DataflowRunner", + "--region=some-region-1", + "--tempLocation=/tmp/testing", + "--project=test-project", + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), + "--pathValidatorClass=" + NoopPathValidator.class.getName(), + }; // Should not crash, because gcpTempLocation should get set from tempLocation TestPipeline.fromOptions(PipelineOptionsFactory.fromArgs(args).create()); } @@ -454,14 +454,16 @@ public void testFromOptionsUserAgentFromPipelineInfo() throws Exception { } /** - * Invasive mock-based test for checking that the JSON generated for the pipeline options has not + * Invasive mock-based test for checking that the JSON generated for the + * pipeline options has not * had vital fields pruned. */ @Test public void testSettingOfSdkPipelineOptions() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); - // These options are important only for this test, and need not be global to the test class + // These options are important only for this test, and need not be global to the + // test class options.setAppName(DataflowRunnerTest.class.getSimpleName()); options.setJobName("some-job-name"); @@ -471,8 +473,7 @@ public void testSettingOfSdkPipelineOptions() throws IOException { ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); Mockito.verify(mockJobs).create(eq(PROJECT_ID), eq(REGION_ID), jobCaptor.capture()); - Map sdkPipelineOptions = - jobCaptor.getValue().getEnvironment().getSdkPipelineOptions(); + Map sdkPipelineOptions = jobCaptor.getValue().getEnvironment().getSdkPipelineOptions(); assertThat(sdkPipelineOptions, hasKey("options")); Map optionsMap = (Map) sdkPipelineOptions.get("options"); @@ -541,7 +542,8 @@ public JacksonIncompatible(String value) { /** A Jackson mixin used to add annotations to other classes. */ @JsonDeserialize(using = JacksonIncompatibleDeserializer.class) @JsonSerialize(using = JacksonIncompatibleSerializer.class) - public static final class JacksonIncompatibleMixin {} + public static final class JacksonIncompatibleMixin { + } /** A Jackson deserializer for {@link JacksonIncompatible}. */ public static class JacksonIncompatibleDeserializer @@ -580,8 +582,7 @@ public void testSettingOfPipelineOptionsWithCustomUserType() throws IOException ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); Mockito.verify(mockJobs).create(eq(PROJECT_ID), eq(REGION_ID), jobCaptor.capture()); - Map sdkPipelineOptions = - jobCaptor.getValue().getEnvironment().getSdkPipelineOptions(); + Map sdkPipelineOptions = jobCaptor.getValue().getEnvironment().getSdkPipelineOptions(); assertThat(sdkPipelineOptions, hasKey("options")); Map optionsMap = (Map) sdkPipelineOptions.get("options"); assertThat(optionsMap, hasEntry("jacksonIncompatible", "userCustomTypeTest")); @@ -589,8 +590,7 @@ public void testSettingOfPipelineOptionsWithCustomUserType() throws IOException @Test public void testZoneAndWorkerRegionMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = - PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); options.setZone("us-east1-b"); options.setWorkerRegion("us-east1"); assertThrows( @@ -599,8 +599,7 @@ public void testZoneAndWorkerRegionMutuallyExclusive() { @Test public void testZoneAndWorkerZoneMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = - PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); options.setZone("us-east1-b"); options.setWorkerZone("us-east1-c"); assertThrows( @@ -609,8 +608,7 @@ public void testZoneAndWorkerZoneMutuallyExclusive() { @Test public void testExperimentRegionAndWorkerRegionMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = - PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); ExperimentalOptions.addExperiment(dataflowOptions, "worker_region=us-west1"); options.setWorkerRegion("us-east1"); @@ -620,8 +618,7 @@ public void testExperimentRegionAndWorkerRegionMutuallyExclusive() { @Test public void testExperimentRegionAndWorkerZoneMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = - PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); ExperimentalOptions.addExperiment(dataflowOptions, "worker_region=us-west1"); options.setWorkerZone("us-east1-b"); @@ -631,8 +628,7 @@ public void testExperimentRegionAndWorkerZoneMutuallyExclusive() { @Test public void testWorkerRegionAndWorkerZoneMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = - PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); options.setWorkerRegion("us-east1"); options.setWorkerZone("us-east1-b"); assertThrows( @@ -641,8 +637,7 @@ public void testWorkerRegionAndWorkerZoneMutuallyExclusive() { @Test public void testZoneAliasWorkerZone() { - DataflowPipelineWorkerPoolOptions options = - PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); options.setZone("us-east1-b"); DataflowRunner.validateWorkerSettings(options); assertNull(options.getZone()); @@ -651,8 +646,7 @@ public void testZoneAliasWorkerZone() { @Test public void testAliasForLegacyWorkerHarnessContainerImage() { - DataflowPipelineWorkerPoolOptions options = - PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); String testImage = "image.url:worker"; options.setWorkerHarnessContainerImage(testImage); DataflowRunner.validateWorkerSettings(options); @@ -662,8 +656,7 @@ public void testAliasForLegacyWorkerHarnessContainerImage() { @Test public void testAliasForSdkContainerImage() { - DataflowPipelineWorkerPoolOptions options = - PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); String testImage = "image.url:sdk"; options.setSdkContainerImage("image.url:sdk"); DataflowRunner.validateWorkerSettings(options); @@ -742,13 +735,12 @@ public void visitPrimitiveTransform(Node node) { public void testRunReturnDifferentRequestId() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); Dataflow mockDataflowClient = options.getDataflowClient(); - Dataflow.Projects.Locations.Jobs.Create mockRequest = - mock(Dataflow.Projects.Locations.Jobs.Create.class); + Dataflow.Projects.Locations.Jobs.Create mockRequest = mock(Dataflow.Projects.Locations.Jobs.Create.class); when(mockDataflowClient - .projects() - .locations() - .jobs() - .create(eq(PROJECT_ID), eq(REGION_ID), any(Job.class))) + .projects() + .locations() + .jobs() + .create(eq(PROJECT_ID), eq(REGION_ID), any(Job.class))) .thenReturn(mockRequest); Job resultJob = new Job(); resultJob.setId("newid"); @@ -815,7 +807,10 @@ public void testUploadGraphV2IsNoOp() throws IOException { assertNull(jobCaptor.getValue().getStepsLocation()); } - /** Test for automatically using upload_graph when the job graph is too large (>10MB). */ + /** + * Test for automatically using upload_graph when the job graph is too large + * (>10MB). + */ @Test public void testUploadGraphWithAutoUpload() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); @@ -851,13 +846,12 @@ public void testUpdateAlreadyUpdatedPipeline() throws IOException { options.setUpdate(true); options.setJobName("oldJobName"); Dataflow mockDataflowClient = options.getDataflowClient(); - Dataflow.Projects.Locations.Jobs.Create mockRequest = - mock(Dataflow.Projects.Locations.Jobs.Create.class); + Dataflow.Projects.Locations.Jobs.Create mockRequest = mock(Dataflow.Projects.Locations.Jobs.Create.class); when(mockDataflowClient - .projects() - .locations() - .jobs() - .create(eq(PROJECT_ID), eq(REGION_ID), any(Job.class))) + .projects() + .locations() + .jobs() + .create(eq(PROJECT_ID), eq(REGION_ID), any(Job.class))) .thenReturn(mockRequest); final Job resultJob = new Job(); resultJob.setId("newid"); @@ -920,12 +914,11 @@ public void testRunWithFiles() throws IOException { when(mockGcsUtil.create(any(GcsPath.class), any(GcsUtil.CreateOptions.class))) .then( - invocation -> - FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE)); + invocation -> FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE)); Pipeline p = buildDataflowPipeline(options); @@ -938,11 +931,9 @@ public void testRunWithFiles() throws IOException { assertValidJob(workflowJob); assertEquals(2, workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size()); - DataflowPackage workflowPackage1 = - workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0); + DataflowPackage workflowPackage1 = workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0); assertThat(workflowPackage1.getName(), endsWith(getFileExtension(temp1.getAbsolutePath()))); - DataflowPackage workflowPackage2 = - workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1); + DataflowPackage workflowPackage2 = workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1); assertEquals(overridePackageName, workflowPackage2.getName()); assertEquals( @@ -958,7 +949,8 @@ public void testRunWithFiles() throws IOException { } /** - * Tests that {@link DataflowRunner} throws an appropriate exception when an explicitly specified + * Tests that {@link DataflowRunner} throws an appropriate exception when an + * explicitly specified * file to stage does not exist locally. */ @Test(expected = RuntimeException.class) @@ -988,12 +980,11 @@ public void testRunWithMissingFiles() throws IOException { when(mockGcsUtil.create(any(GcsPath.class), any(GcsUtil.CreateOptions.class))) .then( - invocation -> - FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE)); + invocation -> FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE)); Pipeline p = buildDataflowPipeline(options); p.run(); @@ -1152,7 +1143,8 @@ public void testNonExistentProfileLocation() throws IOException { public void testNoProjectFails() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); - // Explicitly set to null to prevent the default instance factory from reading credentials + // Explicitly set to null to prevent the default instance factory from reading + // credentials // from a user's environment, causing this test to fail. options.setProject(null); @@ -1290,35 +1282,32 @@ public void testStageArtifactWithoutStagedName() throws IOException { File temp2 = File.createTempFile("artifact2-", ".txt"); temp2.deleteOnExit(); - RunnerApi.ArtifactInformation fooLocalArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(temp1.getAbsolutePath()) - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation barLocalArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(temp2.getAbsolutePath()) - .build() - .toByteString()) - .build(); - RunnerApi.Pipeline pipeline = - RunnerApi.Pipeline.newBuilder() - .setComponents( - RunnerApi.Components.newBuilder() - .putEnvironments( - "env", - RunnerApi.Environment.newBuilder() - .addAllDependencies( - ImmutableList.of(fooLocalArtifact, barLocalArtifact)) - .build())) - .build(); + RunnerApi.ArtifactInformation fooLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(temp1.getAbsolutePath()) + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation barLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(temp2.getAbsolutePath()) + .build() + .toByteString()) + .build(); + RunnerApi.Pipeline pipeline = RunnerApi.Pipeline.newBuilder() + .setComponents( + RunnerApi.Components.newBuilder() + .putEnvironments( + "env", + RunnerApi.Environment.newBuilder() + .addAllDependencies( + ImmutableList.of(fooLocalArtifact, barLocalArtifact)) + .build())) + .build(); List packages = runner.stageArtifacts(pipeline); for (DataflowPackage pkg : packages) { assertThat(pkg.getName(), matchesRegex("artifact[1,2]-.+\\.txt")); @@ -1335,69 +1324,62 @@ public void testStageDuplicatedArtifacts() throws IOException { File bar = File.createTempFile("bar-", ".txt"); bar.deleteOnExit(); - RunnerApi.ArtifactInformation foo1LocalArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(foo.getAbsolutePath()) - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("foo_staged1.jar") - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation foo2LocalArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(foo.getAbsolutePath()) - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("foo_staged2.jar") - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation barLocalArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(bar.getAbsolutePath()) - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("bar_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.Environment env1 = - RunnerApi.Environment.newBuilder() - .addAllDependencies(ImmutableList.of(foo1LocalArtifact, barLocalArtifact)) - .build(); - RunnerApi.Environment env2 = - RunnerApi.Environment.newBuilder() - .addAllDependencies(ImmutableList.of(foo2LocalArtifact, barLocalArtifact)) - .build(); - RunnerApi.Pipeline pipeline = - RunnerApi.Pipeline.newBuilder() - .setComponents( - RunnerApi.Components.newBuilder() - .putEnvironments("env1", env1) - .putEnvironments("env2", env2)) - .build(); + RunnerApi.ArtifactInformation foo1LocalArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(foo.getAbsolutePath()) + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("foo_staged1.jar") + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation foo2LocalArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(foo.getAbsolutePath()) + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("foo_staged2.jar") + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation barLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(bar.getAbsolutePath()) + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("bar_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.Environment env1 = RunnerApi.Environment.newBuilder() + .addAllDependencies(ImmutableList.of(foo1LocalArtifact, barLocalArtifact)) + .build(); + RunnerApi.Environment env2 = RunnerApi.Environment.newBuilder() + .addAllDependencies(ImmutableList.of(foo2LocalArtifact, barLocalArtifact)) + .build(); + RunnerApi.Pipeline pipeline = RunnerApi.Pipeline.newBuilder() + .setComponents( + RunnerApi.Components.newBuilder() + .putEnvironments("env1", env1) + .putEnvironments("env2", env2)) + .build(); List packages = runner.stageArtifacts(pipeline); - List packageNames = - packages.stream().map(DataflowPackage::getName).collect(Collectors.toList()); + List packageNames = packages.stream().map(DataflowPackage::getName).collect(Collectors.toList()); assertThat(packageNames.size(), equalTo(3)); assertThat( packageNames, containsInAnyOrder("foo_staged1.jar", "foo_staged2.jar", "bar_staged.jar")); @@ -1408,89 +1390,83 @@ public void testResolveArtifacts() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); DataflowRunner runner = DataflowRunner.fromOptions(options); String stagingLocation = options.getStagingLocation().replaceFirst("/$", ""); - RunnerApi.ArtifactInformation fooLocalArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath("/tmp/foo.jar") - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("foo_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation barLocalArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath("/tmp/bar.jar") - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("bar_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.Pipeline pipeline = - RunnerApi.Pipeline.newBuilder() - .setComponents( - RunnerApi.Components.newBuilder() - .putEnvironments( - "env", - RunnerApi.Environment.newBuilder() - .addAllDependencies( - ImmutableList.of(fooLocalArtifact, barLocalArtifact)) - .build())) - .build(); - - RunnerApi.ArtifactInformation fooStagedArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL)) - .setTypePayload( - RunnerApi.ArtifactUrlPayload.newBuilder() - .setUrl(stagingLocation + "/foo_staged.jar") - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("foo_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation barStagedArtifact = - RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL)) - .setTypePayload( - RunnerApi.ArtifactUrlPayload.newBuilder() - .setUrl(stagingLocation + "/bar_staged.jar") - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("bar_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.Pipeline expectedPipeline = - RunnerApi.Pipeline.newBuilder() - .setComponents( - RunnerApi.Components.newBuilder() - .putEnvironments( - "env", - RunnerApi.Environment.newBuilder() - .addAllDependencies( - ImmutableList.of(fooStagedArtifact, barStagedArtifact)) - .build())) - .build(); + RunnerApi.ArtifactInformation fooLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath("/tmp/foo.jar") + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("foo_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation barLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath("/tmp/bar.jar") + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("bar_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.Pipeline pipeline = RunnerApi.Pipeline.newBuilder() + .setComponents( + RunnerApi.Components.newBuilder() + .putEnvironments( + "env", + RunnerApi.Environment.newBuilder() + .addAllDependencies( + ImmutableList.of(fooLocalArtifact, barLocalArtifact)) + .build())) + .build(); + + RunnerApi.ArtifactInformation fooStagedArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL)) + .setTypePayload( + RunnerApi.ArtifactUrlPayload.newBuilder() + .setUrl(stagingLocation + "/foo_staged.jar") + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("foo_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation barStagedArtifact = RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL)) + .setTypePayload( + RunnerApi.ArtifactUrlPayload.newBuilder() + .setUrl(stagingLocation + "/bar_staged.jar") + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("bar_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.Pipeline expectedPipeline = RunnerApi.Pipeline.newBuilder() + .setComponents( + RunnerApi.Components.newBuilder() + .putEnvironments( + "env", + RunnerApi.Environment.newBuilder() + .addAllDependencies( + ImmutableList.of(fooStagedArtifact, barStagedArtifact)) + .build())) + .build(); assertThat(runner.resolveArtifacts(pipeline), equalTo(expectedPipeline)); } @@ -1531,8 +1507,7 @@ public void testValidProfileLocation() throws IOException { @Test public void testInvalidJobName() throws IOException { List invalidNames = Arrays.asList("invalid_name", "0invalid", "invalid-"); - List expectedReason = - Arrays.asList("JobName invalid", "JobName invalid", "JobName invalid"); + List expectedReason = Arrays.asList("JobName invalid", "JobName invalid", "JobName invalid"); for (int i = 0; i < invalidNames.size(); ++i) { DataflowPipelineOptions options = buildPipelineOptions(); @@ -1549,8 +1524,7 @@ public void testInvalidJobName() throws IOException { @Test public void testValidJobName() throws IOException { - List names = - Arrays.asList("ok", "Ok", "A-Ok", "ok-123", "this-one-is-fairly-long-01234567890123456789"); + List names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123", "this-one-is-fairly-long-01234567890123456789"); for (String name : names) { DataflowPipelineOptions options = buildPipelineOptions(); @@ -1576,8 +1550,10 @@ public void testGcsUploadBufferSizeIsSetForStreamingWhenDefault() throws IOExcep streamingOptions.setRunner(DataflowRunner.class); Pipeline p = Pipeline.create(streamingOptions); - // Instantiation of a runner prior to run() currently has a side effect of mutating the options. - // This could be tested by DataflowRunner.fromOptions(streamingOptions) but would not ensure + // Instantiation of a runner prior to run() currently has a side effect of + // mutating the options. + // This could be tested by DataflowRunner.fromOptions(streamingOptions) but + // would not ensure // that the pipeline itself had the expected options set. p.run(); @@ -1634,13 +1610,12 @@ public RunnerApi.FunctionSpec translate( } @SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) + "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) @AutoService(TransformPayloadTranslatorRegistrar.class) public static class DataflowTransformTranslator implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> - getTransformPayloadTranslators() { + public Map, ? extends TransformPayloadTranslator> getTransformPayloadTranslators() { return ImmutableMap.of(TestTransform.class, new TestTransformTranslator()); } } @@ -1688,8 +1663,7 @@ public void testTransformTranslator() throws IOException { // Note: This is about the minimum needed to fake out a // translation. This obviously isn't a real translation. - TransformTranslator.StepTranslationContext stepContext = - context.addStep(transform1, "TestTranslate"); + TransformTranslator.StepTranslationContext stepContext = context.addStep(transform1, "TestTranslate"); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform1)); }); @@ -1711,46 +1685,43 @@ private void verifySdkHarnessConfiguration(DataflowPipelineOptions options) { String defaultSdkContainerImage = DataflowRunner.getContainerImageForJob(options); SdkComponents sdkComponents = SdkComponents.create(); - RunnerApi.Environment defaultEnvironmentForDataflow = - Environments.createDockerEnvironment(defaultSdkContainerImage); - RunnerApi.Environment.Builder envBuilder = - defaultEnvironmentForDataflow.toBuilder().addCapabilities("my_dummy_capability"); + RunnerApi.Environment defaultEnvironmentForDataflow = Environments + .createDockerEnvironment(defaultSdkContainerImage); + RunnerApi.Environment.Builder envBuilder = defaultEnvironmentForDataflow.toBuilder() + .addCapabilities("my_dummy_capability"); sdkComponents.registerEnvironment(envBuilder.build()); RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p, sdkComponents, true); - Job job = - DataflowPipelineTranslator.fromOptions(options) - .translate( - p, - pipelineProto, - sdkComponents, - DataflowRunner.fromOptions(options), - Collections.emptyList()) - .getJob(); + Job job = DataflowPipelineTranslator.fromOptions(options) + .translate( + p, + pipelineProto, + sdkComponents, + DataflowRunner.fromOptions(options), + Collections.emptyList()) + .getJob(); DataflowRunner.configureSdkHarnessContainerImages(options, pipelineProto, job); - List sdks = - job.getEnvironment().getWorkerPools().get(0).getSdkHarnessContainerImages(); - - Map expectedEnvIdsAndContainerImages = - pipelineProto.getComponents().getEnvironmentsMap().entrySet().stream() - .filter( - x -> - BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER) - .equals(x.getValue().getUrn())) - .collect( - Collectors.toMap( - x -> x.getKey(), - x -> { - RunnerApi.DockerPayload payload; - try { - payload = RunnerApi.DockerPayload.parseFrom(x.getValue().getPayload()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - return payload.getContainerImage(); - })); + List sdks = job.getEnvironment().getWorkerPools().get(0).getSdkHarnessContainerImages(); + + Map expectedEnvIdsAndContainerImages = pipelineProto.getComponents().getEnvironmentsMap().entrySet() + .stream() + .filter( + x -> BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER) + .equals(x.getValue().getUrn())) + .collect( + Collectors.toMap( + x -> x.getKey(), + x -> { + RunnerApi.DockerPayload payload; + try { + payload = RunnerApi.DockerPayload.parseFrom(x.getValue().getPayload()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + return payload.getContainerImage(); + })); assertEquals(1, expectedEnvIdsAndContainerImages.size()); assertEquals(1, sdks.size()); @@ -1780,9 +1751,8 @@ public void testSdkHarnessConfigurationPrime() throws IOException { @Test public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception { - for (String experiment : - ImmutableList.of( - "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { + for (String experiment : ImmutableList.of( + "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { DataflowPipelineOptions options = buildPipelineOptions(); ExperimentalOptions.addExperiment(options, experiment); Pipeline p = Pipeline.create(options); @@ -1795,9 +1765,8 @@ public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")); } - for (String experiment : - ImmutableList.of( - "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { + for (String experiment : ImmutableList.of( + "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { DataflowPipelineOptions options = buildPipelineOptions(); options.setStreaming(true); ExperimentalOptions.addExperiment(options, experiment); @@ -1819,12 +1788,10 @@ public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception @Test public void testSettingConflictingEnableAndDisableExperimentsThrowsException() throws Exception { - for (String experiment : - ImmutableList.of( - "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { - for (String disabledExperiment : - ImmutableList.of( - "disable_runner_v2", "disable_runner_v2_until_2023", "disable_prime_runner_v2")) { + for (String experiment : ImmutableList.of( + "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { + for (String disabledExperiment : ImmutableList.of( + "disable_runner_v2", "disable_runner_v2_until_2023", "disable_prime_runner_v2")) { DataflowPipelineOptions options = buildPipelineOptions(); ExperimentalOptions.addExperiment(options, experiment); ExperimentalOptions.addExperiment(options, disabledExperiment); @@ -1858,14 +1825,15 @@ public void testApplyIsScopedToExactClass() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); Pipeline p = Pipeline.create(options); - Create.TimestampedValues transform = - Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now()))); + Create.TimestampedValues transform = Create + .timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now()))); p.apply(transform); CompositeTransformRecorder recorder = new CompositeTransformRecorder(); p.traverseTopologically(recorder); - // The recorder will also have seen a Create.Values composite as well, but we can't obtain that + // The recorder will also have seen a Create.Values composite as well, but we + // can't obtain that // transform. assertThat( "Expected to have seen CreateTimestamped composite transform.", @@ -1891,7 +1859,8 @@ public void testToString() { } /** - * Tests that the {@link DataflowRunner} with {@code --templateLocation} returns normally when the + * Tests that the {@link DataflowRunner} with {@code --templateLocation} returns + * normally when the * runner is successfully run. */ @Test @@ -1913,8 +1882,10 @@ public void testTemplateRunnerFullCompletion() throws Exception { } /** - * Tests that the {@link DataflowRunner} with {@code --templateLocation} returns normally when the - * runner is successfully run with upload_graph experiment turned on. The result template should + * Tests that the {@link DataflowRunner} with {@code --templateLocation} returns + * normally when the + * runner is successfully run with upload_graph experiment turned on. The result + * template should * not contain raw steps and stepsLocation file should be set. */ @Test @@ -1941,7 +1912,8 @@ public void testTemplateRunnerWithUploadGraph() throws Exception { } /** - * Tests that the {@link DataflowRunner} with {@code --templateLocation} throws the appropriate + * Tests that the {@link DataflowRunner} with {@code --templateLocation} throws + * the appropriate * exception when an output file is not creatable. */ @Test @@ -1995,7 +1967,8 @@ public boolean isOpen() { } @Override - public void close() {} + public void close() { + } }; } @@ -2004,12 +1977,12 @@ public void testGetContainerImageForJobFromOption() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); String[] testCases = { - "some-container", + "some-container", - // It is important that empty string is preserved, as - // dataflowWorkerJar relies on being passed an empty value vs - // not providing the container image option at all. - "", + // It is important that empty string is preserved, as + // dataflowWorkerJar relies on being passed an empty value vs + // not providing the container image option at all. + "", }; for (String testCase : testCases) { @@ -2075,24 +2048,20 @@ public void testStreamingWriteWithShardingReturnsSameTransform() { TestPipeline p = TestPipeline.fromOptions(options); - StreamingShardedWriteFactory factory = - new StreamingShardedWriteFactory<>(p.getOptions()); - WriteFiles original = - WriteFiles.to(new TestSink(tmpFolder.toString())).withAutoSharding(); + StreamingShardedWriteFactory factory = new StreamingShardedWriteFactory<>(p.getOptions()); + WriteFiles original = WriteFiles.to(new TestSink(tmpFolder.toString())).withAutoSharding(); PCollection objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); - AppliedPTransform, WriteFilesResult, WriteFiles> - originalApplication = - AppliedPTransform.of( - "writefiles", - PValues.expandInput(objs), - Collections.emptyMap(), - original, - ResourceHints.create(), - p); - - WriteFiles replacement = - (WriteFiles) - factory.getReplacementTransform(originalApplication).getTransform(); + AppliedPTransform, WriteFilesResult, WriteFiles> originalApplication = AppliedPTransform + .of( + "writefiles", + PValues.expandInput(objs), + Collections.emptyMap(), + original, + ResourceHints.create(), + p); + + WriteFiles replacement = (WriteFiles) factory + .getReplacementTransform(originalApplication).getTransform(); WriteFilesResult originalResult = objs.apply(original); WriteFilesResult replacementResult = objs.apply(replacement); @@ -2115,7 +2084,8 @@ private void verifyMergingStatefulParDoRejected(PipelineOptions options) throws private final StateSpec> voidState = StateSpecs.value(); @ProcessElement - public void process() {} + public void process() { + } })); thrown.expectMessage("merging"); @@ -2140,28 +2110,25 @@ public void testMergingStatefulRejectedInBatch() throws Exception { private void verifyGroupIntoBatchesOverrideCount( Pipeline p, Boolean withShardedKey, Boolean expectOverridden) { final int batchSize = 2; - List> testValues = - Arrays.asList(KV.of("A", 1), KV.of("B", 0), KV.of("A", 2), KV.of("A", 4), KV.of("A", 8)); + List> testValues = Arrays.asList(KV.of("A", 1), KV.of("B", 0), KV.of("A", 2), KV.of("A", 4), + KV.of("A", 8)); PCollection> input = p.apply("CreateValuesCount", Create.of(testValues)); PCollection>> output; if (withShardedKey) { - output = - input - .apply( - "GroupIntoBatchesCount", - GroupIntoBatches.ofSize(batchSize).withShardedKey()) - .apply( - "StripShardIdCount", - MapElements.via( - new SimpleFunction< - KV, Iterable>, - KV>>() { - @Override - public KV> apply( - KV, Iterable> input) { - return KV.of(input.getKey().getKey(), input.getValue()); - } - })); + output = input + .apply( + "GroupIntoBatchesCount", + GroupIntoBatches.ofSize(batchSize).withShardedKey()) + .apply( + "StripShardIdCount", + MapElements.via( + new SimpleFunction, Iterable>, KV>>() { + @Override + public KV> apply( + KV, Iterable> input) { + return KV.of(input.getKey().getKey(), input.getValue()); + } + })); } else { output = input.apply("GroupIntoBatchesCount", GroupIntoBatches.ofSize(batchSize)); } @@ -2192,8 +2159,7 @@ public KV> apply( @Override public CompositeBehavior enterCompositeTransform(Node node) { if (p.getOptions().as(StreamingOptions.class).isStreaming() - && node.getTransform() - instanceof GroupIntoBatchesOverride.StreamingGroupIntoBatchesWithShardedKey) { + && node.getTransform() instanceof GroupIntoBatchesOverride.StreamingGroupIntoBatchesWithShardedKey) { sawGroupIntoBatchesOverride.set(true); } if (!p.getOptions().as(StreamingOptions.class).isStreaming() @@ -2201,8 +2167,7 @@ public CompositeBehavior enterCompositeTransform(Node node) { sawGroupIntoBatchesOverride.set(true); } if (!p.getOptions().as(StreamingOptions.class).isStreaming() - && node.getTransform() - instanceof GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKey) { + && node.getTransform() instanceof GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKey) { sawGroupIntoBatchesOverride.set(true); } return CompositeBehavior.ENTER_TRANSFORM; @@ -2218,33 +2183,29 @@ public CompositeBehavior enterCompositeTransform(Node node) { private void verifyGroupIntoBatchesOverrideBytes( Pipeline p, Boolean withShardedKey, Boolean expectOverridden) { final long batchSizeBytes = 2; - List> testValues = - Arrays.asList( - KV.of("A", "a"), - KV.of("A", "ab"), - KV.of("A", "abc"), - KV.of("A", "abcd"), - KV.of("A", "abcde")); + List> testValues = Arrays.asList( + KV.of("A", "a"), + KV.of("A", "ab"), + KV.of("A", "abc"), + KV.of("A", "abcd"), + KV.of("A", "abcde")); PCollection> input = p.apply("CreateValuesBytes", Create.of(testValues)); PCollection>> output; if (withShardedKey) { - output = - input - .apply( - "GroupIntoBatchesBytes", - GroupIntoBatches.ofByteSize(batchSizeBytes).withShardedKey()) - .apply( - "StripShardIdBytes", - MapElements.via( - new SimpleFunction< - KV, Iterable>, - KV>>() { - @Override - public KV> apply( - KV, Iterable> input) { - return KV.of(input.getKey().getKey(), input.getValue()); - } - })); + output = input + .apply( + "GroupIntoBatchesBytes", + GroupIntoBatches.ofByteSize(batchSizeBytes).withShardedKey()) + .apply( + "StripShardIdBytes", + MapElements.via( + new SimpleFunction, Iterable>, KV>>() { + @Override + public KV> apply( + KV, Iterable> input) { + return KV.of(input.getKey().getKey(), input.getValue()); + } + })); } else { output = input.apply("GroupIntoBatchesBytes", GroupIntoBatches.ofByteSize(batchSizeBytes)); } @@ -2266,8 +2227,7 @@ public KV> apply( @Override public CompositeBehavior enterCompositeTransform(Node node) { if (p.getOptions().as(StreamingOptions.class).isStreaming() - && node.getTransform() - instanceof GroupIntoBatchesOverride.StreamingGroupIntoBatchesWithShardedKey) { + && node.getTransform() instanceof GroupIntoBatchesOverride.StreamingGroupIntoBatchesWithShardedKey) { sawGroupIntoBatchesOverride.set(true); } if (!p.getOptions().as(StreamingOptions.class).isStreaming() @@ -2275,8 +2235,7 @@ public CompositeBehavior enterCompositeTransform(Node node) { sawGroupIntoBatchesOverride.set(true); } if (!p.getOptions().as(StreamingOptions.class).isStreaming() - && node.getTransform() - instanceof GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKey) { + && node.getTransform() instanceof GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKey) { sawGroupIntoBatchesOverride.set(true); } return CompositeBehavior.ENTER_TRANSFORM; @@ -2290,7 +2249,7 @@ public CompositeBehavior enterCompositeTransform(Node node) { } @Test - @Category({ValidatesRunner.class, UsesStatefulParDo.class}) + @Category({ ValidatesRunner.class, UsesStatefulParDo.class }) public void testBatchGroupIntoBatchesOverrideCount() { // Ignore this test for streaming pipelines. assumeFalse(pipeline.getOptions().as(StreamingOptions.class).isStreaming()); @@ -2298,7 +2257,7 @@ public void testBatchGroupIntoBatchesOverrideCount() { } @Test - @Category({ValidatesRunner.class, UsesStatefulParDo.class}) + @Category({ ValidatesRunner.class, UsesStatefulParDo.class }) public void testBatchGroupIntoBatchesOverrideBytes() { // Ignore this test for streaming pipelines. assumeFalse(pipeline.getOptions().as(StreamingOptions.class).isStreaming()); @@ -2309,7 +2268,7 @@ public void testBatchGroupIntoBatchesOverrideBytes() { public void testBatchGroupIntoBatchesWithShardedKeyOverrideCount() throws IOException { PipelineOptions options = buildPipelineOptions(); // Ignore this test for streaming pipelines. - assumeFalse(options.as(StreamingOptions.class).isStreaming()); + assumeFalse(pipeline.getOptions().as(StreamingOptions.class).isStreaming()); Pipeline p = Pipeline.create(options); verifyGroupIntoBatchesOverrideCount(p, true, true); } @@ -2318,7 +2277,7 @@ public void testBatchGroupIntoBatchesWithShardedKeyOverrideCount() throws IOExce public void testBatchGroupIntoBatchesWithShardedKeyOverrideBytes() throws IOException { PipelineOptions options = buildPipelineOptions(); // Ignore this test for streaming pipelines. - assumeFalse(options.as(StreamingOptions.class).isStreaming()); + assumeFalse(pipeline.getOptions().as(StreamingOptions.class).isStreaming()); Pipeline p = Pipeline.create(options); verifyGroupIntoBatchesOverrideBytes(p, true, true); } @@ -2342,10 +2301,9 @@ public void testStreamingGroupIntoBatchesOverrideBytes() throws IOException { @Test public void testStreamingGroupIntoBatchesWithShardedKeyOverrideCount() throws IOException { PipelineOptions options = buildPipelineOptions(); - List experiments = - new ArrayList<>( - ImmutableList.of( - GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); + List experiments = new ArrayList<>( + ImmutableList.of( + GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); dataflowOptions.setExperiments(experiments); dataflowOptions.setStreaming(true); @@ -2356,10 +2314,9 @@ public void testStreamingGroupIntoBatchesWithShardedKeyOverrideCount() throws IO @Test public void testStreamingGroupIntoBatchesWithShardedKeyOverrideBytes() throws IOException { PipelineOptions options = buildPipelineOptions(); - List experiments = - new ArrayList<>( - ImmutableList.of( - GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); + List experiments = new ArrayList<>( + ImmutableList.of( + GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); dataflowOptions.setExperiments(experiments); dataflowOptions.setStreaming(true); @@ -2370,21 +2327,18 @@ public void testStreamingGroupIntoBatchesWithShardedKeyOverrideBytes() throws IO @Test public void testPubsubSinkOverride() throws IOException { PipelineOptions options = buildPipelineOptions(); - List experiments = - new ArrayList<>( - ImmutableList.of( - GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); + List experiments = new ArrayList<>( + ImmutableList.of( + GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); dataflowOptions.setExperiments(experiments); dataflowOptions.setStreaming(true); Pipeline p = Pipeline.create(options); - List testValues = - Arrays.asList( - new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), Collections.emptyMap())); - PCollection input = - p.apply("CreateValuesBytes", Create.of(testValues)) - .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + List testValues = Arrays.asList( + new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), Collections.emptyMap())); + PCollection input = p.apply("CreateValuesBytes", Create.of(testValues)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); input.apply(PubsubIO.writeMessages().to("projects/project/topics/topic")); p.run(); @@ -2434,24 +2388,21 @@ public void testBigQueryDLQWarningStorageApiALONotConsumed() throws Exception { public void testBigQueryDLQWarning(BigQueryIO.Write.Method method, boolean processFailures) throws IOException { PipelineOptions options = buildPipelineOptions(); - List experiments = - new ArrayList<>(ImmutableList.of(GcpOptions.STREAMING_ENGINE_EXPERIMENT)); + List experiments = new ArrayList<>(ImmutableList.of(GcpOptions.STREAMING_ENGINE_EXPERIMENT)); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); dataflowOptions.setExperiments(experiments); dataflowOptions.setStreaming(true); Pipeline p = Pipeline.create(options); List testValues = Arrays.asList(new TableRow(), new TableRow()); - PCollection input = - p.apply("CreateValuesBytes", Create.of(testValues)) - .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); - - BigQueryIO.Write write = - BigQueryIO.writeTableRows() - .to("project:dataset.table") - .withSchema(new TableSchema()) - .withMethod(method) - .withoutValidation(); + PCollection input = p.apply("CreateValuesBytes", Create.of(testValues)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + + BigQueryIO.Write write = BigQueryIO.writeTableRows() + .to("project:dataset.table") + .withSchema(new TableSchema()) + .withMethod(method) + .withoutValidation(); if (method == BigQueryIO.Write.Method.STORAGE_WRITE_API) { write = write.withAutoSharding().withTriggeringFrequency(Duration.standardSeconds(1)); } @@ -2473,9 +2424,8 @@ public void testBigQueryDLQWarning(BigQueryIO.Write.Method method, boolean proce } p.run(); - final String expectedWarning = - "No transform processes the failed-inserts output from BigQuery sink: BQWrite!" - + " Not processing failed inserts means that those rows will be lost."; + final String expectedWarning = "No transform processes the failed-inserts output from BigQuery sink: BQWrite!" + + " Not processing failed inserts means that those rows will be lost."; if (processFailures) { expectedLogs.verifyNotLogged(expectedWarning); } else { @@ -2490,13 +2440,11 @@ public void testPubsubSinkDynamicOverride() throws IOException { dataflowOptions.setStreaming(true); Pipeline p = Pipeline.create(options); - List testValues = - Arrays.asList( - new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), Collections.emptyMap()) - .withTopic("")); - PCollection input = - p.apply("CreateValuesBytes", Create.of(testValues)) - .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + List testValues = Arrays.asList( + new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), Collections.emptyMap()) + .withTopic("")); + PCollection input = p.apply("CreateValuesBytes", Create.of(testValues)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); input.apply(PubsubIO.writeMessagesDynamic()); p.run(); @@ -2520,34 +2468,31 @@ public void testEnableAllowDuplicatesForRedistributeWithALO() throws IOException options.setDataflowServiceOptions(ImmutableList.of("streaming_mode_at_least_once")); Pipeline pipeline = Pipeline.create(options); - ImmutableList> arbitraryKVs = - ImmutableList.of( - KV.of("k1", 3), - KV.of("k5", Integer.MAX_VALUE), - KV.of("k5", Integer.MIN_VALUE), - KV.of("k2", 66), - KV.of("k1", 4), - KV.of("k2", -33), - KV.of("k3", 0)); - PCollection> input = - pipeline.apply( - Create.of(arbitraryKVs).withCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))); + ImmutableList> arbitraryKVs = ImmutableList.of( + KV.of("k1", 3), + KV.of("k5", Integer.MAX_VALUE), + KV.of("k5", Integer.MIN_VALUE), + KV.of("k2", 66), + KV.of("k1", 4), + KV.of("k2", -33), + KV.of("k3", 0)); + PCollection> input = pipeline.apply( + Create.of(arbitraryKVs).withCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))); // The allowDuplicates for Redistribute is false by default. PCollection> output = input.apply(Redistribute.byKey()); pipeline.run(); - // The DataflowRedistributeByKey transform translated from Redistribute should have + // The DataflowRedistributeByKey transform translated from Redistribute should + // have // allowDuplicates set to true. AtomicBoolean redistributeAllowDuplicates = new AtomicBoolean(false); pipeline.traverseTopologically( new PipelineVisitor.Defaults() { @Override public CompositeBehavior enterCompositeTransform(Node node) { - if (node.getTransform() - instanceof RedistributeByKeyOverrideFactory.DataflowRedistributeByKey) { - RedistributeByKeyOverrideFactory.DataflowRedistributeByKey redistribute = - (RedistributeByKeyOverrideFactory.DataflowRedistributeByKey) - node.getTransform(); + if (node.getTransform() instanceof RedistributeByKeyOverrideFactory.DataflowRedistributeByKey) { + RedistributeByKeyOverrideFactory.DataflowRedistributeByKey redistribute = (RedistributeByKeyOverrideFactory.DataflowRedistributeByKey) node + .getTransform(); redistributeAllowDuplicates.set(redistribute.getAllowDuplicates()); } return CompositeBehavior.ENTER_TRANSFORM; @@ -2567,24 +2512,22 @@ public ExpansionServiceClient getExpansionServiceClient( public ExpansionApi.ExpansionResponse expand(ExpansionApi.ExpansionRequest request) { Pipeline p = TestPipeline.create(); p.apply(Create.of(1, 2, 3)); - SdkComponents sdkComponents = - SdkComponents.create(p.getOptions()).withNewIdPrefix(request.getNamespace()); + SdkComponents sdkComponents = SdkComponents.create(p.getOptions()).withNewIdPrefix(request.getNamespace()); RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p, sdkComponents); String transformId = Iterables.getOnlyElement(pipelineProto.getRootTransformIdsList()); RunnerApi.Components components = pipelineProto.getComponents(); ImmutableList.Builder requirementsBuilder = ImmutableList.builder(); requirementsBuilder.addAll(pipelineProto.getRequirementsList()); requirementsBuilder.add("ExternalTranslationTest_Requirement_URN"); - response = - ExpansionApi.ExpansionResponse.newBuilder() - .setComponents(components) - .setTransform( - components - .getTransformsOrThrow(transformId) - .toBuilder() - .setUniqueName(transformId)) - .addAllRequirements(requirementsBuilder.build()) - .build(); + response = ExpansionApi.ExpansionResponse.newBuilder() + .setComponents(components) + .setTransform( + components + .getTransformsOrThrow(transformId) + .toBuilder() + .setUniqueName(transformId)) + .addAllRequirements(requirementsBuilder.build()) + .build(); return response; } @@ -2611,12 +2554,11 @@ public void close() { public void testIsMultiLanguage() throws IOException { PipelineOptions options = buildPipelineOptions(); Pipeline pipeline = Pipeline.create(options); - PCollection col = - pipeline - .apply(Create.of("1", "2", "3")) - .apply( - External.of( - "dummy_urn", new byte[] {}, "", new TestExpansionServiceClientFactory())); + PCollection col = pipeline + .apply(Create.of("1", "2", "3")) + .apply( + External.of( + "dummy_urn", new byte[] {}, "", new TestExpansionServiceClientFactory())); assertTrue(DataflowRunner.isMultiLanguagePipeline(pipeline)); } @@ -2624,30 +2566,27 @@ public void testIsMultiLanguage() throws IOException { private void testStreamingWriteOverride(PipelineOptions options, int expectedNumShards) { TestPipeline p = TestPipeline.fromOptions(options); - StreamingShardedWriteFactory factory = - new StreamingShardedWriteFactory<>(p.getOptions()); + StreamingShardedWriteFactory factory = new StreamingShardedWriteFactory<>(p.getOptions()); WriteFiles original = WriteFiles.to(new TestSink(tmpFolder.toString())); PCollection objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); - AppliedPTransform, WriteFilesResult, WriteFiles> - originalApplication = - AppliedPTransform.of( - "writefiles", - PValues.expandInput(objs), - Collections.emptyMap(), - original, - ResourceHints.create(), - p); - - WriteFiles replacement = - (WriteFiles) - factory.getReplacementTransform(originalApplication).getTransform(); + AppliedPTransform, WriteFilesResult, WriteFiles> originalApplication = AppliedPTransform + .of( + "writefiles", + PValues.expandInput(objs), + Collections.emptyMap(), + original, + ResourceHints.create(), + p); + + WriteFiles replacement = (WriteFiles) factory + .getReplacementTransform(originalApplication).getTransform(); assertThat(replacement, not(equalTo((Object) original))); assertThat(replacement.getNumShardsProvider().get(), equalTo(expectedNumShards)); WriteFilesResult originalResult = objs.apply(original); WriteFilesResult replacementResult = objs.apply(replacement); - Map, ReplacementOutput> res = - factory.mapOutputs(PValues.expandOutput(originalResult), replacementResult); + Map, ReplacementOutput> res = factory.mapOutputs(PValues.expandOutput(originalResult), + replacementResult); assertEquals(1, res.size()); assertEquals( originalResult.getPerDestinationOutputFilenames(), @@ -2657,7 +2596,8 @@ private void testStreamingWriteOverride(PipelineOptions options, int expectedNum private static class TestSink extends FileBasedSink { @Override - public void validate(PipelineOptions options) {} + public void validate(PipelineOptions options) { + } TestSink(String tmpFolder) { super( @@ -2716,15 +2656,15 @@ public void testBatchStateSupported() throws IOException { private final StateSpec> mapState = StateSpecs.map(); @StateId("multimap") - private final StateSpec> multimapState = - StateSpecs.multimap(); + private final StateSpec> multimapState = StateSpecs.multimap(); @StateId("ordered list") - private final StateSpec> orderedListState = - StateSpecs.orderedList(VoidCoder.of()); + private final StateSpec> orderedListState = StateSpecs + .orderedList(VoidCoder.of()); @ProcessElement - public void process() {} + public void process() { + } })); p.run(); } @@ -2752,15 +2692,15 @@ public void testStreamingStateSupported() throws IOException { private final StateSpec> mapState = StateSpecs.map(); @StateId("multimap") - private final StateSpec> multimapState = - StateSpecs.multimap(); + private final StateSpec> multimapState = StateSpecs.multimap(); @StateId("ordered list") - private final StateSpec> orderedListState = - StateSpecs.orderedList(VoidCoder.of()); + private final StateSpec> orderedListState = StateSpecs + .orderedList(VoidCoder.of()); @ProcessElement - public void process() {} + public void process() { + } })); p.run(); } From 5a52fc685f3d700b29cba407532a7b026f25923d Mon Sep 17 00:00:00 2001 From: Atharva262005 Date: Fri, 23 Jan 2026 22:07:16 +0530 Subject: [PATCH 4/7] Apply spotless formatting --- .../runners/dataflow/DataflowRunnerTest.java | 958 ++++++++++-------- 1 file changed, 513 insertions(+), 445 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 07df0cb34735..3b05c7c08d14 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -203,8 +203,7 @@ /** * Tests for the {@link DataflowRunner}. * - *

- * Implements {@link Serializable} because it is caught in closures. + *

Implements {@link Serializable} because it is caught in closures. */ @RunWith(JUnit4.class) // TODO(https://github.com/apache/beam/issues/21230): Remove when new version of @@ -222,14 +221,10 @@ public class DataflowRunnerTest implements Serializable { private static final String PROJECT_ID = "some-project"; private static final String REGION_ID = "some-region-1"; - @Rule - public transient TemporaryFolder tmpFolder = new TemporaryFolder(); - @Rule - public transient ExpectedException thrown = ExpectedException.none(); - @Rule - public transient ExpectedLogs expectedLogs = ExpectedLogs.none(DataflowRunner.class); - @Rule - public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Rule public transient ExpectedLogs expectedLogs = ExpectedLogs.none(DataflowRunner.class); + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); private transient Dataflow.Projects.Locations.Jobs mockJobs; private transient GcsUtil mockGcsUtil; @@ -285,8 +280,10 @@ static Dataflow buildMockDataflow(Dataflow.Projects.Locations.Jobs mockJobs) thr Dataflow mockDataflowClient = mock(Dataflow.class); Dataflow.Projects mockProjects = mock(Dataflow.Projects.class); Dataflow.Projects.Locations mockLocations = mock(Dataflow.Projects.Locations.class); - Dataflow.Projects.Locations.Jobs.Create mockRequest = mock(Dataflow.Projects.Locations.Jobs.Create.class); - Dataflow.Projects.Locations.Jobs.List mockList = mock(Dataflow.Projects.Locations.Jobs.List.class); + Dataflow.Projects.Locations.Jobs.Create mockRequest = + mock(Dataflow.Projects.Locations.Jobs.Create.class); + Dataflow.Projects.Locations.Jobs.List mockList = + mock(Dataflow.Projects.Locations.Jobs.List.class); when(mockDataflowClient.projects()).thenReturn(mockProjects); when(mockDataflowClient.getBaseUrl()).thenReturn("dataflow.googleapis.com"); @@ -316,19 +313,21 @@ static GcsUtil buildMockGcsUtil() throws IOException { when(mockGcsUtil.create(any(GcsPath.class), any(GcsUtil.CreateOptions.class))) .then( - invocation -> FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE)); + invocation -> + FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE)); when(mockGcsUtil.create(any(GcsPath.class), any(GcsUtil.CreateOptions.class))) .then( - invocation -> FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE)); + invocation -> + FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE)); when(mockGcsUtil.expand(any(GcsPath.class))) .then(invocation -> ImmutableList.of((GcsPath) invocation.getArguments()[0])); @@ -388,13 +387,14 @@ private DataflowPipelineOptions buildPipelineOptions() throws IOException { @Test public void testPathValidation() { - String[] args = new String[] { - "--runner=DataflowRunner", - "--region=some-region-1", - "--tempLocation=/tmp/not/a/gs/path", - "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), - }; + String[] args = + new String[] { + "--runner=DataflowRunner", + "--region=some-region-1", + "--tempLocation=/tmp/not/a/gs/path", + "--project=test-project", + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), + }; thrown.expect(IllegalArgumentException.class); thrown.expectMessage("DataflowRunner requires gcpTempLocation"); @@ -403,13 +403,14 @@ public void testPathValidation() { @Test public void testPathExistsValidation() { - String[] args = new String[] { - "--runner=DataflowRunner", - "--region=some-region-1", - "--tempLocation=gs://does/not/exist", - "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), - }; + String[] args = + new String[] { + "--runner=DataflowRunner", + "--region=some-region-1", + "--tempLocation=gs://does/not/exist", + "--project=test-project", + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), + }; thrown.expect(IllegalArgumentException.class); thrown.expectMessage("gcpTempLocation"); @@ -419,14 +420,15 @@ public void testPathExistsValidation() { @Test public void testPathValidatorOverride() { - String[] args = new String[] { - "--runner=DataflowRunner", - "--region=some-region-1", - "--tempLocation=/tmp/testing", - "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), - "--pathValidatorClass=" + NoopPathValidator.class.getName(), - }; + String[] args = + new String[] { + "--runner=DataflowRunner", + "--region=some-region-1", + "--tempLocation=/tmp/testing", + "--project=test-project", + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), + "--pathValidatorClass=" + NoopPathValidator.class.getName(), + }; // Should not crash, because gcpTempLocation should get set from tempLocation TestPipeline.fromOptions(PipelineOptionsFactory.fromArgs(args).create()); } @@ -454,8 +456,7 @@ public void testFromOptionsUserAgentFromPipelineInfo() throws Exception { } /** - * Invasive mock-based test for checking that the JSON generated for the - * pipeline options has not + * Invasive mock-based test for checking that the JSON generated for the pipeline options has not * had vital fields pruned. */ @Test @@ -473,7 +474,8 @@ public void testSettingOfSdkPipelineOptions() throws IOException { ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); Mockito.verify(mockJobs).create(eq(PROJECT_ID), eq(REGION_ID), jobCaptor.capture()); - Map sdkPipelineOptions = jobCaptor.getValue().getEnvironment().getSdkPipelineOptions(); + Map sdkPipelineOptions = + jobCaptor.getValue().getEnvironment().getSdkPipelineOptions(); assertThat(sdkPipelineOptions, hasKey("options")); Map optionsMap = (Map) sdkPipelineOptions.get("options"); @@ -542,8 +544,7 @@ public JacksonIncompatible(String value) { /** A Jackson mixin used to add annotations to other classes. */ @JsonDeserialize(using = JacksonIncompatibleDeserializer.class) @JsonSerialize(using = JacksonIncompatibleSerializer.class) - public static final class JacksonIncompatibleMixin { - } + public static final class JacksonIncompatibleMixin {} /** A Jackson deserializer for {@link JacksonIncompatible}. */ public static class JacksonIncompatibleDeserializer @@ -582,7 +583,8 @@ public void testSettingOfPipelineOptionsWithCustomUserType() throws IOException ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); Mockito.verify(mockJobs).create(eq(PROJECT_ID), eq(REGION_ID), jobCaptor.capture()); - Map sdkPipelineOptions = jobCaptor.getValue().getEnvironment().getSdkPipelineOptions(); + Map sdkPipelineOptions = + jobCaptor.getValue().getEnvironment().getSdkPipelineOptions(); assertThat(sdkPipelineOptions, hasKey("options")); Map optionsMap = (Map) sdkPipelineOptions.get("options"); assertThat(optionsMap, hasEntry("jacksonIncompatible", "userCustomTypeTest")); @@ -590,7 +592,8 @@ public void testSettingOfPipelineOptionsWithCustomUserType() throws IOException @Test public void testZoneAndWorkerRegionMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = + PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); options.setZone("us-east1-b"); options.setWorkerRegion("us-east1"); assertThrows( @@ -599,7 +602,8 @@ public void testZoneAndWorkerRegionMutuallyExclusive() { @Test public void testZoneAndWorkerZoneMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = + PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); options.setZone("us-east1-b"); options.setWorkerZone("us-east1-c"); assertThrows( @@ -608,7 +612,8 @@ public void testZoneAndWorkerZoneMutuallyExclusive() { @Test public void testExperimentRegionAndWorkerRegionMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = + PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); ExperimentalOptions.addExperiment(dataflowOptions, "worker_region=us-west1"); options.setWorkerRegion("us-east1"); @@ -618,7 +623,8 @@ public void testExperimentRegionAndWorkerRegionMutuallyExclusive() { @Test public void testExperimentRegionAndWorkerZoneMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = + PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); ExperimentalOptions.addExperiment(dataflowOptions, "worker_region=us-west1"); options.setWorkerZone("us-east1-b"); @@ -628,7 +634,8 @@ public void testExperimentRegionAndWorkerZoneMutuallyExclusive() { @Test public void testWorkerRegionAndWorkerZoneMutuallyExclusive() { - DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = + PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); options.setWorkerRegion("us-east1"); options.setWorkerZone("us-east1-b"); assertThrows( @@ -637,7 +644,8 @@ public void testWorkerRegionAndWorkerZoneMutuallyExclusive() { @Test public void testZoneAliasWorkerZone() { - DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = + PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); options.setZone("us-east1-b"); DataflowRunner.validateWorkerSettings(options); assertNull(options.getZone()); @@ -646,7 +654,8 @@ public void testZoneAliasWorkerZone() { @Test public void testAliasForLegacyWorkerHarnessContainerImage() { - DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = + PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); String testImage = "image.url:worker"; options.setWorkerHarnessContainerImage(testImage); DataflowRunner.validateWorkerSettings(options); @@ -656,7 +665,8 @@ public void testAliasForLegacyWorkerHarnessContainerImage() { @Test public void testAliasForSdkContainerImage() { - DataflowPipelineWorkerPoolOptions options = PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); + DataflowPipelineWorkerPoolOptions options = + PipelineOptionsFactory.as(DataflowPipelineWorkerPoolOptions.class); String testImage = "image.url:sdk"; options.setSdkContainerImage("image.url:sdk"); DataflowRunner.validateWorkerSettings(options); @@ -735,12 +745,13 @@ public void visitPrimitiveTransform(Node node) { public void testRunReturnDifferentRequestId() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); Dataflow mockDataflowClient = options.getDataflowClient(); - Dataflow.Projects.Locations.Jobs.Create mockRequest = mock(Dataflow.Projects.Locations.Jobs.Create.class); + Dataflow.Projects.Locations.Jobs.Create mockRequest = + mock(Dataflow.Projects.Locations.Jobs.Create.class); when(mockDataflowClient - .projects() - .locations() - .jobs() - .create(eq(PROJECT_ID), eq(REGION_ID), any(Job.class))) + .projects() + .locations() + .jobs() + .create(eq(PROJECT_ID), eq(REGION_ID), any(Job.class))) .thenReturn(mockRequest); Job resultJob = new Job(); resultJob.setId("newid"); @@ -807,10 +818,7 @@ public void testUploadGraphV2IsNoOp() throws IOException { assertNull(jobCaptor.getValue().getStepsLocation()); } - /** - * Test for automatically using upload_graph when the job graph is too large - * (>10MB). - */ + /** Test for automatically using upload_graph when the job graph is too large (>10MB). */ @Test public void testUploadGraphWithAutoUpload() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); @@ -846,12 +854,13 @@ public void testUpdateAlreadyUpdatedPipeline() throws IOException { options.setUpdate(true); options.setJobName("oldJobName"); Dataflow mockDataflowClient = options.getDataflowClient(); - Dataflow.Projects.Locations.Jobs.Create mockRequest = mock(Dataflow.Projects.Locations.Jobs.Create.class); + Dataflow.Projects.Locations.Jobs.Create mockRequest = + mock(Dataflow.Projects.Locations.Jobs.Create.class); when(mockDataflowClient - .projects() - .locations() - .jobs() - .create(eq(PROJECT_ID), eq(REGION_ID), any(Job.class))) + .projects() + .locations() + .jobs() + .create(eq(PROJECT_ID), eq(REGION_ID), any(Job.class))) .thenReturn(mockRequest); final Job resultJob = new Job(); resultJob.setId("newid"); @@ -914,11 +923,12 @@ public void testRunWithFiles() throws IOException { when(mockGcsUtil.create(any(GcsPath.class), any(GcsUtil.CreateOptions.class))) .then( - invocation -> FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE)); + invocation -> + FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE)); Pipeline p = buildDataflowPipeline(options); @@ -931,9 +941,11 @@ public void testRunWithFiles() throws IOException { assertValidJob(workflowJob); assertEquals(2, workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size()); - DataflowPackage workflowPackage1 = workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0); + DataflowPackage workflowPackage1 = + workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0); assertThat(workflowPackage1.getName(), endsWith(getFileExtension(temp1.getAbsolutePath()))); - DataflowPackage workflowPackage2 = workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1); + DataflowPackage workflowPackage2 = + workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1); assertEquals(overridePackageName, workflowPackage2.getName()); assertEquals( @@ -949,8 +961,7 @@ public void testRunWithFiles() throws IOException { } /** - * Tests that {@link DataflowRunner} throws an appropriate exception when an - * explicitly specified + * Tests that {@link DataflowRunner} throws an appropriate exception when an explicitly specified * file to stage does not exist locally. */ @Test(expected = RuntimeException.class) @@ -980,11 +991,12 @@ public void testRunWithMissingFiles() throws IOException { when(mockGcsUtil.create(any(GcsPath.class), any(GcsUtil.CreateOptions.class))) .then( - invocation -> FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE)); + invocation -> + FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE)); Pipeline p = buildDataflowPipeline(options); p.run(); @@ -1282,32 +1294,35 @@ public void testStageArtifactWithoutStagedName() throws IOException { File temp2 = File.createTempFile("artifact2-", ".txt"); temp2.deleteOnExit(); - RunnerApi.ArtifactInformation fooLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(temp1.getAbsolutePath()) - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation barLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(temp2.getAbsolutePath()) - .build() - .toByteString()) - .build(); - RunnerApi.Pipeline pipeline = RunnerApi.Pipeline.newBuilder() - .setComponents( - RunnerApi.Components.newBuilder() - .putEnvironments( - "env", - RunnerApi.Environment.newBuilder() - .addAllDependencies( - ImmutableList.of(fooLocalArtifact, barLocalArtifact)) - .build())) - .build(); + RunnerApi.ArtifactInformation fooLocalArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(temp1.getAbsolutePath()) + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation barLocalArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(temp2.getAbsolutePath()) + .build() + .toByteString()) + .build(); + RunnerApi.Pipeline pipeline = + RunnerApi.Pipeline.newBuilder() + .setComponents( + RunnerApi.Components.newBuilder() + .putEnvironments( + "env", + RunnerApi.Environment.newBuilder() + .addAllDependencies( + ImmutableList.of(fooLocalArtifact, barLocalArtifact)) + .build())) + .build(); List packages = runner.stageArtifacts(pipeline); for (DataflowPackage pkg : packages) { assertThat(pkg.getName(), matchesRegex("artifact[1,2]-.+\\.txt")); @@ -1324,62 +1339,69 @@ public void testStageDuplicatedArtifacts() throws IOException { File bar = File.createTempFile("bar-", ".txt"); bar.deleteOnExit(); - RunnerApi.ArtifactInformation foo1LocalArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(foo.getAbsolutePath()) - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("foo_staged1.jar") - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation foo2LocalArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(foo.getAbsolutePath()) - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("foo_staged2.jar") - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation barLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath(bar.getAbsolutePath()) - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("bar_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.Environment env1 = RunnerApi.Environment.newBuilder() - .addAllDependencies(ImmutableList.of(foo1LocalArtifact, barLocalArtifact)) - .build(); - RunnerApi.Environment env2 = RunnerApi.Environment.newBuilder() - .addAllDependencies(ImmutableList.of(foo2LocalArtifact, barLocalArtifact)) - .build(); - RunnerApi.Pipeline pipeline = RunnerApi.Pipeline.newBuilder() - .setComponents( - RunnerApi.Components.newBuilder() - .putEnvironments("env1", env1) - .putEnvironments("env2", env2)) - .build(); + RunnerApi.ArtifactInformation foo1LocalArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(foo.getAbsolutePath()) + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("foo_staged1.jar") + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation foo2LocalArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(foo.getAbsolutePath()) + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("foo_staged2.jar") + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation barLocalArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath(bar.getAbsolutePath()) + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("bar_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.Environment env1 = + RunnerApi.Environment.newBuilder() + .addAllDependencies(ImmutableList.of(foo1LocalArtifact, barLocalArtifact)) + .build(); + RunnerApi.Environment env2 = + RunnerApi.Environment.newBuilder() + .addAllDependencies(ImmutableList.of(foo2LocalArtifact, barLocalArtifact)) + .build(); + RunnerApi.Pipeline pipeline = + RunnerApi.Pipeline.newBuilder() + .setComponents( + RunnerApi.Components.newBuilder() + .putEnvironments("env1", env1) + .putEnvironments("env2", env2)) + .build(); List packages = runner.stageArtifacts(pipeline); - List packageNames = packages.stream().map(DataflowPackage::getName).collect(Collectors.toList()); + List packageNames = + packages.stream().map(DataflowPackage::getName).collect(Collectors.toList()); assertThat(packageNames.size(), equalTo(3)); assertThat( packageNames, containsInAnyOrder("foo_staged1.jar", "foo_staged2.jar", "bar_staged.jar")); @@ -1390,83 +1412,89 @@ public void testResolveArtifacts() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); DataflowRunner runner = DataflowRunner.fromOptions(options); String stagingLocation = options.getStagingLocation().replaceFirst("/$", ""); - RunnerApi.ArtifactInformation fooLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath("/tmp/foo.jar") - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("foo_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation barLocalArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) - .setTypePayload( - RunnerApi.ArtifactFilePayload.newBuilder() - .setPath("/tmp/bar.jar") - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("bar_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.Pipeline pipeline = RunnerApi.Pipeline.newBuilder() - .setComponents( - RunnerApi.Components.newBuilder() - .putEnvironments( - "env", - RunnerApi.Environment.newBuilder() - .addAllDependencies( - ImmutableList.of(fooLocalArtifact, barLocalArtifact)) - .build())) - .build(); - - RunnerApi.ArtifactInformation fooStagedArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL)) - .setTypePayload( - RunnerApi.ArtifactUrlPayload.newBuilder() - .setUrl(stagingLocation + "/foo_staged.jar") - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("foo_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.ArtifactInformation barStagedArtifact = RunnerApi.ArtifactInformation.newBuilder() - .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL)) - .setTypePayload( - RunnerApi.ArtifactUrlPayload.newBuilder() - .setUrl(stagingLocation + "/bar_staged.jar") - .build() - .toByteString()) - .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) - .setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName("bar_staged.jar") - .build() - .toByteString()) - .build(); - RunnerApi.Pipeline expectedPipeline = RunnerApi.Pipeline.newBuilder() - .setComponents( - RunnerApi.Components.newBuilder() - .putEnvironments( - "env", - RunnerApi.Environment.newBuilder() - .addAllDependencies( - ImmutableList.of(fooStagedArtifact, barStagedArtifact)) - .build())) - .build(); + RunnerApi.ArtifactInformation fooLocalArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath("/tmp/foo.jar") + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("foo_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation barLocalArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE)) + .setTypePayload( + RunnerApi.ArtifactFilePayload.newBuilder() + .setPath("/tmp/bar.jar") + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("bar_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.Pipeline pipeline = + RunnerApi.Pipeline.newBuilder() + .setComponents( + RunnerApi.Components.newBuilder() + .putEnvironments( + "env", + RunnerApi.Environment.newBuilder() + .addAllDependencies( + ImmutableList.of(fooLocalArtifact, barLocalArtifact)) + .build())) + .build(); + + RunnerApi.ArtifactInformation fooStagedArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL)) + .setTypePayload( + RunnerApi.ArtifactUrlPayload.newBuilder() + .setUrl(stagingLocation + "/foo_staged.jar") + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("foo_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.ArtifactInformation barStagedArtifact = + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL)) + .setTypePayload( + RunnerApi.ArtifactUrlPayload.newBuilder() + .setUrl(stagingLocation + "/bar_staged.jar") + .build() + .toByteString()) + .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)) + .setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName("bar_staged.jar") + .build() + .toByteString()) + .build(); + RunnerApi.Pipeline expectedPipeline = + RunnerApi.Pipeline.newBuilder() + .setComponents( + RunnerApi.Components.newBuilder() + .putEnvironments( + "env", + RunnerApi.Environment.newBuilder() + .addAllDependencies( + ImmutableList.of(fooStagedArtifact, barStagedArtifact)) + .build())) + .build(); assertThat(runner.resolveArtifacts(pipeline), equalTo(expectedPipeline)); } @@ -1507,7 +1535,8 @@ public void testValidProfileLocation() throws IOException { @Test public void testInvalidJobName() throws IOException { List invalidNames = Arrays.asList("invalid_name", "0invalid", "invalid-"); - List expectedReason = Arrays.asList("JobName invalid", "JobName invalid", "JobName invalid"); + List expectedReason = + Arrays.asList("JobName invalid", "JobName invalid", "JobName invalid"); for (int i = 0; i < invalidNames.size(); ++i) { DataflowPipelineOptions options = buildPipelineOptions(); @@ -1524,7 +1553,8 @@ public void testInvalidJobName() throws IOException { @Test public void testValidJobName() throws IOException { - List names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123", "this-one-is-fairly-long-01234567890123456789"); + List names = + Arrays.asList("ok", "Ok", "A-Ok", "ok-123", "this-one-is-fairly-long-01234567890123456789"); for (String name : names) { DataflowPipelineOptions options = buildPipelineOptions(); @@ -1610,12 +1640,13 @@ public RunnerApi.FunctionSpec translate( } @SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) + "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) @AutoService(TransformPayloadTranslatorRegistrar.class) public static class DataflowTransformTranslator implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> getTransformPayloadTranslators() { + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { return ImmutableMap.of(TestTransform.class, new TestTransformTranslator()); } } @@ -1663,7 +1694,8 @@ public void testTransformTranslator() throws IOException { // Note: This is about the minimum needed to fake out a // translation. This obviously isn't a real translation. - TransformTranslator.StepTranslationContext stepContext = context.addStep(transform1, "TestTranslate"); + TransformTranslator.StepTranslationContext stepContext = + context.addStep(transform1, "TestTranslate"); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform1)); }); @@ -1685,43 +1717,46 @@ private void verifySdkHarnessConfiguration(DataflowPipelineOptions options) { String defaultSdkContainerImage = DataflowRunner.getContainerImageForJob(options); SdkComponents sdkComponents = SdkComponents.create(); - RunnerApi.Environment defaultEnvironmentForDataflow = Environments - .createDockerEnvironment(defaultSdkContainerImage); - RunnerApi.Environment.Builder envBuilder = defaultEnvironmentForDataflow.toBuilder() - .addCapabilities("my_dummy_capability"); + RunnerApi.Environment defaultEnvironmentForDataflow = + Environments.createDockerEnvironment(defaultSdkContainerImage); + RunnerApi.Environment.Builder envBuilder = + defaultEnvironmentForDataflow.toBuilder().addCapabilities("my_dummy_capability"); sdkComponents.registerEnvironment(envBuilder.build()); RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p, sdkComponents, true); - Job job = DataflowPipelineTranslator.fromOptions(options) - .translate( - p, - pipelineProto, - sdkComponents, - DataflowRunner.fromOptions(options), - Collections.emptyList()) - .getJob(); + Job job = + DataflowPipelineTranslator.fromOptions(options) + .translate( + p, + pipelineProto, + sdkComponents, + DataflowRunner.fromOptions(options), + Collections.emptyList()) + .getJob(); DataflowRunner.configureSdkHarnessContainerImages(options, pipelineProto, job); - List sdks = job.getEnvironment().getWorkerPools().get(0).getSdkHarnessContainerImages(); - - Map expectedEnvIdsAndContainerImages = pipelineProto.getComponents().getEnvironmentsMap().entrySet() - .stream() - .filter( - x -> BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER) - .equals(x.getValue().getUrn())) - .collect( - Collectors.toMap( - x -> x.getKey(), - x -> { - RunnerApi.DockerPayload payload; - try { - payload = RunnerApi.DockerPayload.parseFrom(x.getValue().getPayload()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - return payload.getContainerImage(); - })); + List sdks = + job.getEnvironment().getWorkerPools().get(0).getSdkHarnessContainerImages(); + + Map expectedEnvIdsAndContainerImages = + pipelineProto.getComponents().getEnvironmentsMap().entrySet().stream() + .filter( + x -> + BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER) + .equals(x.getValue().getUrn())) + .collect( + Collectors.toMap( + x -> x.getKey(), + x -> { + RunnerApi.DockerPayload payload; + try { + payload = RunnerApi.DockerPayload.parseFrom(x.getValue().getPayload()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + return payload.getContainerImage(); + })); assertEquals(1, expectedEnvIdsAndContainerImages.size()); assertEquals(1, sdks.size()); @@ -1751,8 +1786,9 @@ public void testSdkHarnessConfigurationPrime() throws IOException { @Test public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception { - for (String experiment : ImmutableList.of( - "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { + for (String experiment : + ImmutableList.of( + "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { DataflowPipelineOptions options = buildPipelineOptions(); ExperimentalOptions.addExperiment(options, experiment); Pipeline p = Pipeline.create(options); @@ -1765,8 +1801,9 @@ public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")); } - for (String experiment : ImmutableList.of( - "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { + for (String experiment : + ImmutableList.of( + "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { DataflowPipelineOptions options = buildPipelineOptions(); options.setStreaming(true); ExperimentalOptions.addExperiment(options, experiment); @@ -1788,10 +1825,12 @@ public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception @Test public void testSettingConflictingEnableAndDisableExperimentsThrowsException() throws Exception { - for (String experiment : ImmutableList.of( - "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { - for (String disabledExperiment : ImmutableList.of( - "disable_runner_v2", "disable_runner_v2_until_2023", "disable_prime_runner_v2")) { + for (String experiment : + ImmutableList.of( + "beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) { + for (String disabledExperiment : + ImmutableList.of( + "disable_runner_v2", "disable_runner_v2_until_2023", "disable_prime_runner_v2")) { DataflowPipelineOptions options = buildPipelineOptions(); ExperimentalOptions.addExperiment(options, experiment); ExperimentalOptions.addExperiment(options, disabledExperiment); @@ -1825,8 +1864,8 @@ public void testApplyIsScopedToExactClass() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); Pipeline p = Pipeline.create(options); - Create.TimestampedValues transform = Create - .timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now()))); + Create.TimestampedValues transform = + Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now()))); p.apply(transform); CompositeTransformRecorder recorder = new CompositeTransformRecorder(); @@ -1859,8 +1898,7 @@ public void testToString() { } /** - * Tests that the {@link DataflowRunner} with {@code --templateLocation} returns - * normally when the + * Tests that the {@link DataflowRunner} with {@code --templateLocation} returns normally when the * runner is successfully run. */ @Test @@ -1882,10 +1920,8 @@ public void testTemplateRunnerFullCompletion() throws Exception { } /** - * Tests that the {@link DataflowRunner} with {@code --templateLocation} returns - * normally when the - * runner is successfully run with upload_graph experiment turned on. The result - * template should + * Tests that the {@link DataflowRunner} with {@code --templateLocation} returns normally when the + * runner is successfully run with upload_graph experiment turned on. The result template should * not contain raw steps and stepsLocation file should be set. */ @Test @@ -1912,8 +1948,7 @@ public void testTemplateRunnerWithUploadGraph() throws Exception { } /** - * Tests that the {@link DataflowRunner} with {@code --templateLocation} throws - * the appropriate + * Tests that the {@link DataflowRunner} with {@code --templateLocation} throws the appropriate * exception when an output file is not creatable. */ @Test @@ -1967,8 +2002,7 @@ public boolean isOpen() { } @Override - public void close() { - } + public void close() {} }; } @@ -1977,12 +2011,12 @@ public void testGetContainerImageForJobFromOption() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); String[] testCases = { - "some-container", + "some-container", - // It is important that empty string is preserved, as - // dataflowWorkerJar relies on being passed an empty value vs - // not providing the container image option at all. - "", + // It is important that empty string is preserved, as + // dataflowWorkerJar relies on being passed an empty value vs + // not providing the container image option at all. + "", }; for (String testCase : testCases) { @@ -2048,20 +2082,24 @@ public void testStreamingWriteWithShardingReturnsSameTransform() { TestPipeline p = TestPipeline.fromOptions(options); - StreamingShardedWriteFactory factory = new StreamingShardedWriteFactory<>(p.getOptions()); - WriteFiles original = WriteFiles.to(new TestSink(tmpFolder.toString())).withAutoSharding(); + StreamingShardedWriteFactory factory = + new StreamingShardedWriteFactory<>(p.getOptions()); + WriteFiles original = + WriteFiles.to(new TestSink(tmpFolder.toString())).withAutoSharding(); PCollection objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); - AppliedPTransform, WriteFilesResult, WriteFiles> originalApplication = AppliedPTransform - .of( - "writefiles", - PValues.expandInput(objs), - Collections.emptyMap(), - original, - ResourceHints.create(), - p); - - WriteFiles replacement = (WriteFiles) factory - .getReplacementTransform(originalApplication).getTransform(); + AppliedPTransform, WriteFilesResult, WriteFiles> + originalApplication = + AppliedPTransform.of( + "writefiles", + PValues.expandInput(objs), + Collections.emptyMap(), + original, + ResourceHints.create(), + p); + + WriteFiles replacement = + (WriteFiles) + factory.getReplacementTransform(originalApplication).getTransform(); WriteFilesResult originalResult = objs.apply(original); WriteFilesResult replacementResult = objs.apply(replacement); @@ -2084,8 +2122,7 @@ private void verifyMergingStatefulParDoRejected(PipelineOptions options) throws private final StateSpec> voidState = StateSpecs.value(); @ProcessElement - public void process() { - } + public void process() {} })); thrown.expectMessage("merging"); @@ -2110,25 +2147,28 @@ public void testMergingStatefulRejectedInBatch() throws Exception { private void verifyGroupIntoBatchesOverrideCount( Pipeline p, Boolean withShardedKey, Boolean expectOverridden) { final int batchSize = 2; - List> testValues = Arrays.asList(KV.of("A", 1), KV.of("B", 0), KV.of("A", 2), KV.of("A", 4), - KV.of("A", 8)); + List> testValues = + Arrays.asList(KV.of("A", 1), KV.of("B", 0), KV.of("A", 2), KV.of("A", 4), KV.of("A", 8)); PCollection> input = p.apply("CreateValuesCount", Create.of(testValues)); PCollection>> output; if (withShardedKey) { - output = input - .apply( - "GroupIntoBatchesCount", - GroupIntoBatches.ofSize(batchSize).withShardedKey()) - .apply( - "StripShardIdCount", - MapElements.via( - new SimpleFunction, Iterable>, KV>>() { - @Override - public KV> apply( - KV, Iterable> input) { - return KV.of(input.getKey().getKey(), input.getValue()); - } - })); + output = + input + .apply( + "GroupIntoBatchesCount", + GroupIntoBatches.ofSize(batchSize).withShardedKey()) + .apply( + "StripShardIdCount", + MapElements.via( + new SimpleFunction< + KV, Iterable>, + KV>>() { + @Override + public KV> apply( + KV, Iterable> input) { + return KV.of(input.getKey().getKey(), input.getValue()); + } + })); } else { output = input.apply("GroupIntoBatchesCount", GroupIntoBatches.ofSize(batchSize)); } @@ -2159,7 +2199,8 @@ public KV> apply( @Override public CompositeBehavior enterCompositeTransform(Node node) { if (p.getOptions().as(StreamingOptions.class).isStreaming() - && node.getTransform() instanceof GroupIntoBatchesOverride.StreamingGroupIntoBatchesWithShardedKey) { + && node.getTransform() + instanceof GroupIntoBatchesOverride.StreamingGroupIntoBatchesWithShardedKey) { sawGroupIntoBatchesOverride.set(true); } if (!p.getOptions().as(StreamingOptions.class).isStreaming() @@ -2167,7 +2208,8 @@ public CompositeBehavior enterCompositeTransform(Node node) { sawGroupIntoBatchesOverride.set(true); } if (!p.getOptions().as(StreamingOptions.class).isStreaming() - && node.getTransform() instanceof GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKey) { + && node.getTransform() + instanceof GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKey) { sawGroupIntoBatchesOverride.set(true); } return CompositeBehavior.ENTER_TRANSFORM; @@ -2183,29 +2225,33 @@ public CompositeBehavior enterCompositeTransform(Node node) { private void verifyGroupIntoBatchesOverrideBytes( Pipeline p, Boolean withShardedKey, Boolean expectOverridden) { final long batchSizeBytes = 2; - List> testValues = Arrays.asList( - KV.of("A", "a"), - KV.of("A", "ab"), - KV.of("A", "abc"), - KV.of("A", "abcd"), - KV.of("A", "abcde")); + List> testValues = + Arrays.asList( + KV.of("A", "a"), + KV.of("A", "ab"), + KV.of("A", "abc"), + KV.of("A", "abcd"), + KV.of("A", "abcde")); PCollection> input = p.apply("CreateValuesBytes", Create.of(testValues)); PCollection>> output; if (withShardedKey) { - output = input - .apply( - "GroupIntoBatchesBytes", - GroupIntoBatches.ofByteSize(batchSizeBytes).withShardedKey()) - .apply( - "StripShardIdBytes", - MapElements.via( - new SimpleFunction, Iterable>, KV>>() { - @Override - public KV> apply( - KV, Iterable> input) { - return KV.of(input.getKey().getKey(), input.getValue()); - } - })); + output = + input + .apply( + "GroupIntoBatchesBytes", + GroupIntoBatches.ofByteSize(batchSizeBytes).withShardedKey()) + .apply( + "StripShardIdBytes", + MapElements.via( + new SimpleFunction< + KV, Iterable>, + KV>>() { + @Override + public KV> apply( + KV, Iterable> input) { + return KV.of(input.getKey().getKey(), input.getValue()); + } + })); } else { output = input.apply("GroupIntoBatchesBytes", GroupIntoBatches.ofByteSize(batchSizeBytes)); } @@ -2227,7 +2273,8 @@ public KV> apply( @Override public CompositeBehavior enterCompositeTransform(Node node) { if (p.getOptions().as(StreamingOptions.class).isStreaming() - && node.getTransform() instanceof GroupIntoBatchesOverride.StreamingGroupIntoBatchesWithShardedKey) { + && node.getTransform() + instanceof GroupIntoBatchesOverride.StreamingGroupIntoBatchesWithShardedKey) { sawGroupIntoBatchesOverride.set(true); } if (!p.getOptions().as(StreamingOptions.class).isStreaming() @@ -2235,7 +2282,8 @@ public CompositeBehavior enterCompositeTransform(Node node) { sawGroupIntoBatchesOverride.set(true); } if (!p.getOptions().as(StreamingOptions.class).isStreaming() - && node.getTransform() instanceof GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKey) { + && node.getTransform() + instanceof GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKey) { sawGroupIntoBatchesOverride.set(true); } return CompositeBehavior.ENTER_TRANSFORM; @@ -2249,7 +2297,7 @@ public CompositeBehavior enterCompositeTransform(Node node) { } @Test - @Category({ ValidatesRunner.class, UsesStatefulParDo.class }) + @Category({ValidatesRunner.class, UsesStatefulParDo.class}) public void testBatchGroupIntoBatchesOverrideCount() { // Ignore this test for streaming pipelines. assumeFalse(pipeline.getOptions().as(StreamingOptions.class).isStreaming()); @@ -2257,7 +2305,7 @@ public void testBatchGroupIntoBatchesOverrideCount() { } @Test - @Category({ ValidatesRunner.class, UsesStatefulParDo.class }) + @Category({ValidatesRunner.class, UsesStatefulParDo.class}) public void testBatchGroupIntoBatchesOverrideBytes() { // Ignore this test for streaming pipelines. assumeFalse(pipeline.getOptions().as(StreamingOptions.class).isStreaming()); @@ -2301,9 +2349,10 @@ public void testStreamingGroupIntoBatchesOverrideBytes() throws IOException { @Test public void testStreamingGroupIntoBatchesWithShardedKeyOverrideCount() throws IOException { PipelineOptions options = buildPipelineOptions(); - List experiments = new ArrayList<>( - ImmutableList.of( - GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); + List experiments = + new ArrayList<>( + ImmutableList.of( + GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); dataflowOptions.setExperiments(experiments); dataflowOptions.setStreaming(true); @@ -2314,9 +2363,10 @@ public void testStreamingGroupIntoBatchesWithShardedKeyOverrideCount() throws IO @Test public void testStreamingGroupIntoBatchesWithShardedKeyOverrideBytes() throws IOException { PipelineOptions options = buildPipelineOptions(); - List experiments = new ArrayList<>( - ImmutableList.of( - GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); + List experiments = + new ArrayList<>( + ImmutableList.of( + GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); dataflowOptions.setExperiments(experiments); dataflowOptions.setStreaming(true); @@ -2327,18 +2377,21 @@ public void testStreamingGroupIntoBatchesWithShardedKeyOverrideBytes() throws IO @Test public void testPubsubSinkOverride() throws IOException { PipelineOptions options = buildPipelineOptions(); - List experiments = new ArrayList<>( - ImmutableList.of( - GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); + List experiments = + new ArrayList<>( + ImmutableList.of( + GcpOptions.STREAMING_ENGINE_EXPERIMENT, GcpOptions.WINDMILL_SERVICE_EXPERIMENT)); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); dataflowOptions.setExperiments(experiments); dataflowOptions.setStreaming(true); Pipeline p = Pipeline.create(options); - List testValues = Arrays.asList( - new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), Collections.emptyMap())); - PCollection input = p.apply("CreateValuesBytes", Create.of(testValues)) - .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + List testValues = + Arrays.asList( + new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), Collections.emptyMap())); + PCollection input = + p.apply("CreateValuesBytes", Create.of(testValues)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); input.apply(PubsubIO.writeMessages().to("projects/project/topics/topic")); p.run(); @@ -2388,21 +2441,24 @@ public void testBigQueryDLQWarningStorageApiALONotConsumed() throws Exception { public void testBigQueryDLQWarning(BigQueryIO.Write.Method method, boolean processFailures) throws IOException { PipelineOptions options = buildPipelineOptions(); - List experiments = new ArrayList<>(ImmutableList.of(GcpOptions.STREAMING_ENGINE_EXPERIMENT)); + List experiments = + new ArrayList<>(ImmutableList.of(GcpOptions.STREAMING_ENGINE_EXPERIMENT)); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); dataflowOptions.setExperiments(experiments); dataflowOptions.setStreaming(true); Pipeline p = Pipeline.create(options); List testValues = Arrays.asList(new TableRow(), new TableRow()); - PCollection input = p.apply("CreateValuesBytes", Create.of(testValues)) - .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); - - BigQueryIO.Write write = BigQueryIO.writeTableRows() - .to("project:dataset.table") - .withSchema(new TableSchema()) - .withMethod(method) - .withoutValidation(); + PCollection input = + p.apply("CreateValuesBytes", Create.of(testValues)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + + BigQueryIO.Write write = + BigQueryIO.writeTableRows() + .to("project:dataset.table") + .withSchema(new TableSchema()) + .withMethod(method) + .withoutValidation(); if (method == BigQueryIO.Write.Method.STORAGE_WRITE_API) { write = write.withAutoSharding().withTriggeringFrequency(Duration.standardSeconds(1)); } @@ -2424,8 +2480,9 @@ public void testBigQueryDLQWarning(BigQueryIO.Write.Method method, boolean proce } p.run(); - final String expectedWarning = "No transform processes the failed-inserts output from BigQuery sink: BQWrite!" - + " Not processing failed inserts means that those rows will be lost."; + final String expectedWarning = + "No transform processes the failed-inserts output from BigQuery sink: BQWrite!" + + " Not processing failed inserts means that those rows will be lost."; if (processFailures) { expectedLogs.verifyNotLogged(expectedWarning); } else { @@ -2440,11 +2497,13 @@ public void testPubsubSinkDynamicOverride() throws IOException { dataflowOptions.setStreaming(true); Pipeline p = Pipeline.create(options); - List testValues = Arrays.asList( - new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), Collections.emptyMap()) - .withTopic("")); - PCollection input = p.apply("CreateValuesBytes", Create.of(testValues)) - .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + List testValues = + Arrays.asList( + new PubsubMessage("foo".getBytes(StandardCharsets.UTF_8), Collections.emptyMap()) + .withTopic("")); + PCollection input = + p.apply("CreateValuesBytes", Create.of(testValues)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); input.apply(PubsubIO.writeMessagesDynamic()); p.run(); @@ -2468,16 +2527,18 @@ public void testEnableAllowDuplicatesForRedistributeWithALO() throws IOException options.setDataflowServiceOptions(ImmutableList.of("streaming_mode_at_least_once")); Pipeline pipeline = Pipeline.create(options); - ImmutableList> arbitraryKVs = ImmutableList.of( - KV.of("k1", 3), - KV.of("k5", Integer.MAX_VALUE), - KV.of("k5", Integer.MIN_VALUE), - KV.of("k2", 66), - KV.of("k1", 4), - KV.of("k2", -33), - KV.of("k3", 0)); - PCollection> input = pipeline.apply( - Create.of(arbitraryKVs).withCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))); + ImmutableList> arbitraryKVs = + ImmutableList.of( + KV.of("k1", 3), + KV.of("k5", Integer.MAX_VALUE), + KV.of("k5", Integer.MIN_VALUE), + KV.of("k2", 66), + KV.of("k1", 4), + KV.of("k2", -33), + KV.of("k3", 0)); + PCollection> input = + pipeline.apply( + Create.of(arbitraryKVs).withCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))); // The allowDuplicates for Redistribute is false by default. PCollection> output = input.apply(Redistribute.byKey()); pipeline.run(); @@ -2490,9 +2551,11 @@ public void testEnableAllowDuplicatesForRedistributeWithALO() throws IOException new PipelineVisitor.Defaults() { @Override public CompositeBehavior enterCompositeTransform(Node node) { - if (node.getTransform() instanceof RedistributeByKeyOverrideFactory.DataflowRedistributeByKey) { - RedistributeByKeyOverrideFactory.DataflowRedistributeByKey redistribute = (RedistributeByKeyOverrideFactory.DataflowRedistributeByKey) node - .getTransform(); + if (node.getTransform() + instanceof RedistributeByKeyOverrideFactory.DataflowRedistributeByKey) { + RedistributeByKeyOverrideFactory.DataflowRedistributeByKey redistribute = + (RedistributeByKeyOverrideFactory.DataflowRedistributeByKey) + node.getTransform(); redistributeAllowDuplicates.set(redistribute.getAllowDuplicates()); } return CompositeBehavior.ENTER_TRANSFORM; @@ -2512,22 +2575,24 @@ public ExpansionServiceClient getExpansionServiceClient( public ExpansionApi.ExpansionResponse expand(ExpansionApi.ExpansionRequest request) { Pipeline p = TestPipeline.create(); p.apply(Create.of(1, 2, 3)); - SdkComponents sdkComponents = SdkComponents.create(p.getOptions()).withNewIdPrefix(request.getNamespace()); + SdkComponents sdkComponents = + SdkComponents.create(p.getOptions()).withNewIdPrefix(request.getNamespace()); RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p, sdkComponents); String transformId = Iterables.getOnlyElement(pipelineProto.getRootTransformIdsList()); RunnerApi.Components components = pipelineProto.getComponents(); ImmutableList.Builder requirementsBuilder = ImmutableList.builder(); requirementsBuilder.addAll(pipelineProto.getRequirementsList()); requirementsBuilder.add("ExternalTranslationTest_Requirement_URN"); - response = ExpansionApi.ExpansionResponse.newBuilder() - .setComponents(components) - .setTransform( - components - .getTransformsOrThrow(transformId) - .toBuilder() - .setUniqueName(transformId)) - .addAllRequirements(requirementsBuilder.build()) - .build(); + response = + ExpansionApi.ExpansionResponse.newBuilder() + .setComponents(components) + .setTransform( + components + .getTransformsOrThrow(transformId) + .toBuilder() + .setUniqueName(transformId)) + .addAllRequirements(requirementsBuilder.build()) + .build(); return response; } @@ -2554,11 +2619,12 @@ public void close() { public void testIsMultiLanguage() throws IOException { PipelineOptions options = buildPipelineOptions(); Pipeline pipeline = Pipeline.create(options); - PCollection col = pipeline - .apply(Create.of("1", "2", "3")) - .apply( - External.of( - "dummy_urn", new byte[] {}, "", new TestExpansionServiceClientFactory())); + PCollection col = + pipeline + .apply(Create.of("1", "2", "3")) + .apply( + External.of( + "dummy_urn", new byte[] {}, "", new TestExpansionServiceClientFactory())); assertTrue(DataflowRunner.isMultiLanguagePipeline(pipeline)); } @@ -2566,27 +2632,30 @@ public void testIsMultiLanguage() throws IOException { private void testStreamingWriteOverride(PipelineOptions options, int expectedNumShards) { TestPipeline p = TestPipeline.fromOptions(options); - StreamingShardedWriteFactory factory = new StreamingShardedWriteFactory<>(p.getOptions()); + StreamingShardedWriteFactory factory = + new StreamingShardedWriteFactory<>(p.getOptions()); WriteFiles original = WriteFiles.to(new TestSink(tmpFolder.toString())); PCollection objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); - AppliedPTransform, WriteFilesResult, WriteFiles> originalApplication = AppliedPTransform - .of( - "writefiles", - PValues.expandInput(objs), - Collections.emptyMap(), - original, - ResourceHints.create(), - p); - - WriteFiles replacement = (WriteFiles) factory - .getReplacementTransform(originalApplication).getTransform(); + AppliedPTransform, WriteFilesResult, WriteFiles> + originalApplication = + AppliedPTransform.of( + "writefiles", + PValues.expandInput(objs), + Collections.emptyMap(), + original, + ResourceHints.create(), + p); + + WriteFiles replacement = + (WriteFiles) + factory.getReplacementTransform(originalApplication).getTransform(); assertThat(replacement, not(equalTo((Object) original))); assertThat(replacement.getNumShardsProvider().get(), equalTo(expectedNumShards)); WriteFilesResult originalResult = objs.apply(original); WriteFilesResult replacementResult = objs.apply(replacement); - Map, ReplacementOutput> res = factory.mapOutputs(PValues.expandOutput(originalResult), - replacementResult); + Map, ReplacementOutput> res = + factory.mapOutputs(PValues.expandOutput(originalResult), replacementResult); assertEquals(1, res.size()); assertEquals( originalResult.getPerDestinationOutputFilenames(), @@ -2596,8 +2665,7 @@ private void testStreamingWriteOverride(PipelineOptions options, int expectedNum private static class TestSink extends FileBasedSink { @Override - public void validate(PipelineOptions options) { - } + public void validate(PipelineOptions options) {} TestSink(String tmpFolder) { super( @@ -2656,15 +2724,15 @@ public void testBatchStateSupported() throws IOException { private final StateSpec> mapState = StateSpecs.map(); @StateId("multimap") - private final StateSpec> multimapState = StateSpecs.multimap(); + private final StateSpec> multimapState = + StateSpecs.multimap(); @StateId("ordered list") - private final StateSpec> orderedListState = StateSpecs - .orderedList(VoidCoder.of()); + private final StateSpec> orderedListState = + StateSpecs.orderedList(VoidCoder.of()); @ProcessElement - public void process() { - } + public void process() {} })); p.run(); } @@ -2692,15 +2760,15 @@ public void testStreamingStateSupported() throws IOException { private final StateSpec> mapState = StateSpecs.map(); @StateId("multimap") - private final StateSpec> multimapState = StateSpecs.multimap(); + private final StateSpec> multimapState = + StateSpecs.multimap(); @StateId("ordered list") - private final StateSpec> orderedListState = StateSpecs - .orderedList(VoidCoder.of()); + private final StateSpec> orderedListState = + StateSpecs.orderedList(VoidCoder.of()); @ProcessElement - public void process() { - } + public void process() {} })); p.run(); } From 1bb1271301ddc9274abfe87fe59d63fc98ae40dd Mon Sep 17 00:00:00 2001 From: Atharva262005 Date: Fri, 23 Jan 2026 22:31:41 +0530 Subject: [PATCH 5/7] Revert 'Fix assumption to use TestPipeline options' - Remove guards --- .../org/apache/beam/runners/dataflow/DataflowRunnerTest.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 3b05c7c08d14..bb2adfb08655 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -2315,8 +2315,6 @@ public void testBatchGroupIntoBatchesOverrideBytes() { @Test public void testBatchGroupIntoBatchesWithShardedKeyOverrideCount() throws IOException { PipelineOptions options = buildPipelineOptions(); - // Ignore this test for streaming pipelines. - assumeFalse(pipeline.getOptions().as(StreamingOptions.class).isStreaming()); Pipeline p = Pipeline.create(options); verifyGroupIntoBatchesOverrideCount(p, true, true); } @@ -2324,8 +2322,6 @@ public void testBatchGroupIntoBatchesWithShardedKeyOverrideCount() throws IOExce @Test public void testBatchGroupIntoBatchesWithShardedKeyOverrideBytes() throws IOException { PipelineOptions options = buildPipelineOptions(); - // Ignore this test for streaming pipelines. - assumeFalse(pipeline.getOptions().as(StreamingOptions.class).isStreaming()); Pipeline p = Pipeline.create(options); verifyGroupIntoBatchesOverrideBytes(p, true, true); } From d064e8ca69446f5416f3aabf6db6fc39181f7859 Mon Sep 17 00:00:00 2001 From: Atharva262005 Date: Fri, 23 Jan 2026 22:52:20 +0530 Subject: [PATCH 6/7] Fix GroupIntoBatches V2 Batch override and add regression test --- .../beam/runners/dataflow/DataflowRunner.java | 662 +++++++++--------- .../runners/dataflow/DataflowRunnerTest.java | 10 + 2 files changed, 360 insertions(+), 312 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 775e7b91de93..aa8bed9296c8 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -186,24 +186,37 @@ import org.slf4j.LoggerFactory; /** - * A {@link PipelineRunner} that executes the operations in the pipeline by first translating them - * to the Dataflow representation using the {@link DataflowPipelineTranslator} and then submitting + * A {@link PipelineRunner} that executes the operations in the pipeline by + * first translating them + * to the Dataflow representation using the {@link DataflowPipelineTranslator} + * and then submitting * them to a Dataflow service for execution. * *

Permissions

* - *

When reading from a Dataflow source or writing to a Dataflow sink using {@code - * DataflowRunner}, the Google cloudservices account and the Google compute engine service account - * of the GCP project running the Dataflow Job will need access to the corresponding source/sink. + *

+ * When reading from a Dataflow source or writing to a Dataflow sink using + * {@code + * DataflowRunner}, the Google cloudservices account and the Google compute + * engine service account + * of the GCP project running the Dataflow Job will need access to the + * corresponding source/sink. * - *

Please see Google Cloud + *

+ * Please see + * Google + * Cloud * Dataflow Security and Permissions for more details. * - *

DataflowRunner now supports creating job templates using the {@code --templateLocation} - * option. If this option is set, the runner will generate a template instead of running the + *

+ * DataflowRunner now supports creating job templates using the + * {@code --templateLocation} + * option. If this option is set, the runner will generate a template instead of + * running the * pipeline immediately. * - *

Example: + *

+ * Example: * *

{@code
  * --runner=DataflowRunner
@@ -211,14 +224,13 @@
  * }
*/ @SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public class DataflowRunner extends PipelineRunner { /** Experiment to "unsafely attempt to process unbounded data in batch mode". */ - public static final String UNSAFELY_ATTEMPT_TO_PROCESS_UNBOUNDED_DATA_IN_BATCH_MODE = - "unsafely_attempt_to_process_unbounded_data_in_batch_mode"; + public static final String UNSAFELY_ATTEMPT_TO_PROCESS_UNBOUNDED_DATA_IN_BATCH_MODE = "unsafely_attempt_to_process_unbounded_data_in_batch_mode"; private static final Logger LOG = LoggerFactory.getLogger(DataflowRunner.class); /** Provided configuration options. */ @@ -230,27 +242,33 @@ public class DataflowRunner extends PipelineRunner { /** Translator for this DataflowRunner, based on options. */ private final DataflowPipelineTranslator translator; - /** A set of user defined functions to invoke at different points in execution. */ + /** + * A set of user defined functions to invoke at different points in execution. + */ private DataflowRunnerHooks hooks; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; - @VisibleForTesting static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1024 * 1024; + @VisibleForTesting + static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1024 * 1024; - @VisibleForTesting static final String PIPELINE_FILE_NAME = "pipeline.pb"; - @VisibleForTesting static final String DATAFLOW_GRAPH_FILE_NAME = "dataflow_graph.json"; + @VisibleForTesting + static final String PIPELINE_FILE_NAME = "pipeline.pb"; + @VisibleForTesting + static final String DATAFLOW_GRAPH_FILE_NAME = "dataflow_graph.json"; private static final ObjectMapper MAPPER = new ObjectMapper(); /** - * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing for - * user specified configuration injection into the ObjectMapper. This supports user custom types + * Use an {@link ObjectMapper} configured with any {@link Module}s in the class + * path allowing for + * user specified configuration injection into the ObjectMapper. This supports + * user custom types * on {@link PipelineOptions}. */ - private static final ObjectMapper MAPPER_WITH_MODULES = - new ObjectMapper() - .registerModules(ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + private static final ObjectMapper MAPPER_WITH_MODULES = new ObjectMapper() + .registerModules(ObjectMapper.findModules(ReflectHelpers.findClassLoader())); private final Set> pcollectionsRequiringIndexedFormat; @@ -258,9 +276,12 @@ public class DataflowRunner extends PipelineRunner { private final Set> pcollectionsRequiringAutoSharding; /** - * Project IDs must contain lowercase letters, digits, or dashes. IDs must start with a letter and - * may not end with a dash. This regex isn't exact - this allows for patterns that would be - * rejected by the service, but this is sufficient for basic validation of project IDs. + * Project IDs must contain lowercase letters, digits, or dashes. IDs must start + * with a letter and + * may not end with a dash. This regex isn't exact - this allows for patterns + * that would be + * rejected by the service, but this is sufficient for basic validation of + * project IDs. */ public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]"; @@ -268,11 +289,15 @@ public class DataflowRunner extends PipelineRunner { static final String ENDPOINT_REGEXP = "https://[\\S]*googleapis\\.com[/]?"; /** - * Replaces GCS file paths with local file paths by downloading the GCS files locally. This is - * useful when files need to be accessed locally before being staged to Dataflow. + * Replaces GCS file paths with local file paths by downloading the GCS files + * locally. This is + * useful when files need to be accessed locally before being staged to + * Dataflow. * - * @param filesToStage List of file paths that may contain GCS paths (gs://) and local paths - * @return List of local file paths where any GCS paths have been downloaded locally + * @param filesToStage List of file paths that may contain GCS paths (gs://) and + * local paths + * @return List of local file paths where any GCS paths have been downloaded + * locally * @throws RuntimeException if there are errors copying GCS files locally */ public static List replaceGcsFilesWithLocalFiles(List filesToStage) { @@ -360,8 +385,7 @@ public static List replaceGcsFilesWithLocalFiles(List filesToSta * @return The newly created runner. */ public static DataflowRunner fromOptions(PipelineOptions options) { - DataflowPipelineOptions dataflowOptions = - PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options); + DataflowPipelineOptions dataflowOptions = PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options); ArrayList missing = new ArrayList<>(); if (dataflowOptions.getAppName() == null) { @@ -411,8 +435,10 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { // Replace GCS file paths with local file paths dataflowOptions.setFilesToStage( replaceGcsFilesWithLocalFiles(dataflowOptions.getFilesToStage())); - // The user specifically requested these files, so fail now if they do not exist. - // (automatically detected classpath elements are permitted to not exist, so later + // The user specifically requested these files, so fail now if they do not + // exist. + // (automatically detected classpath elements are permitted to not exist, so + // later // staging will not fail on nonexistent files) dataflowOptions.getFilesToStage().stream() .forEach( @@ -446,7 +472,8 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { } } - // Verify jobName according to service requirements, truncating converting to lowercase if + // Verify jobName according to service requirements, truncating converting to + // lowercase if // necessary. String jobName = dataflowOptions.getJobName().toLowerCase(); checkArgument( @@ -478,8 +505,7 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { + " description."); } - DataflowPipelineDebugOptions debugOptions = - dataflowOptions.as(DataflowPipelineDebugOptions.class); + DataflowPipelineDebugOptions debugOptions = dataflowOptions.as(DataflowPipelineDebugOptions.class); // Verify the number of worker threads is a valid value if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) { throw new IllegalArgumentException( @@ -493,8 +519,7 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { } // Adding the Java version to the SDK name for user's and support convenience. - String agentJavaVer = - String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); + String agentJavaVer = String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); String userAgentName = dataflowRunnerInfo.getName(); @@ -503,8 +528,7 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { String userAgentVersion = dataflowRunnerInfo.getVersion(); Preconditions.checkArgument( !userAgentVersion.equals(""), "Dataflow runner's `version` property cannot be empty."); - String userAgent = - String.format("%s/%s%s", userAgentName, userAgentVersion, agentJavaVer).replace(" ", "_"); + String userAgent = String.format("%s/%s%s", userAgentName, userAgentVersion, agentJavaVer).replace(" ", "_"); dataflowOptions.setUserAgent(userAgent); return new DataflowRunner(dataflowOptions); @@ -595,9 +619,8 @@ private static class AlwaysCreateViaRead implements PTransformOverrideFactory, Create.Values> { @Override - public PTransformOverrideFactory.PTransformReplacement> - getReplacementTransform( - AppliedPTransform, Create.Values> appliedTransform) { + public PTransformOverrideFactory.PTransformReplacement> getReplacementTransform( + AppliedPTransform, Create.Values> appliedTransform) { return PTransformOverrideFactory.PTransformReplacement.of( appliedTransform.getPipeline().begin(), appliedTransform.getTransform().alwaysUseRead()); } @@ -612,7 +635,8 @@ public final Map, ReplacementOutput> mapOutputs( private List getOverrides(boolean streaming) { ImmutableList.Builder overridesBuilder = ImmutableList.builder(); - // Create is implemented in terms of a Read, so it must precede the override to Read in + // Create is implemented in terms of a Read, so it must precede the override to + // Read in // streaming overridesBuilder .add( @@ -630,7 +654,8 @@ private List getOverrides(boolean streaming) { PTransformMatchers.classEqualTo(Create.Values.class), new AlwaysCreateViaRead())); } - // By default Dataflow runner replaces single-output ParDo with a ParDoSingle override. + // By default Dataflow runner replaces single-output ParDo with a ParDoSingle + // override. // However, we want a different expansion for single-output splittable ParDo. overridesBuilder .add( @@ -661,7 +686,8 @@ private List getOverrides(boolean streaming) { overridesBuilder.add(KafkaIO.Read.KAFKA_READ_OVERRIDE); overridesBuilder.add(KafkaIO.Read.KAFKA_REDISTRIBUTE_OVERRIDE); } catch (NoClassDefFoundError e) { - // Do nothing. io-kafka is an optional dependency of runners-google-cloud-dataflow-java + // Do nothing. io-kafka is an optional dependency of + // runners-google-cloud-dataflow-java // and only needed when KafkaIO is used in the pipeline. } @@ -687,7 +713,8 @@ private List getOverrides(boolean streaming) { overridesBuilder .add( - // Streaming Bounded Read is implemented in terms of Streaming Unbounded Read, and + // Streaming Bounded Read is implemented in terms of Streaming Unbounded Read, + // and // must precede it PTransformOverride.of( PTransformMatchers.classEqualTo(Read.Bounded.class), @@ -729,8 +756,10 @@ private List getOverrides(boolean streaming) { PTransformOverride.of( PTransformMatchers.stateOrTimerParDoSingle(), BatchStatefulParDoOverrides.singleOutputOverrideFactory())); - // Dataflow Batch runner uses the naive override of the SPLITTABLE_PROCESS_KEYED transform - // for now, but eventually (when liquid sharding is implemented) will also override it + // Dataflow Batch runner uses the naive override of the SPLITTABLE_PROCESS_KEYED + // transform + // for now, but eventually (when liquid sharding is implemented) will also + // override it // natively in the Dataflow service. overridesBuilder.add( PTransformOverride.of( @@ -762,20 +791,24 @@ private List getOverrides(boolean streaming) { new ReflectiveViewOverrideFactory( BatchViewOverrides.BatchViewAsIterable.class, this))); } - /* TODO(Beam-4684): Support @RequiresStableInput on Dataflow in a more intelligent way - Use Reshuffle might cause an extra and unnecessary shuffle to be inserted. To enable this, we - should make sure that we do not add extra shuffles for transforms whose input is already stable. - // Uses Reshuffle, so has to be before the Reshuffle override - overridesBuilder.add( - PTransformOverride.of( - PTransformMatchers.requiresStableInputParDoSingle(), - RequiresStableInputParDoOverrides.singleOutputOverrideFactory())); - // Uses Reshuffle, so has to be before the Reshuffle override - overridesBuilder.add( - PTransformOverride.of( - PTransformMatchers.requiresStableInputParDoMulti(), - RequiresStableInputParDoOverrides.multiOutputOverrideFactory())); - */ + /* + * TODO(Beam-4684): Support @RequiresStableInput on Dataflow in a more + * intelligent way + * Use Reshuffle might cause an extra and unnecessary shuffle to be inserted. To + * enable this, we + * should make sure that we do not add extra shuffles for transforms whose input + * is already stable. + * // Uses Reshuffle, so has to be before the Reshuffle override + * overridesBuilder.add( + * PTransformOverride.of( + * PTransformMatchers.requiresStableInputParDoSingle(), + * RequiresStableInputParDoOverrides.singleOutputOverrideFactory())); + * // Uses Reshuffle, so has to be before the Reshuffle override + * overridesBuilder.add( + * PTransformOverride.of( + * PTransformMatchers.requiresStableInputParDoMulti(), + * RequiresStableInputParDoOverrides.multiOutputOverrideFactory())); + */ overridesBuilder .add( PTransformOverride.of( @@ -790,9 +823,8 @@ private List getOverrides(boolean streaming) { PTransformMatchers.classEqualTo(ParDo.SingleOutput.class), new PrimitiveParDoSingleFactory())); - boolean usesAtLeastOnceStreamingMode = - options.getDataflowServiceOptions() != null - && options.getDataflowServiceOptions().contains("streaming_mode_at_least_once"); + boolean usesAtLeastOnceStreamingMode = options.getDataflowServiceOptions() != null + && options.getDataflowServiceOptions().contains("streaming_mode_at_least_once"); overridesBuilder.add( PTransformOverride.of( PTransformMatchers.classEqualTo(RedistributeByKey.class), @@ -824,8 +856,10 @@ private List getOverrides(boolean streaming) { } /** - * Replace the {@link Combine.GloballyAsSingletonView} transform with a specialization which - * re-applies the {@link CombineFn} and adds a specialization specific to the Dataflow runner. + * Replace the {@link Combine.GloballyAsSingletonView} transform with a + * specialization which + * re-applies the {@link CombineFn} and adds a specialization specific to the + * Dataflow runner. */ private static class CombineGloballyAsSingletonViewOverrideFactory extends ReflectiveViewOverrideFactory { @@ -835,15 +869,10 @@ private CombineGloballyAsSingletonViewOverrideFactory(DataflowRunner runner) { } @Override - public PTransformReplacement, PCollectionView> - getReplacementTransform( - AppliedPTransform< - PCollection, - PCollectionView, - PTransform, PCollectionView>> - transform) { - Combine.GloballyAsSingletonView combineTransform = - (Combine.GloballyAsSingletonView) transform.getTransform(); + public PTransformReplacement, PCollectionView> getReplacementTransform( + AppliedPTransform, PCollectionView, PTransform, PCollectionView>> transform) { + Combine.GloballyAsSingletonView combineTransform = (Combine.GloballyAsSingletonView) transform + .getTransform(); return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), new BatchViewOverrides.BatchViewAsSingleton( @@ -855,15 +884,15 @@ private CombineGloballyAsSingletonViewOverrideFactory(DataflowRunner runner) { } /** - * Replace the View.AsYYY transform with specialized view overrides for Dataflow. It is required - * that the new replacement transform uses the supplied PCollectionView and does not create + * Replace the View.AsYYY transform with specialized view overrides for + * Dataflow. It is required + * that the new replacement transform uses the supplied PCollectionView and does + * not create * another instance. */ private static class ReflectiveViewOverrideFactory - implements PTransformOverrideFactory< - PCollection, - PCollectionView, - PTransform, PCollectionView>> { + implements + PTransformOverrideFactory, PCollectionView, PTransform, PCollectionView>> { final Class, PCollectionView>> replacement; final DataflowRunner runner; @@ -876,11 +905,7 @@ private ReflectiveViewOverrideFactory( } CreatePCollectionView findCreatePCollectionView( - final AppliedPTransform< - PCollection, - PCollectionView, - PTransform, PCollectionView>> - transform) { + final AppliedPTransform, PCollectionView, PTransform, PCollectionView>> transform) { final AtomicReference viewTransformRef = new AtomicReference<>(); transform .getPipeline() @@ -927,19 +952,13 @@ public void leaveCompositeTransform(Node node) { } @Override - public PTransformReplacement, PCollectionView> - getReplacementTransform( - final AppliedPTransform< - PCollection, - PCollectionView, - PTransform, PCollectionView>> - transform) { - - PTransform, PCollectionView> rep = - InstanceBuilder.ofType(replacement) - .withArg(DataflowRunner.class, runner) - .withArg(CreatePCollectionView.class, findCreatePCollectionView(transform)) - .build(); + public PTransformReplacement, PCollectionView> getReplacementTransform( + final AppliedPTransform, PCollectionView, PTransform, PCollectionView>> transform) { + + PTransform, PCollectionView> rep = InstanceBuilder.ofType(replacement) + .withArg(DataflowRunner.class, runner) + .withArg(CreatePCollectionView.class, findCreatePCollectionView(transform)) + .build(); return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), (PTransform) rep); } @@ -948,19 +967,20 @@ public void leaveCompositeTransform(Node node) { public Map, ReplacementOutput> mapOutputs( Map, PCollection> outputs, PCollectionView newOutput) { /* - The output of View.AsXYZ is a PCollectionView that expands to the PCollection to be materialized. - The PCollectionView itself must have the same tag since that tag may have been embedded in serialized DoFns - previously and cannot easily be rewired. The PCollection may differ, so we rewire it, even if the rewiring - is a noop. - */ + * The output of View.AsXYZ is a PCollectionView that expands to the PCollection + * to be materialized. + * The PCollectionView itself must have the same tag since that tag may have + * been embedded in serialized DoFns + * previously and cannot easily be rewired. The PCollection may differ, so we + * rewire it, even if the rewiring + * is a noop. + */ return ReplacementOutputs.singleton(outputs, newOutput); } } - private static class ReflectiveOneToOneOverrideFactory< - InputT, OutputT, TransformT extends PTransform, PCollection>> - extends SingleInputOutputOverrideFactory< - PCollection, PCollection, TransformT> { + private static class ReflectiveOneToOneOverrideFactory, PCollection>> + extends SingleInputOutputOverrideFactory, PCollection, TransformT> { private final Class, PCollection>> replacement; private final DataflowRunner runner; @@ -975,12 +995,11 @@ private ReflectiveOneToOneOverrideFactory( @Override public PTransformReplacement, PCollection> getReplacementTransform( AppliedPTransform, PCollection, TransformT> transform) { - PTransform, PCollection> rep = - InstanceBuilder.ofType(replacement) - .withArg(DataflowRunner.class, runner) - .withArg( - (Class) transform.getTransform().getClass(), transform.getTransform()) - .build(); + PTransform, PCollection> rep = InstanceBuilder.ofType(replacement) + .withArg(DataflowRunner.class, runner) + .withArg( + (Class) transform.getTransform().getClass(), transform.getTransform()) + .build(); return PTransformReplacement.of(PTransformReplacements.getSingletonMainInput(transform), rep); } } @@ -989,8 +1008,7 @@ private RunnerApi.Pipeline resolveAnyOfEnvironments(RunnerApi.Pipeline pipeline) RunnerApi.Pipeline.Builder pipelineBuilder = pipeline.toBuilder(); RunnerApi.Components.Builder componentsBuilder = pipelineBuilder.getComponentsBuilder(); componentsBuilder.clearEnvironments(); - for (Map.Entry entry : - pipeline.getComponents().getEnvironmentsMap().entrySet()) { + for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { componentsBuilder.putEnvironments( entry.getKey(), Environments.resolveAnyOfEnvironment( @@ -1003,10 +1021,9 @@ private RunnerApi.Pipeline resolveAnyOfEnvironments(RunnerApi.Pipeline pipeline) protected RunnerApi.Pipeline applySdkEnvironmentOverrides( RunnerApi.Pipeline pipeline, DataflowPipelineOptions options) { String sdkHarnessContainerImageOverrides = options.getSdkHarnessContainerImageOverrides(); - String[] overrides = - Strings.isNullOrEmpty(sdkHarnessContainerImageOverrides) - ? new String[0] - : sdkHarnessContainerImageOverrides.split(",", -1); + String[] overrides = Strings.isNullOrEmpty(sdkHarnessContainerImageOverrides) + ? new String[0] + : sdkHarnessContainerImageOverrides.split(",", -1); if (overrides.length % 2 != 0) { throw new RuntimeException( "invalid syntax for SdkHarnessContainerImageOverrides: " @@ -1015,8 +1032,7 @@ protected RunnerApi.Pipeline applySdkEnvironmentOverrides( RunnerApi.Pipeline.Builder pipelineBuilder = pipeline.toBuilder(); RunnerApi.Components.Builder componentsBuilder = pipelineBuilder.getComponentsBuilder(); componentsBuilder.clearEnvironments(); - for (Map.Entry entry : - pipeline.getComponents().getEnvironmentsMap().entrySet()) { + for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { RunnerApi.Environment.Builder environmentBuilder = entry.getValue().toBuilder(); if (BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER) .equals(environmentBuilder.getUrn())) { @@ -1038,12 +1054,10 @@ protected RunnerApi.Pipeline applySdkEnvironmentOverrides( && !updated // don't update if the container image is already configured by DataflowRunner && !containerImage.equals(getContainerImageForJob(options))) { - String imageAndTag = - normalizeDataflowImageAndTag( - containerImage.substring(containerImage.lastIndexOf("/"))); - containerImage = - DataflowRunnerInfo.getDataflowRunnerInfo().getContainerImageBaseRepository() - + imageAndTag; + String imageAndTag = normalizeDataflowImageAndTag( + containerImage.substring(containerImage.lastIndexOf("/"))); + containerImage = DataflowRunnerInfo.getDataflowRunnerInfo().getContainerImageBaseRepository() + + imageAndTag; } environmentBuilder.setPayload( RunnerApi.DockerPayload.newBuilder() @@ -1062,7 +1076,8 @@ static String normalizeDataflowImageAndTag(String imageAndTag) { || imageAndTag.startsWith("/beam_go_")) { int tagIdx = imageAndTag.lastIndexOf(":"); if (tagIdx > 0) { - // For release candidates, apache/beam_ images has rc tag while Dataflow does not + // For release candidates, apache/beam_ images has rc tag while Dataflow does + // not String tag = imageAndTag.substring(tagIdx); // e,g, ":2.xx.0rc1" int mayRc = tag.toLowerCase().lastIndexOf("rc"); if (mayRc > 0) { @@ -1078,8 +1093,7 @@ protected RunnerApi.Pipeline resolveArtifacts(RunnerApi.Pipeline pipeline) { RunnerApi.Pipeline.Builder pipelineBuilder = pipeline.toBuilder(); RunnerApi.Components.Builder componentsBuilder = pipelineBuilder.getComponentsBuilder(); componentsBuilder.clearEnvironments(); - for (Map.Entry entry : - pipeline.getComponents().getEnvironmentsMap().entrySet()) { + for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { RunnerApi.Environment.Builder environmentBuilder = entry.getValue().toBuilder(); environmentBuilder.clearDependencies(); for (RunnerApi.ArtifactInformation info : entry.getValue().getDependenciesList()) { @@ -1097,8 +1111,8 @@ protected RunnerApi.Pipeline resolveArtifacts(RunnerApi.Pipeline pipeline) { if (BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO) .equals(info.getRoleUrn())) { try { - RunnerApi.ArtifactStagingToRolePayload stagingPayload = - RunnerApi.ArtifactStagingToRolePayload.parseFrom(info.getRolePayload()); + RunnerApi.ArtifactStagingToRolePayload stagingPayload = RunnerApi.ArtifactStagingToRolePayload + .parseFrom(info.getRolePayload()); stagedName = stagingPayload.getStagedName(); } catch (InvalidProtocolBufferException e) { throw new RuntimeException("Error parsing artifact staging_to role payload.", e); @@ -1136,8 +1150,7 @@ protected RunnerApi.Pipeline resolveArtifacts(RunnerApi.Pipeline pipeline) { protected List stageArtifacts(RunnerApi.Pipeline pipeline) { ImmutableList.Builder filesToStageBuilder = ImmutableList.builder(); Set stagedNames = new HashSet<>(); - for (Map.Entry entry : - pipeline.getComponents().getEnvironmentsMap().entrySet()) { + for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { for (RunnerApi.ArtifactInformation info : entry.getValue().getDependenciesList()) { if (!BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE).equals(info.getTypeUrn())) { throw new RuntimeException( @@ -1153,8 +1166,8 @@ protected List stageArtifacts(RunnerApi.Pipeline pipeline) { if (BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO) .equals(info.getRoleUrn())) { try { - RunnerApi.ArtifactStagingToRolePayload stagingPayload = - RunnerApi.ArtifactStagingToRolePayload.parseFrom(info.getRolePayload()); + RunnerApi.ArtifactStagingToRolePayload stagingPayload = RunnerApi.ArtifactStagingToRolePayload + .parseFrom(info.getRolePayload()); stagedName = stagingPayload.getStagedName(); } catch (InvalidProtocolBufferException e) { throw new RuntimeException("Error parsing artifact staging_to role payload.", e); @@ -1184,11 +1197,11 @@ protected List stageArtifacts(RunnerApi.Pipeline pipeline) { private List getDefaultArtifacts() { ImmutableList.Builder pathsToStageBuilder = ImmutableList.builder(); - String windmillBinary = - options.as(DataflowStreamingPipelineOptions.class).getOverrideWindmillBinary(); + String windmillBinary = options.as(DataflowStreamingPipelineOptions.class).getOverrideWindmillBinary(); String dataflowWorkerJar = options.getDataflowWorkerJar(); if (dataflowWorkerJar != null && !dataflowWorkerJar.isEmpty() && !useUnifiedWorker(options)) { - // Put the user specified worker jar at the start of the classpath, to be consistent with the + // Put the user specified worker jar at the start of the classpath, to be + // consistent with the // built-in worker order. pathsToStageBuilder.add("dataflow-worker.jar=" + dataflowWorkerJar); } @@ -1232,18 +1245,18 @@ public void visitPrimitiveTransform(Node node) { private static boolean includesTransformUpgrades(Pipeline pipeline) { return (pipeline - .getOptions() - .as(ExternalTranslationOptions.class) - .getTransformsToOverride() - .size() - > 0); + .getOptions() + .as(ExternalTranslationOptions.class) + .getTransformsToOverride() + .size() > 0); } private static final Random RANDOM = new Random(); @Override public DataflowPipelineJob run(Pipeline pipeline) { - // Multi-language pipelines and pipelines that include upgrades should automatically be upgraded + // Multi-language pipelines and pipelines that include upgrades should + // automatically be upgraded // to Runner v2. if (DataflowRunner.isMultiLanguagePipeline(pipeline) || includesTransformUpgrades(pipeline)) { List experiments = firstNonNull(options.getExperiments(), Collections.emptyList()); @@ -1262,8 +1275,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { throw new IllegalArgumentException( "Runner V2 both disabled and enabled: at least one of ['beam_fn_api', 'use_unified_worker', 'use_runner_v2', 'use_portable_job_submission'] is set and also one of ['disable_runner_v2', 'disable_runner_v2_until_2023', 'disable_prime_runner_v2'] is set."); } - List experiments = - new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true + List experiments = new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true if (!experiments.contains("use_runner_v2")) { experiments.add("use_runner_v2"); } @@ -1304,8 +1316,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { if (useUnifiedWorker(options)) { options.setEnableStreamingEngine(true); - List experiments = - new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true + List experiments = new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true if (!experiments.contains("enable_streaming_engine")) { experiments.add("enable_streaming_engine"); } @@ -1326,15 +1337,18 @@ public DataflowPipelineJob run(Pipeline pipeline) { DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); String workerHarnessContainerImageURL = DataflowRunner.getContainerImageForJob(dataflowOptions); - // This incorrectly puns the worker harness container image (which implements v1beta3 API) + // This incorrectly puns the worker harness container image (which implements + // v1beta3 API) // with the SDK harness image (which implements Fn API). // - // The same Environment is used in different and contradictory ways, depending on whether + // The same Environment is used in different and contradictory ways, depending + // on whether // it is a v1 or v2 job submission. - RunnerApi.Environment defaultEnvironmentForDataflow = - Environments.createDockerEnvironment(workerHarnessContainerImageURL); + RunnerApi.Environment defaultEnvironmentForDataflow = Environments + .createDockerEnvironment(workerHarnessContainerImageURL); - // The SdkComponents for portable an non-portable job submission must be kept distinct. Both + // The SdkComponents for portable an non-portable job submission must be kept + // distinct. Both // need the default environment. SdkComponents portableComponents = SdkComponents.create(); portableComponents.registerEnvironment( @@ -1344,8 +1358,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { .addAllCapabilities(Environments.getJavaCapabilities()) .build()); - RunnerApi.Pipeline portablePipelineProto = - PipelineTranslation.toProto(pipeline, portableComponents, false); + RunnerApi.Pipeline portablePipelineProto = PipelineTranslation.toProto(pipeline, portableComponents, false); // Note that `stageArtifacts` has to be called before `resolveArtifact` because // `resolveArtifact` updates local paths to staged paths in pipeline proto. portablePipelineProto = resolveAnyOfEnvironments(portablePipelineProto); @@ -1357,18 +1370,33 @@ public DataflowPipelineJob run(Pipeline pipeline) { "Portable pipeline proto:\n{}", TextFormat.printer().printToString(portablePipelineProto)); } - // Stage the portable pipeline proto, retrieving the staged pipeline path, then update + // Stage the portable pipeline proto, retrieving the staged pipeline path, then + // update // the options on the new job - // TODO: add an explicit `pipeline` parameter to the submission instead of pipeline options + // TODO: add an explicit `pipeline` parameter to the submission instead of + // pipeline options LOG.info("Staging portable pipeline proto to {}", options.getStagingLocation()); byte[] serializedProtoPipeline = portablePipelineProto.toByteArray(); - DataflowPackage stagedPipeline = - options.getStager().stageToFile(serializedProtoPipeline, PIPELINE_FILE_NAME); + DataflowPackage stagedPipeline = options.getStager().stageToFile(serializedProtoPipeline, PIPELINE_FILE_NAME); dataflowOptions.setPipelineUrl(stagedPipeline.getLocation()); if (useUnifiedWorker(options)) { LOG.info("Skipping v1 transform replacements since job will run on v2."); + if (!options.isStreaming()) { + LOG.info("Applying GroupIntoBatches overrides for V2 Batch mode."); + List overrides = ImmutableList.builder() + .add( + PTransformOverride.of( + PTransformMatchers.classEqualTo(GroupIntoBatches.class), + new GroupIntoBatchesOverride.BatchGroupIntoBatchesOverrideFactory<>(this))) + .add( + PTransformOverride.of( + PTransformMatchers.classEqualTo(GroupIntoBatches.WithShardedKey.class), + new GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKeyOverrideFactory<>(this))) + .build(); + pipeline.replaceAll(overrides); + } } else { // Now rewrite things to be as needed for v1 (mutates the pipeline) // This way the job submitted is valid for v1 and v2, simultaneously @@ -1383,8 +1411,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { .addAllCapabilities(Environments.getJavaCapabilities()) .build()); // No need to perform transform upgrading for the Runner v1 proto. - RunnerApi.Pipeline dataflowV1PipelineProto = - PipelineTranslation.toProto(pipeline, dataflowV1Components, true, false); + RunnerApi.Pipeline dataflowV1PipelineProto = PipelineTranslation.toProto(pipeline, dataflowV1Components, true, + false); if (LOG.isDebugEnabled()) { LOG.debug( @@ -1400,20 +1428,17 @@ public DataflowPipelineJob run(Pipeline pipeline) { // has been effectively rejected. The SDK should return // Error::Already_Exists to user in that case. int randomNum = RANDOM.nextInt(9000) + 1000; - String requestId = - DateTimeFormat.forPattern("YYYYMMddHHmmssmmm") - .withZone(DateTimeZone.UTC) - .print(DateTimeUtils.currentTimeMillis()) - + "_" - + randomNum; + String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm") + .withZone(DateTimeZone.UTC) + .print(DateTimeUtils.currentTimeMillis()) + + "_" + + randomNum; - JobSpecification jobSpecification = - translator.translate( - pipeline, dataflowV1PipelineProto, dataflowV1Components, this, packages); + JobSpecification jobSpecification = translator.translate( + pipeline, dataflowV1PipelineProto, dataflowV1Components, this, packages); if (!isNullOrEmpty(dataflowOptions.getDataflowWorkerJar()) && !useUnifiedWorker(options)) { - List experiments = - firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); + List experiments = firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); if (!experiments.contains("use_staged_dataflow_worker_jar")) { dataflowOptions.setExperiments( ImmutableList.builder() @@ -1460,23 +1485,20 @@ public DataflowPipelineJob run(Pipeline pipeline) { newJob.getEnvironment().setWorkerZone(options.getWorkerZone()); } - if (options.getFlexRSGoal() - == DataflowPipelineOptions.FlexResourceSchedulingGoal.COST_OPTIMIZED) { + if (options.getFlexRSGoal() == DataflowPipelineOptions.FlexResourceSchedulingGoal.COST_OPTIMIZED) { newJob.getEnvironment().setFlexResourceSchedulingGoal("FLEXRS_COST_OPTIMIZED"); - } else if (options.getFlexRSGoal() - == DataflowPipelineOptions.FlexResourceSchedulingGoal.SPEED_OPTIMIZED) { + } else if (options.getFlexRSGoal() == DataflowPipelineOptions.FlexResourceSchedulingGoal.SPEED_OPTIMIZED) { newJob.getEnvironment().setFlexResourceSchedulingGoal("FLEXRS_SPEED_OPTIMIZED"); } - // Represent the minCpuPlatform pipeline option as an experiment, if not already present. + // Represent the minCpuPlatform pipeline option as an experiment, if not already + // present. if (!isNullOrEmpty(dataflowOptions.getMinCpuPlatform())) { - List experiments = - firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); + List experiments = firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); - List minCpuFlags = - experiments.stream() - .filter(p -> p.startsWith("min_cpu_platform")) - .collect(Collectors.toList()); + List minCpuFlags = experiments.stream() + .filter(p -> p.startsWith("min_cpu_platform")) + .collect(Collectors.toList()); if (minCpuFlags.isEmpty()) { dataflowOptions.setExperiments( @@ -1498,10 +1520,13 @@ public DataflowPipelineJob run(Pipeline pipeline) { ImmutableList.copyOf( firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()))); - // Set the Docker container image that executes Dataflow worker harness, residing in Google - // Container Registry. Translator is guaranteed to create a worker pool prior to this point. + // Set the Docker container image that executes Dataflow worker harness, + // residing in Google + // Container Registry. Translator is guaranteed to create a worker pool prior to + // this point. // For runner_v1, only worker_harness_container is set. - // For runner_v2, both worker_harness_container and sdk_harness_container are set to the same + // For runner_v2, both worker_harness_container and sdk_harness_container are + // set to the same // value. String containerImage = getContainerImageForJob(options); for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) { @@ -1534,18 +1559,19 @@ public DataflowPipelineJob run(Pipeline pipeline) { if (hasExperiment(options, "upload_graph") && useUnifiedWorker(options)) { ArrayList experiments = new ArrayList<>(options.getExperiments()); - while (experiments.remove("upload_graph")) {} + while (experiments.remove("upload_graph")) { + } options.setExperiments(experiments); LOG.warn( "The upload_graph experiment was specified, but it does not apply " + "to runner v2 jobs. Option has been automatically removed."); } - // Upload the job to GCS and remove the graph object from the API call. The graph + // Upload the job to GCS and remove the graph object from the API call. The + // graph // will be downloaded from GCS by the service. if (hasExperiment(options, "upload_graph")) { - DataflowPackage stagedGraph = - options.getStager().stageToFile(jobGraphBytes, DATAFLOW_GRAPH_FILE_NAME); + DataflowPackage stagedGraph = options.getStager().stageToFile(jobGraphBytes, DATAFLOW_GRAPH_FILE_NAME); newJob.getSteps().clear(); newJob.setStepsLocation(stagedGraph.getLocation()); } @@ -1558,8 +1584,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { isNullOrEmpty(options.getDataflowJobFile()), "--dataflowJobFile and --templateLocation are mutually exclusive."); } - String fileLocation = - firstNonNull(options.getTemplateLocation(), options.getDataflowJobFile()); + String fileLocation = firstNonNull(options.getTemplateLocation(), options.getDataflowJobFile()); checkArgument( fileLocation.startsWith("/") || fileLocation.startsWith("gs://"), "Location must be local or on Cloud Storage, got %s.", @@ -1601,11 +1626,10 @@ public DataflowPipelineJob run(Pipeline pipeline) { String errorMessages = "Unexpected errors"; if (e.getDetails() != null) { if (jobGraphByteSize >= CREATE_JOB_REQUEST_LIMIT_BYTES) { - errorMessages = - "The size of the serialized JSON representation of the pipeline " - + "exceeds the allowable limit. " - + "For more information, please see the documentation on job submission:\n" - + "https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#jobs"; + errorMessages = "The size of the serialized JSON representation of the pipeline " + + "exceeds the allowable limit. " + + "For more information, please see the documentation on job submission:\n" + + "https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#jobs"; } else { errorMessages = e.getDetails().getMessage(); } @@ -1617,13 +1641,12 @@ public DataflowPipelineJob run(Pipeline pipeline) { // Use a raw client for post-launch monitoring, as status calls may fail // regularly and need not be retried automatically. - DataflowPipelineJob dataflowPipelineJob = - new DataflowPipelineJob( - DataflowClient.create(options), - jobResult.getId(), - options, - jobSpecification != null ? jobSpecification.getStepNames() : Collections.emptyMap(), - portablePipelineProto); + DataflowPipelineJob dataflowPipelineJob = new DataflowPipelineJob( + DataflowClient.create(options), + jobResult.getId(), + options, + jobSpecification != null ? jobSpecification.getStepNames() : Collections.emptyMap(), + portablePipelineProto); // If the service returned client request id, the SDK needs to compare it // with the original id generated in the request, if they are not the same @@ -1667,9 +1690,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { private static void printWorkSpecJsonToFile(String fileLocation, Job job) throws IOException { String workSpecJson = DataflowPipelineTranslator.jobToString(job); ResourceId fileResource = FileSystems.matchNewResource(fileLocation, false /* isDirectory */); - try (OutputStreamWriter writer = - new OutputStreamWriter( - Channels.newOutputStream(FileSystems.create(fileResource, MimeTypes.TEXT)), UTF_8)) { + try (OutputStreamWriter writer = new OutputStreamWriter( + Channels.newOutputStream(FileSystems.create(fileResource, MimeTypes.TEXT)), UTF_8)) { // Not using PrintWriter as it swallows IOException. // Not using BufferedWriter as this invokes write() only once. writer.write(workSpecJson); @@ -1678,8 +1700,7 @@ private static void printWorkSpecJsonToFile(String fileLocation, Job job) throws private static EnvironmentInfo getEnvironmentInfoFromEnvironmentId( String environmentId, RunnerApi.Pipeline pipelineProto) { - RunnerApi.Environment environment = - pipelineProto.getComponents().getEnvironmentsMap().get(environmentId); + RunnerApi.Environment environment = pipelineProto.getComponents().getEnvironmentsMap().get(environmentId); if (!BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER) .equals(environment.getUrn())) { throw new RuntimeException( @@ -1723,37 +1744,41 @@ private static List getAllEnvironmentInfo(RunnerApi.Pipeline pi static void configureSdkHarnessContainerImages( DataflowPipelineOptions options, RunnerApi.Pipeline pipelineProto, Job newJob) { - List sdkContainerList = - getAllEnvironmentInfo(pipelineProto).stream() - .map( - environmentInfo -> { - SdkHarnessContainerImage image = new SdkHarnessContainerImage(); - image.setEnvironmentId(environmentInfo.environmentId()); - image.setContainerImage(environmentInfo.containerUrl()); - if (!environmentInfo - .capabilities() - .contains( - BeamUrns.getUrn( - RunnerApi.StandardProtocols.Enum.MULTI_CORE_BUNDLE_PROCESSING))) { - image.setUseSingleCorePerContainer(true); - } - image.setCapabilities(environmentInfo.capabilities()); - return image; - }) - .collect(Collectors.toList()); + List sdkContainerList = getAllEnvironmentInfo(pipelineProto).stream() + .map( + environmentInfo -> { + SdkHarnessContainerImage image = new SdkHarnessContainerImage(); + image.setEnvironmentId(environmentInfo.environmentId()); + image.setContainerImage(environmentInfo.containerUrl()); + if (!environmentInfo + .capabilities() + .contains( + BeamUrns.getUrn( + RunnerApi.StandardProtocols.Enum.MULTI_CORE_BUNDLE_PROCESSING))) { + image.setUseSingleCorePerContainer(true); + } + image.setCapabilities(environmentInfo.capabilities()); + return image; + }) + .collect(Collectors.toList()); for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) { workerPool.setSdkHarnessContainerImages(sdkContainerList); } } - /** Returns true if the specified experiment is enabled, handling null experiments. */ + /** + * Returns true if the specified experiment is enabled, handling null + * experiments. + */ public static boolean hasExperiment(DataflowPipelineDebugOptions options, String experiment) { - List experiments = - firstNonNull(options.getExperiments(), Collections.emptyList()); + List experiments = firstNonNull(options.getExperiments(), Collections.emptyList()); return experiments.contains(experiment); } - /** Helper to configure the Dataflow Job Environment based on the user's job options. */ + /** + * Helper to configure the Dataflow Job Environment based on the user's job + * options. + */ private static Map getEnvironmentVersion(DataflowPipelineOptions options) { DataflowRunnerInfo runnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); String majorVersion; @@ -1833,7 +1858,9 @@ public DataflowPipelineTranslator getTranslator() { return translator; } - /** Sets callbacks to invoke during execution see {@code DataflowRunnerHooks}. */ + /** + * Sets callbacks to invoke during execution see {@code DataflowRunnerHooks}. + */ public void setHooks(DataflowRunnerHooks hooks) { this.hooks = hooks; } @@ -1852,18 +1879,21 @@ public CompositeBehavior enterCompositeTransform(Node node) { String rootBigQueryTransform = ""; if (transform.getClass().equals(StorageApiLoads.class)) { StorageApiLoads storageLoads = (StorageApiLoads) transform; - // If the storage load is directing exceptions to an error handler, we don't need to + // If the storage load is directing exceptions to an error handler, we don't + // need to // warn for unconsumed rows if (!storageLoads.usesErrorHandler()) { failedTag = storageLoads.getFailedRowsTag(); } - // For storage API the transform that outputs failed rows is nested one layer below + // For storage API the transform that outputs failed rows is nested one layer + // below // BigQueryIO. rootBigQueryTransform = node.getEnclosingNode().getFullName(); } else if (transform.getClass().equals(StreamingWriteTables.class)) { StreamingWriteTables streamingInserts = (StreamingWriteTables) transform; failedTag = streamingInserts.getFailedRowsTupleTag(); - // For streaming inserts the transform that outputs failed rows is nested two layers + // For streaming inserts the transform that outputs failed rows is nested two + // layers // below BigQueryIO. rootBigQueryTransform = node.getEnclosingNode().getEnclosingNode().getFullName(); } @@ -1896,9 +1926,12 @@ public void visitPrimitiveTransform(Node node) { } } - /** Outputs a warning about PCollection views without deterministic key coders. */ + /** + * Outputs a warning about PCollection views without deterministic key coders. + */ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) { - // We need to wait till this point to determine the names of the transforms since only + // We need to wait till this point to determine the names of the transforms + // since only // at this time do we know the hierarchy of the transforms otherwise we could // have just recorded the full names during apply time. if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) { @@ -1906,7 +1939,8 @@ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pip pipeline.traverseTopologically( new PipelineVisitor.Defaults() { @Override - public void visitValue(PValue value, TransformHierarchy.Node producer) {} + public void visitValue(PValue value, TransformHierarchy.Node producer) { + } @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { @@ -1919,8 +1953,8 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { if (node.getTransform() instanceof View.AsMap || node.getTransform() instanceof View.AsMultimap) { - PCollection> input = - (PCollection>) Iterables.getOnlyElement(node.getInputs().values()); + PCollection> input = (PCollection>) Iterables + .getOnlyElement(node.getInputs().values()); KvCoder inputCoder = (KvCoder) input.getCoder(); try { inputCoder.getKeyCoder().verifyDeterministic(); @@ -1935,7 +1969,8 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { } @Override - public void leaveCompositeTransform(TransformHierarchy.Node node) {} + public void leaveCompositeTransform(TransformHierarchy.Node node) { + } }); LOG.warn( @@ -1948,7 +1983,8 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) {} } /** - * Returns true if the passed in {@link PCollection} needs to be materialized using an indexed + * Returns true if the passed in {@link PCollection} needs to be materialized + * using an indexed * format. */ boolean doesPCollectionRequireIndexedFormat(PCollection pcol) { @@ -1956,7 +1992,8 @@ boolean doesPCollectionRequireIndexedFormat(PCollection pcol) { } /** - * Marks the passed in {@link PCollection} as requiring to be materialized using an indexed + * Marks the passed in {@link PCollection} as requiring to be materialized using + * an indexed * format. */ void addPCollectionRequiringIndexedFormat(PCollection pcol) { @@ -1998,8 +2035,7 @@ void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) { // PubsubIO translations // ================================================================================ private static class StreamingPubsubIOReadOverrideFactory - implements PTransformOverrideFactory< - PBegin, PCollection, PubsubUnboundedSource> { + implements PTransformOverrideFactory, PubsubUnboundedSource> { @Override public PTransformReplacement> getReplacementTransform( @@ -2016,7 +2052,8 @@ public Map, ReplacementOutput> mapOutputs( } /** - * Suppress application of {@link PubsubUnboundedSource#expand} in streaming mode so that we can + * Suppress application of {@link PubsubUnboundedSource#expand} in streaming + * mode so that we can * instead defer to Windmill's implementation. */ private static class StreamingPubsubIORead @@ -2034,10 +2071,9 @@ public PubsubUnboundedSource getOverriddenTransform() { @Override public PCollection expand(PBegin input) { - Coder coder = - transform.getNeedsMessageId() - ? new PubsubMessageWithAttributesAndMessageIdCoder() - : new PubsubMessageWithAttributesCoder(); + Coder coder = transform.getNeedsMessageId() + ? new PubsubMessageWithAttributesAndMessageIdCoder() + : new PubsubMessageWithAttributesCoder(); return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED, coder); } @@ -2084,10 +2120,14 @@ private static void translateOverriddenPubsubSourceStep( stepTranslationContext.addInput( PropertyNames.PUBSUB_ID_ATTRIBUTE, overriddenTransform.getIdAttribute()); } - // In both cases, the transform needs to read PubsubMessage. However, in case it needs - // the attributes or messageId, we supply an identity "parse fn" so the worker will - // read PubsubMessage's from Windmill and simply pass them around; and in case it - // doesn't need attributes, we're already implicitly using a "Coder" that interprets + // In both cases, the transform needs to read PubsubMessage. However, in case it + // needs + // the attributes or messageId, we supply an identity "parse fn" so the worker + // will + // read PubsubMessage's from Windmill and simply pass them around; and in case + // it + // doesn't need attributes, we're already implicitly using a "Coder" that + // interprets // the data as a PubsubMessage's payload. if (overriddenTransform.getNeedsAttributes() || overriddenTransform.getNeedsMessageId()) { stepTranslationContext.addInput( @@ -2121,7 +2161,8 @@ public PubsubMessage apply(PubsubMessage input) { } /** - * Suppress application of {@link PubsubUnboundedSink#expand} in streaming mode so that we can + * Suppress application of {@link PubsubUnboundedSink#expand} in streaming mode + * so that we can * instead defer to Windmill's implementation. */ static class StreamingPubsubIOWrite extends PTransform, PDone> { @@ -2210,7 +2251,8 @@ private static void translate( PropertyNames.PUBSUB_SERIALIZED_ATTRIBUTES_FN, byteArrayToJsonString(serializeToByteArray(new IdentityMessageFn()))); - // Using a GlobalWindowCoder as a placeholder because GlobalWindowCoder is known coder. + // Using a GlobalWindowCoder as a placeholder because GlobalWindowCoder is known + // coder. stepContext.addEncodingInput( WindowedValues.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE)); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); @@ -2269,13 +2311,11 @@ public void translate(Impulse transform, TransformTranslator.TranslationContext } else { StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, "impulse"); - WindowedValues.FullWindowedValueCoder coder = - WindowedValues.getFullCoder( - context.getOutput(transform).getCoder(), GlobalWindow.Coder.INSTANCE); + WindowedValues.FullWindowedValueCoder coder = WindowedValues.getFullCoder( + context.getOutput(transform).getCoder(), GlobalWindow.Coder.INSTANCE); byte[] encodedImpulse; try { - encodedImpulse = - encodeToByteArray(coder, WindowedValues.valueInGlobalWindow(new byte[0])); + encodedImpulse = encodeToByteArray(coder, WindowedValues.valueInGlobalWindow(new byte[0])); } catch (Exception e) { throw new RuntimeException(e); } @@ -2307,10 +2347,13 @@ public Map, ReplacementOutput> mapOutputs( } /** - * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Unbounded Read.Unbounded} for + * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Unbounded + * Read.Unbounded} for * the Dataflow runner in streaming mode. * - *

In particular, if an UnboundedSource requires deduplication, then features of WindmillSink + *

+ * In particular, if an UnboundedSource requires deduplication, then features of + * WindmillSink * are leveraged to do the deduplication. */ private static class StreamingUnboundedRead extends PTransform> { @@ -2389,8 +2432,9 @@ public void translate( private static class Deduplicate extends PTransform>, PCollection> { - // Use a finite set of keys to improve bundling. Without this, the key space - // will be the space of ids which is potentially very large, which results in much + // Use a finite set of keys to improve bundling. Without this, the key space + // will be the space of ids which is potentially very large, which results in + // much // more per-key overhead. private static final int NUM_RESHARD_KEYS = 10000; @@ -2399,10 +2443,10 @@ public PCollection expand(PCollection> input) { return input .apply( WithKeys.of( - (ValueWithRecordId value) -> - Arrays.hashCode(value.getId()) % NUM_RESHARD_KEYS) + (ValueWithRecordId value) -> Arrays.hashCode(value.getId()) % NUM_RESHARD_KEYS) .withKeyType(TypeDescriptors.integers())) - // Reshuffle will dedup based on ids in ValueWithRecordId by passing the data through + // Reshuffle will dedup based on ids in ValueWithRecordId by passing the data + // through // WindmillSink. .apply(Reshuffle.of()) .apply( @@ -2435,7 +2479,8 @@ public Map, ReplacementOutput> mapOutputs( } /** - * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded Read.Bounded} for the + * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded + * Read.Bounded} for the * Dataflow runner in streaming mode. */ private static class StreamingBoundedRead extends PTransform> { @@ -2456,7 +2501,8 @@ public final PCollection expand(PBegin input) { } /** - * A marker {@link DoFn} for writing the contents of a {@link PCollection} to a streaming {@link + * A marker {@link DoFn} for writing the contents of a {@link PCollection} to a + * streaming {@link * PCollectionView} backend implementation. */ public static class StreamingPCollectionViewWriterFn extends DoFn, T> { @@ -2546,19 +2592,12 @@ public Combine.GroupedValues getOriginalCombine() { } private static class PrimitiveCombineGroupedValuesOverrideFactory - implements PTransformOverrideFactory< - PCollection>>, - PCollection>, - Combine.GroupedValues> { + implements + PTransformOverrideFactory>>, PCollection>, Combine.GroupedValues> { @Override - public PTransformReplacement>>, PCollection>> - getReplacementTransform( - AppliedPTransform< - PCollection>>, - PCollection>, - GroupedValues> - transform) { + public PTransformReplacement>>, PCollection>> getReplacementTransform( + AppliedPTransform>>, PCollection>, GroupedValues> transform) { return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), new CombineGroupedValues<>( @@ -2599,12 +2638,11 @@ public Map, ReplacementOutput> mapOutputs( @VisibleForTesting static class StreamingShardedWriteFactory - implements PTransformOverrideFactory< - PCollection, - WriteFilesResult, - WriteFiles> { + implements + PTransformOverrideFactory, WriteFilesResult, WriteFiles> { - // We pick 10 as a default, as it works well with the default number of workers started + // We pick 10 as a default, as it works well with the default number of workers + // started // by Dataflow. static final int DEFAULT_NUM_SHARDS = 10; DataflowPipelineWorkerPoolOptions options; @@ -2614,19 +2652,19 @@ static class StreamingShardedWriteFactory } @Override - public PTransformReplacement, WriteFilesResult> - getReplacementTransform( - AppliedPTransform< - PCollection, - WriteFilesResult, - WriteFiles> - transform) { - // By default, if numShards is not set WriteFiles will produce one file per bundle. In - // streaming, there are large numbers of small bundles, resulting in many tiny files. - // Instead, we pick max workers * 2 to ensure full parallelism, but prevent too-many files. - // (current_num_workers * 2 might be a better choice, but that value is not easily available + public PTransformReplacement, WriteFilesResult> getReplacementTransform( + AppliedPTransform, WriteFilesResult, WriteFiles> transform) { + // By default, if numShards is not set WriteFiles will produce one file per + // bundle. In + // streaming, there are large numbers of small bundles, resulting in many tiny + // files. + // Instead, we pick max workers * 2 to ensure full parallelism, but prevent + // too-many files. + // (current_num_workers * 2 might be a better choice, but that value is not + // easily available // today). - // If the user does not set either numWorkers or maxNumWorkers, default to 10 shards. + // If the user does not set either numWorkers or maxNumWorkers, default to 10 + // shards. int numShards; if (options.getMaxNumWorkers() > 0) { numShards = options.getMaxNumWorkers() * 2; @@ -2637,11 +2675,9 @@ static class StreamingShardedWriteFactory } try { - List> sideInputs = - WriteFilesTranslation.getDynamicDestinationSideInputs(transform); + List> sideInputs = WriteFilesTranslation.getDynamicDestinationSideInputs(transform); FileBasedSink sink = WriteFilesTranslation.getSink(transform); - WriteFiles replacement = - WriteFiles.to(sink).withSideInputs(sideInputs); + WriteFiles replacement = WriteFiles.to(sink).withSideInputs(sideInputs); if (WriteFilesTranslation.isWindowedWrites(transform)) { replacement = replacement.withWindowedWrites(); } @@ -2693,7 +2729,8 @@ static String getDefaultContainerImageUrl(DataflowPipelineOptions options) { } /** - * Construct the default Dataflow container image name based on pipeline type and Java version. + * Construct the default Dataflow container image name based on pipeline type + * and Java version. */ static String getDefaultContainerImageNameForJob(DataflowPipelineOptions options) { Environments.JavaVersion javaVersion = Environments.getJavaVersion(); @@ -2707,7 +2744,8 @@ static String getDefaultContainerImageNameForJob(DataflowPipelineOptions options } /** - * Construct the default Dataflow container image name based on pipeline type and Java version. + * Construct the default Dataflow container image name based on pipeline type + * and Java version. */ static String getDefaultContainerVersion(DataflowPipelineOptions options) { DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); @@ -2758,7 +2796,8 @@ static void verifyStateSupportForWindowingStrategy(WindowingStrategy strategy) { } /** - * These are for dataflow-specific classes where we put fake stubs in the pipeline proto to pass + * These are for dataflow-specific classes where we put fake stubs in the + * pipeline proto to pass * validation. */ private static class DataflowPayloadTranslator @@ -2784,14 +2823,13 @@ public RunnerApi.FunctionSpec translate( } @SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) + "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) @AutoService(TransformPayloadTranslatorRegistrar.class) public static class DataflowTransformTranslator implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> - getTransformPayloadTranslators() { + public Map, ? extends TransformPayloadTranslator> getTransformPayloadTranslators() { TransformPayloadTranslator dummyTranslator = new DataflowPayloadTranslator(); return ImmutableMap., TransformPayloadTranslator>builder() .put(CreateDataflowView.class, dummyTranslator) diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index bb2adfb08655..31663b840f31 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -2319,6 +2319,16 @@ public void testBatchGroupIntoBatchesWithShardedKeyOverrideCount() throws IOExce verifyGroupIntoBatchesOverrideCount(p, true, true); } + @Test + public void testBatchGroupIntoBatchesWithShardedKeyOverrideCountV2() throws IOException { + PipelineOptions options = buildPipelineOptions(); + options + .as(DataflowPipelineOptions.class) + .setExperiments(Arrays.asList("use_runner_v2", "use_unified_worker")); + Pipeline p = Pipeline.create(options); + verifyGroupIntoBatchesOverrideCount(p, true, true); + } + @Test public void testBatchGroupIntoBatchesWithShardedKeyOverrideBytes() throws IOException { PipelineOptions options = buildPipelineOptions(); From 076fda71ab51bfdfc6d4f2a3d2fdf58074035a2f Mon Sep 17 00:00:00 2001 From: Atharva262005 Date: Fri, 23 Jan 2026 23:05:48 +0530 Subject: [PATCH 7/7] Fix Spotless violations in DataflowRunner.java --- .../beam/runners/dataflow/DataflowRunner.java | 506 +++++++++--------- 1 file changed, 265 insertions(+), 241 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index aa8bed9296c8..335916a6abca 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -186,37 +186,24 @@ import org.slf4j.LoggerFactory; /** - * A {@link PipelineRunner} that executes the operations in the pipeline by - * first translating them - * to the Dataflow representation using the {@link DataflowPipelineTranslator} - * and then submitting + * A {@link PipelineRunner} that executes the operations in the pipeline by first translating them + * to the Dataflow representation using the {@link DataflowPipelineTranslator} and then submitting * them to a Dataflow service for execution. * *

Permissions

* - *

- * When reading from a Dataflow source or writing to a Dataflow sink using - * {@code - * DataflowRunner}, the Google cloudservices account and the Google compute - * engine service account - * of the GCP project running the Dataflow Job will need access to the - * corresponding source/sink. + *

When reading from a Dataflow source or writing to a Dataflow sink using {@code + * DataflowRunner}, the Google cloudservices account and the Google compute engine service account + * of the GCP project running the Dataflow Job will need access to the corresponding source/sink. * - *

- * Please see - * Google - * Cloud + *

Please see Google Cloud * Dataflow Security and Permissions for more details. * - *

- * DataflowRunner now supports creating job templates using the - * {@code --templateLocation} - * option. If this option is set, the runner will generate a template instead of - * running the + *

DataflowRunner now supports creating job templates using the {@code --templateLocation} + * option. If this option is set, the runner will generate a template instead of running the * pipeline immediately. * - *

- * Example: + *

Example: * *

{@code
  * --runner=DataflowRunner
@@ -224,13 +211,14 @@
  * }
*/ @SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public class DataflowRunner extends PipelineRunner { /** Experiment to "unsafely attempt to process unbounded data in batch mode". */ - public static final String UNSAFELY_ATTEMPT_TO_PROCESS_UNBOUNDED_DATA_IN_BATCH_MODE = "unsafely_attempt_to_process_unbounded_data_in_batch_mode"; + public static final String UNSAFELY_ATTEMPT_TO_PROCESS_UNBOUNDED_DATA_IN_BATCH_MODE = + "unsafely_attempt_to_process_unbounded_data_in_batch_mode"; private static final Logger LOG = LoggerFactory.getLogger(DataflowRunner.class); /** Provided configuration options. */ @@ -242,33 +230,27 @@ public class DataflowRunner extends PipelineRunner { /** Translator for this DataflowRunner, based on options. */ private final DataflowPipelineTranslator translator; - /** - * A set of user defined functions to invoke at different points in execution. - */ + /** A set of user defined functions to invoke at different points in execution. */ private DataflowRunnerHooks hooks; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; - @VisibleForTesting - static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1024 * 1024; + @VisibleForTesting static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1024 * 1024; - @VisibleForTesting - static final String PIPELINE_FILE_NAME = "pipeline.pb"; - @VisibleForTesting - static final String DATAFLOW_GRAPH_FILE_NAME = "dataflow_graph.json"; + @VisibleForTesting static final String PIPELINE_FILE_NAME = "pipeline.pb"; + @VisibleForTesting static final String DATAFLOW_GRAPH_FILE_NAME = "dataflow_graph.json"; private static final ObjectMapper MAPPER = new ObjectMapper(); /** - * Use an {@link ObjectMapper} configured with any {@link Module}s in the class - * path allowing for - * user specified configuration injection into the ObjectMapper. This supports - * user custom types + * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing for + * user specified configuration injection into the ObjectMapper. This supports user custom types * on {@link PipelineOptions}. */ - private static final ObjectMapper MAPPER_WITH_MODULES = new ObjectMapper() - .registerModules(ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + private static final ObjectMapper MAPPER_WITH_MODULES = + new ObjectMapper() + .registerModules(ObjectMapper.findModules(ReflectHelpers.findClassLoader())); private final Set> pcollectionsRequiringIndexedFormat; @@ -276,12 +258,9 @@ public class DataflowRunner extends PipelineRunner { private final Set> pcollectionsRequiringAutoSharding; /** - * Project IDs must contain lowercase letters, digits, or dashes. IDs must start - * with a letter and - * may not end with a dash. This regex isn't exact - this allows for patterns - * that would be - * rejected by the service, but this is sufficient for basic validation of - * project IDs. + * Project IDs must contain lowercase letters, digits, or dashes. IDs must start with a letter and + * may not end with a dash. This regex isn't exact - this allows for patterns that would be + * rejected by the service, but this is sufficient for basic validation of project IDs. */ public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]"; @@ -289,15 +268,11 @@ public class DataflowRunner extends PipelineRunner { static final String ENDPOINT_REGEXP = "https://[\\S]*googleapis\\.com[/]?"; /** - * Replaces GCS file paths with local file paths by downloading the GCS files - * locally. This is - * useful when files need to be accessed locally before being staged to - * Dataflow. + * Replaces GCS file paths with local file paths by downloading the GCS files locally. This is + * useful when files need to be accessed locally before being staged to Dataflow. * - * @param filesToStage List of file paths that may contain GCS paths (gs://) and - * local paths - * @return List of local file paths where any GCS paths have been downloaded - * locally + * @param filesToStage List of file paths that may contain GCS paths (gs://) and local paths + * @return List of local file paths where any GCS paths have been downloaded locally * @throws RuntimeException if there are errors copying GCS files locally */ public static List replaceGcsFilesWithLocalFiles(List filesToStage) { @@ -385,7 +360,8 @@ public static List replaceGcsFilesWithLocalFiles(List filesToSta * @return The newly created runner. */ public static DataflowRunner fromOptions(PipelineOptions options) { - DataflowPipelineOptions dataflowOptions = PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options); + DataflowPipelineOptions dataflowOptions = + PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options); ArrayList missing = new ArrayList<>(); if (dataflowOptions.getAppName() == null) { @@ -505,7 +481,8 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { + " description."); } - DataflowPipelineDebugOptions debugOptions = dataflowOptions.as(DataflowPipelineDebugOptions.class); + DataflowPipelineDebugOptions debugOptions = + dataflowOptions.as(DataflowPipelineDebugOptions.class); // Verify the number of worker threads is a valid value if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) { throw new IllegalArgumentException( @@ -519,7 +496,8 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { } // Adding the Java version to the SDK name for user's and support convenience. - String agentJavaVer = String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); + String agentJavaVer = + String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); String userAgentName = dataflowRunnerInfo.getName(); @@ -528,7 +506,8 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { String userAgentVersion = dataflowRunnerInfo.getVersion(); Preconditions.checkArgument( !userAgentVersion.equals(""), "Dataflow runner's `version` property cannot be empty."); - String userAgent = String.format("%s/%s%s", userAgentName, userAgentVersion, agentJavaVer).replace(" ", "_"); + String userAgent = + String.format("%s/%s%s", userAgentName, userAgentVersion, agentJavaVer).replace(" ", "_"); dataflowOptions.setUserAgent(userAgent); return new DataflowRunner(dataflowOptions); @@ -619,8 +598,9 @@ private static class AlwaysCreateViaRead implements PTransformOverrideFactory, Create.Values> { @Override - public PTransformOverrideFactory.PTransformReplacement> getReplacementTransform( - AppliedPTransform, Create.Values> appliedTransform) { + public PTransformOverrideFactory.PTransformReplacement> + getReplacementTransform( + AppliedPTransform, Create.Values> appliedTransform) { return PTransformOverrideFactory.PTransformReplacement.of( appliedTransform.getPipeline().begin(), appliedTransform.getTransform().alwaysUseRead()); } @@ -823,8 +803,9 @@ private List getOverrides(boolean streaming) { PTransformMatchers.classEqualTo(ParDo.SingleOutput.class), new PrimitiveParDoSingleFactory())); - boolean usesAtLeastOnceStreamingMode = options.getDataflowServiceOptions() != null - && options.getDataflowServiceOptions().contains("streaming_mode_at_least_once"); + boolean usesAtLeastOnceStreamingMode = + options.getDataflowServiceOptions() != null + && options.getDataflowServiceOptions().contains("streaming_mode_at_least_once"); overridesBuilder.add( PTransformOverride.of( PTransformMatchers.classEqualTo(RedistributeByKey.class), @@ -856,10 +837,8 @@ private List getOverrides(boolean streaming) { } /** - * Replace the {@link Combine.GloballyAsSingletonView} transform with a - * specialization which - * re-applies the {@link CombineFn} and adds a specialization specific to the - * Dataflow runner. + * Replace the {@link Combine.GloballyAsSingletonView} transform with a specialization which + * re-applies the {@link CombineFn} and adds a specialization specific to the Dataflow runner. */ private static class CombineGloballyAsSingletonViewOverrideFactory extends ReflectiveViewOverrideFactory { @@ -869,10 +848,15 @@ private CombineGloballyAsSingletonViewOverrideFactory(DataflowRunner runner) { } @Override - public PTransformReplacement, PCollectionView> getReplacementTransform( - AppliedPTransform, PCollectionView, PTransform, PCollectionView>> transform) { - Combine.GloballyAsSingletonView combineTransform = (Combine.GloballyAsSingletonView) transform - .getTransform(); + public PTransformReplacement, PCollectionView> + getReplacementTransform( + AppliedPTransform< + PCollection, + PCollectionView, + PTransform, PCollectionView>> + transform) { + Combine.GloballyAsSingletonView combineTransform = + (Combine.GloballyAsSingletonView) transform.getTransform(); return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), new BatchViewOverrides.BatchViewAsSingleton( @@ -884,15 +868,15 @@ public PTransformReplacement, PCollectionView> getRep } /** - * Replace the View.AsYYY transform with specialized view overrides for - * Dataflow. It is required - * that the new replacement transform uses the supplied PCollectionView and does - * not create + * Replace the View.AsYYY transform with specialized view overrides for Dataflow. It is required + * that the new replacement transform uses the supplied PCollectionView and does not create * another instance. */ private static class ReflectiveViewOverrideFactory - implements - PTransformOverrideFactory, PCollectionView, PTransform, PCollectionView>> { + implements PTransformOverrideFactory< + PCollection, + PCollectionView, + PTransform, PCollectionView>> { final Class, PCollectionView>> replacement; final DataflowRunner runner; @@ -905,7 +889,11 @@ private ReflectiveViewOverrideFactory( } CreatePCollectionView findCreatePCollectionView( - final AppliedPTransform, PCollectionView, PTransform, PCollectionView>> transform) { + final AppliedPTransform< + PCollection, + PCollectionView, + PTransform, PCollectionView>> + transform) { final AtomicReference viewTransformRef = new AtomicReference<>(); transform .getPipeline() @@ -952,13 +940,19 @@ public void leaveCompositeTransform(Node node) { } @Override - public PTransformReplacement, PCollectionView> getReplacementTransform( - final AppliedPTransform, PCollectionView, PTransform, PCollectionView>> transform) { - - PTransform, PCollectionView> rep = InstanceBuilder.ofType(replacement) - .withArg(DataflowRunner.class, runner) - .withArg(CreatePCollectionView.class, findCreatePCollectionView(transform)) - .build(); + public PTransformReplacement, PCollectionView> + getReplacementTransform( + final AppliedPTransform< + PCollection, + PCollectionView, + PTransform, PCollectionView>> + transform) { + + PTransform, PCollectionView> rep = + InstanceBuilder.ofType(replacement) + .withArg(DataflowRunner.class, runner) + .withArg(CreatePCollectionView.class, findCreatePCollectionView(transform)) + .build(); return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), (PTransform) rep); } @@ -979,8 +973,10 @@ public Map, ReplacementOutput> mapOutputs( } } - private static class ReflectiveOneToOneOverrideFactory, PCollection>> - extends SingleInputOutputOverrideFactory, PCollection, TransformT> { + private static class ReflectiveOneToOneOverrideFactory< + InputT, OutputT, TransformT extends PTransform, PCollection>> + extends SingleInputOutputOverrideFactory< + PCollection, PCollection, TransformT> { private final Class, PCollection>> replacement; private final DataflowRunner runner; @@ -995,11 +991,12 @@ private ReflectiveOneToOneOverrideFactory( @Override public PTransformReplacement, PCollection> getReplacementTransform( AppliedPTransform, PCollection, TransformT> transform) { - PTransform, PCollection> rep = InstanceBuilder.ofType(replacement) - .withArg(DataflowRunner.class, runner) - .withArg( - (Class) transform.getTransform().getClass(), transform.getTransform()) - .build(); + PTransform, PCollection> rep = + InstanceBuilder.ofType(replacement) + .withArg(DataflowRunner.class, runner) + .withArg( + (Class) transform.getTransform().getClass(), transform.getTransform()) + .build(); return PTransformReplacement.of(PTransformReplacements.getSingletonMainInput(transform), rep); } } @@ -1008,7 +1005,8 @@ private RunnerApi.Pipeline resolveAnyOfEnvironments(RunnerApi.Pipeline pipeline) RunnerApi.Pipeline.Builder pipelineBuilder = pipeline.toBuilder(); RunnerApi.Components.Builder componentsBuilder = pipelineBuilder.getComponentsBuilder(); componentsBuilder.clearEnvironments(); - for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { + for (Map.Entry entry : + pipeline.getComponents().getEnvironmentsMap().entrySet()) { componentsBuilder.putEnvironments( entry.getKey(), Environments.resolveAnyOfEnvironment( @@ -1021,9 +1019,10 @@ private RunnerApi.Pipeline resolveAnyOfEnvironments(RunnerApi.Pipeline pipeline) protected RunnerApi.Pipeline applySdkEnvironmentOverrides( RunnerApi.Pipeline pipeline, DataflowPipelineOptions options) { String sdkHarnessContainerImageOverrides = options.getSdkHarnessContainerImageOverrides(); - String[] overrides = Strings.isNullOrEmpty(sdkHarnessContainerImageOverrides) - ? new String[0] - : sdkHarnessContainerImageOverrides.split(",", -1); + String[] overrides = + Strings.isNullOrEmpty(sdkHarnessContainerImageOverrides) + ? new String[0] + : sdkHarnessContainerImageOverrides.split(",", -1); if (overrides.length % 2 != 0) { throw new RuntimeException( "invalid syntax for SdkHarnessContainerImageOverrides: " @@ -1032,7 +1031,8 @@ protected RunnerApi.Pipeline applySdkEnvironmentOverrides( RunnerApi.Pipeline.Builder pipelineBuilder = pipeline.toBuilder(); RunnerApi.Components.Builder componentsBuilder = pipelineBuilder.getComponentsBuilder(); componentsBuilder.clearEnvironments(); - for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { + for (Map.Entry entry : + pipeline.getComponents().getEnvironmentsMap().entrySet()) { RunnerApi.Environment.Builder environmentBuilder = entry.getValue().toBuilder(); if (BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER) .equals(environmentBuilder.getUrn())) { @@ -1054,10 +1054,12 @@ protected RunnerApi.Pipeline applySdkEnvironmentOverrides( && !updated // don't update if the container image is already configured by DataflowRunner && !containerImage.equals(getContainerImageForJob(options))) { - String imageAndTag = normalizeDataflowImageAndTag( - containerImage.substring(containerImage.lastIndexOf("/"))); - containerImage = DataflowRunnerInfo.getDataflowRunnerInfo().getContainerImageBaseRepository() - + imageAndTag; + String imageAndTag = + normalizeDataflowImageAndTag( + containerImage.substring(containerImage.lastIndexOf("/"))); + containerImage = + DataflowRunnerInfo.getDataflowRunnerInfo().getContainerImageBaseRepository() + + imageAndTag; } environmentBuilder.setPayload( RunnerApi.DockerPayload.newBuilder() @@ -1093,7 +1095,8 @@ protected RunnerApi.Pipeline resolveArtifacts(RunnerApi.Pipeline pipeline) { RunnerApi.Pipeline.Builder pipelineBuilder = pipeline.toBuilder(); RunnerApi.Components.Builder componentsBuilder = pipelineBuilder.getComponentsBuilder(); componentsBuilder.clearEnvironments(); - for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { + for (Map.Entry entry : + pipeline.getComponents().getEnvironmentsMap().entrySet()) { RunnerApi.Environment.Builder environmentBuilder = entry.getValue().toBuilder(); environmentBuilder.clearDependencies(); for (RunnerApi.ArtifactInformation info : entry.getValue().getDependenciesList()) { @@ -1111,8 +1114,8 @@ protected RunnerApi.Pipeline resolveArtifacts(RunnerApi.Pipeline pipeline) { if (BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO) .equals(info.getRoleUrn())) { try { - RunnerApi.ArtifactStagingToRolePayload stagingPayload = RunnerApi.ArtifactStagingToRolePayload - .parseFrom(info.getRolePayload()); + RunnerApi.ArtifactStagingToRolePayload stagingPayload = + RunnerApi.ArtifactStagingToRolePayload.parseFrom(info.getRolePayload()); stagedName = stagingPayload.getStagedName(); } catch (InvalidProtocolBufferException e) { throw new RuntimeException("Error parsing artifact staging_to role payload.", e); @@ -1150,7 +1153,8 @@ protected RunnerApi.Pipeline resolveArtifacts(RunnerApi.Pipeline pipeline) { protected List stageArtifacts(RunnerApi.Pipeline pipeline) { ImmutableList.Builder filesToStageBuilder = ImmutableList.builder(); Set stagedNames = new HashSet<>(); - for (Map.Entry entry : pipeline.getComponents().getEnvironmentsMap().entrySet()) { + for (Map.Entry entry : + pipeline.getComponents().getEnvironmentsMap().entrySet()) { for (RunnerApi.ArtifactInformation info : entry.getValue().getDependenciesList()) { if (!BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE).equals(info.getTypeUrn())) { throw new RuntimeException( @@ -1166,8 +1170,8 @@ protected List stageArtifacts(RunnerApi.Pipeline pipeline) { if (BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO) .equals(info.getRoleUrn())) { try { - RunnerApi.ArtifactStagingToRolePayload stagingPayload = RunnerApi.ArtifactStagingToRolePayload - .parseFrom(info.getRolePayload()); + RunnerApi.ArtifactStagingToRolePayload stagingPayload = + RunnerApi.ArtifactStagingToRolePayload.parseFrom(info.getRolePayload()); stagedName = stagingPayload.getStagedName(); } catch (InvalidProtocolBufferException e) { throw new RuntimeException("Error parsing artifact staging_to role payload.", e); @@ -1197,7 +1201,8 @@ protected List stageArtifacts(RunnerApi.Pipeline pipeline) { private List getDefaultArtifacts() { ImmutableList.Builder pathsToStageBuilder = ImmutableList.builder(); - String windmillBinary = options.as(DataflowStreamingPipelineOptions.class).getOverrideWindmillBinary(); + String windmillBinary = + options.as(DataflowStreamingPipelineOptions.class).getOverrideWindmillBinary(); String dataflowWorkerJar = options.getDataflowWorkerJar(); if (dataflowWorkerJar != null && !dataflowWorkerJar.isEmpty() && !useUnifiedWorker(options)) { // Put the user specified worker jar at the start of the classpath, to be @@ -1245,10 +1250,11 @@ public void visitPrimitiveTransform(Node node) { private static boolean includesTransformUpgrades(Pipeline pipeline) { return (pipeline - .getOptions() - .as(ExternalTranslationOptions.class) - .getTransformsToOverride() - .size() > 0); + .getOptions() + .as(ExternalTranslationOptions.class) + .getTransformsToOverride() + .size() + > 0); } private static final Random RANDOM = new Random(); @@ -1275,7 +1281,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { throw new IllegalArgumentException( "Runner V2 both disabled and enabled: at least one of ['beam_fn_api', 'use_unified_worker', 'use_runner_v2', 'use_portable_job_submission'] is set and also one of ['disable_runner_v2', 'disable_runner_v2_until_2023', 'disable_prime_runner_v2'] is set."); } - List experiments = new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true + List experiments = + new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true if (!experiments.contains("use_runner_v2")) { experiments.add("use_runner_v2"); } @@ -1316,7 +1323,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { if (useUnifiedWorker(options)) { options.setEnableStreamingEngine(true); - List experiments = new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true + List experiments = + new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true if (!experiments.contains("enable_streaming_engine")) { experiments.add("enable_streaming_engine"); } @@ -1344,8 +1352,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { // The same Environment is used in different and contradictory ways, depending // on whether // it is a v1 or v2 job submission. - RunnerApi.Environment defaultEnvironmentForDataflow = Environments - .createDockerEnvironment(workerHarnessContainerImageURL); + RunnerApi.Environment defaultEnvironmentForDataflow = + Environments.createDockerEnvironment(workerHarnessContainerImageURL); // The SdkComponents for portable an non-portable job submission must be kept // distinct. Both @@ -1358,7 +1366,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { .addAllCapabilities(Environments.getJavaCapabilities()) .build()); - RunnerApi.Pipeline portablePipelineProto = PipelineTranslation.toProto(pipeline, portableComponents, false); + RunnerApi.Pipeline portablePipelineProto = + PipelineTranslation.toProto(pipeline, portableComponents, false); // Note that `stageArtifacts` has to be called before `resolveArtifact` because // `resolveArtifact` updates local paths to staged paths in pipeline proto. portablePipelineProto = resolveAnyOfEnvironments(portablePipelineProto); @@ -1378,23 +1387,26 @@ public DataflowPipelineJob run(Pipeline pipeline) { LOG.info("Staging portable pipeline proto to {}", options.getStagingLocation()); byte[] serializedProtoPipeline = portablePipelineProto.toByteArray(); - DataflowPackage stagedPipeline = options.getStager().stageToFile(serializedProtoPipeline, PIPELINE_FILE_NAME); + DataflowPackage stagedPipeline = + options.getStager().stageToFile(serializedProtoPipeline, PIPELINE_FILE_NAME); dataflowOptions.setPipelineUrl(stagedPipeline.getLocation()); if (useUnifiedWorker(options)) { LOG.info("Skipping v1 transform replacements since job will run on v2."); if (!options.isStreaming()) { LOG.info("Applying GroupIntoBatches overrides for V2 Batch mode."); - List overrides = ImmutableList.builder() - .add( - PTransformOverride.of( - PTransformMatchers.classEqualTo(GroupIntoBatches.class), - new GroupIntoBatchesOverride.BatchGroupIntoBatchesOverrideFactory<>(this))) - .add( - PTransformOverride.of( - PTransformMatchers.classEqualTo(GroupIntoBatches.WithShardedKey.class), - new GroupIntoBatchesOverride.BatchGroupIntoBatchesWithShardedKeyOverrideFactory<>(this))) - .build(); + List overrides = + ImmutableList.builder() + .add( + PTransformOverride.of( + PTransformMatchers.classEqualTo(GroupIntoBatches.class), + new GroupIntoBatchesOverride.BatchGroupIntoBatchesOverrideFactory<>(this))) + .add( + PTransformOverride.of( + PTransformMatchers.classEqualTo(GroupIntoBatches.WithShardedKey.class), + new GroupIntoBatchesOverride + .BatchGroupIntoBatchesWithShardedKeyOverrideFactory<>(this))) + .build(); pipeline.replaceAll(overrides); } } else { @@ -1411,8 +1423,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { .addAllCapabilities(Environments.getJavaCapabilities()) .build()); // No need to perform transform upgrading for the Runner v1 proto. - RunnerApi.Pipeline dataflowV1PipelineProto = PipelineTranslation.toProto(pipeline, dataflowV1Components, true, - false); + RunnerApi.Pipeline dataflowV1PipelineProto = + PipelineTranslation.toProto(pipeline, dataflowV1Components, true, false); if (LOG.isDebugEnabled()) { LOG.debug( @@ -1428,17 +1440,20 @@ public DataflowPipelineJob run(Pipeline pipeline) { // has been effectively rejected. The SDK should return // Error::Already_Exists to user in that case. int randomNum = RANDOM.nextInt(9000) + 1000; - String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm") - .withZone(DateTimeZone.UTC) - .print(DateTimeUtils.currentTimeMillis()) - + "_" - + randomNum; + String requestId = + DateTimeFormat.forPattern("YYYYMMddHHmmssmmm") + .withZone(DateTimeZone.UTC) + .print(DateTimeUtils.currentTimeMillis()) + + "_" + + randomNum; - JobSpecification jobSpecification = translator.translate( - pipeline, dataflowV1PipelineProto, dataflowV1Components, this, packages); + JobSpecification jobSpecification = + translator.translate( + pipeline, dataflowV1PipelineProto, dataflowV1Components, this, packages); if (!isNullOrEmpty(dataflowOptions.getDataflowWorkerJar()) && !useUnifiedWorker(options)) { - List experiments = firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); + List experiments = + firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); if (!experiments.contains("use_staged_dataflow_worker_jar")) { dataflowOptions.setExperiments( ImmutableList.builder() @@ -1485,20 +1500,24 @@ public DataflowPipelineJob run(Pipeline pipeline) { newJob.getEnvironment().setWorkerZone(options.getWorkerZone()); } - if (options.getFlexRSGoal() == DataflowPipelineOptions.FlexResourceSchedulingGoal.COST_OPTIMIZED) { + if (options.getFlexRSGoal() + == DataflowPipelineOptions.FlexResourceSchedulingGoal.COST_OPTIMIZED) { newJob.getEnvironment().setFlexResourceSchedulingGoal("FLEXRS_COST_OPTIMIZED"); - } else if (options.getFlexRSGoal() == DataflowPipelineOptions.FlexResourceSchedulingGoal.SPEED_OPTIMIZED) { + } else if (options.getFlexRSGoal() + == DataflowPipelineOptions.FlexResourceSchedulingGoal.SPEED_OPTIMIZED) { newJob.getEnvironment().setFlexResourceSchedulingGoal("FLEXRS_SPEED_OPTIMIZED"); } // Represent the minCpuPlatform pipeline option as an experiment, if not already // present. if (!isNullOrEmpty(dataflowOptions.getMinCpuPlatform())) { - List experiments = firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); + List experiments = + firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); - List minCpuFlags = experiments.stream() - .filter(p -> p.startsWith("min_cpu_platform")) - .collect(Collectors.toList()); + List minCpuFlags = + experiments.stream() + .filter(p -> p.startsWith("min_cpu_platform")) + .collect(Collectors.toList()); if (minCpuFlags.isEmpty()) { dataflowOptions.setExperiments( @@ -1559,8 +1578,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { if (hasExperiment(options, "upload_graph") && useUnifiedWorker(options)) { ArrayList experiments = new ArrayList<>(options.getExperiments()); - while (experiments.remove("upload_graph")) { - } + while (experiments.remove("upload_graph")) {} options.setExperiments(experiments); LOG.warn( "The upload_graph experiment was specified, but it does not apply " @@ -1571,7 +1589,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { // graph // will be downloaded from GCS by the service. if (hasExperiment(options, "upload_graph")) { - DataflowPackage stagedGraph = options.getStager().stageToFile(jobGraphBytes, DATAFLOW_GRAPH_FILE_NAME); + DataflowPackage stagedGraph = + options.getStager().stageToFile(jobGraphBytes, DATAFLOW_GRAPH_FILE_NAME); newJob.getSteps().clear(); newJob.setStepsLocation(stagedGraph.getLocation()); } @@ -1584,7 +1603,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { isNullOrEmpty(options.getDataflowJobFile()), "--dataflowJobFile and --templateLocation are mutually exclusive."); } - String fileLocation = firstNonNull(options.getTemplateLocation(), options.getDataflowJobFile()); + String fileLocation = + firstNonNull(options.getTemplateLocation(), options.getDataflowJobFile()); checkArgument( fileLocation.startsWith("/") || fileLocation.startsWith("gs://"), "Location must be local or on Cloud Storage, got %s.", @@ -1626,10 +1646,11 @@ public DataflowPipelineJob run(Pipeline pipeline) { String errorMessages = "Unexpected errors"; if (e.getDetails() != null) { if (jobGraphByteSize >= CREATE_JOB_REQUEST_LIMIT_BYTES) { - errorMessages = "The size of the serialized JSON representation of the pipeline " - + "exceeds the allowable limit. " - + "For more information, please see the documentation on job submission:\n" - + "https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#jobs"; + errorMessages = + "The size of the serialized JSON representation of the pipeline " + + "exceeds the allowable limit. " + + "For more information, please see the documentation on job submission:\n" + + "https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#jobs"; } else { errorMessages = e.getDetails().getMessage(); } @@ -1641,12 +1662,13 @@ public DataflowPipelineJob run(Pipeline pipeline) { // Use a raw client for post-launch monitoring, as status calls may fail // regularly and need not be retried automatically. - DataflowPipelineJob dataflowPipelineJob = new DataflowPipelineJob( - DataflowClient.create(options), - jobResult.getId(), - options, - jobSpecification != null ? jobSpecification.getStepNames() : Collections.emptyMap(), - portablePipelineProto); + DataflowPipelineJob dataflowPipelineJob = + new DataflowPipelineJob( + DataflowClient.create(options), + jobResult.getId(), + options, + jobSpecification != null ? jobSpecification.getStepNames() : Collections.emptyMap(), + portablePipelineProto); // If the service returned client request id, the SDK needs to compare it // with the original id generated in the request, if they are not the same @@ -1690,8 +1712,9 @@ public DataflowPipelineJob run(Pipeline pipeline) { private static void printWorkSpecJsonToFile(String fileLocation, Job job) throws IOException { String workSpecJson = DataflowPipelineTranslator.jobToString(job); ResourceId fileResource = FileSystems.matchNewResource(fileLocation, false /* isDirectory */); - try (OutputStreamWriter writer = new OutputStreamWriter( - Channels.newOutputStream(FileSystems.create(fileResource, MimeTypes.TEXT)), UTF_8)) { + try (OutputStreamWriter writer = + new OutputStreamWriter( + Channels.newOutputStream(FileSystems.create(fileResource, MimeTypes.TEXT)), UTF_8)) { // Not using PrintWriter as it swallows IOException. // Not using BufferedWriter as this invokes write() only once. writer.write(workSpecJson); @@ -1700,7 +1723,8 @@ private static void printWorkSpecJsonToFile(String fileLocation, Job job) throws private static EnvironmentInfo getEnvironmentInfoFromEnvironmentId( String environmentId, RunnerApi.Pipeline pipelineProto) { - RunnerApi.Environment environment = pipelineProto.getComponents().getEnvironmentsMap().get(environmentId); + RunnerApi.Environment environment = + pipelineProto.getComponents().getEnvironmentsMap().get(environmentId); if (!BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER) .equals(environment.getUrn())) { throw new RuntimeException( @@ -1744,41 +1768,37 @@ private static List getAllEnvironmentInfo(RunnerApi.Pipeline pi static void configureSdkHarnessContainerImages( DataflowPipelineOptions options, RunnerApi.Pipeline pipelineProto, Job newJob) { - List sdkContainerList = getAllEnvironmentInfo(pipelineProto).stream() - .map( - environmentInfo -> { - SdkHarnessContainerImage image = new SdkHarnessContainerImage(); - image.setEnvironmentId(environmentInfo.environmentId()); - image.setContainerImage(environmentInfo.containerUrl()); - if (!environmentInfo - .capabilities() - .contains( - BeamUrns.getUrn( - RunnerApi.StandardProtocols.Enum.MULTI_CORE_BUNDLE_PROCESSING))) { - image.setUseSingleCorePerContainer(true); - } - image.setCapabilities(environmentInfo.capabilities()); - return image; - }) - .collect(Collectors.toList()); + List sdkContainerList = + getAllEnvironmentInfo(pipelineProto).stream() + .map( + environmentInfo -> { + SdkHarnessContainerImage image = new SdkHarnessContainerImage(); + image.setEnvironmentId(environmentInfo.environmentId()); + image.setContainerImage(environmentInfo.containerUrl()); + if (!environmentInfo + .capabilities() + .contains( + BeamUrns.getUrn( + RunnerApi.StandardProtocols.Enum.MULTI_CORE_BUNDLE_PROCESSING))) { + image.setUseSingleCorePerContainer(true); + } + image.setCapabilities(environmentInfo.capabilities()); + return image; + }) + .collect(Collectors.toList()); for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) { workerPool.setSdkHarnessContainerImages(sdkContainerList); } } - /** - * Returns true if the specified experiment is enabled, handling null - * experiments. - */ + /** Returns true if the specified experiment is enabled, handling null experiments. */ public static boolean hasExperiment(DataflowPipelineDebugOptions options, String experiment) { - List experiments = firstNonNull(options.getExperiments(), Collections.emptyList()); + List experiments = + firstNonNull(options.getExperiments(), Collections.emptyList()); return experiments.contains(experiment); } - /** - * Helper to configure the Dataflow Job Environment based on the user's job - * options. - */ + /** Helper to configure the Dataflow Job Environment based on the user's job options. */ private static Map getEnvironmentVersion(DataflowPipelineOptions options) { DataflowRunnerInfo runnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); String majorVersion; @@ -1858,9 +1878,7 @@ public DataflowPipelineTranslator getTranslator() { return translator; } - /** - * Sets callbacks to invoke during execution see {@code DataflowRunnerHooks}. - */ + /** Sets callbacks to invoke during execution see {@code DataflowRunnerHooks}. */ public void setHooks(DataflowRunnerHooks hooks) { this.hooks = hooks; } @@ -1926,9 +1944,7 @@ public void visitPrimitiveTransform(Node node) { } } - /** - * Outputs a warning about PCollection views without deterministic key coders. - */ + /** Outputs a warning about PCollection views without deterministic key coders. */ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) { // We need to wait till this point to determine the names of the transforms // since only @@ -1939,8 +1955,7 @@ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pip pipeline.traverseTopologically( new PipelineVisitor.Defaults() { @Override - public void visitValue(PValue value, TransformHierarchy.Node producer) { - } + public void visitValue(PValue value, TransformHierarchy.Node producer) {} @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { @@ -1953,8 +1968,8 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { if (node.getTransform() instanceof View.AsMap || node.getTransform() instanceof View.AsMultimap) { - PCollection> input = (PCollection>) Iterables - .getOnlyElement(node.getInputs().values()); + PCollection> input = + (PCollection>) Iterables.getOnlyElement(node.getInputs().values()); KvCoder inputCoder = (KvCoder) input.getCoder(); try { inputCoder.getKeyCoder().verifyDeterministic(); @@ -1969,8 +1984,7 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { } @Override - public void leaveCompositeTransform(TransformHierarchy.Node node) { - } + public void leaveCompositeTransform(TransformHierarchy.Node node) {} }); LOG.warn( @@ -1983,8 +1997,7 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { } /** - * Returns true if the passed in {@link PCollection} needs to be materialized - * using an indexed + * Returns true if the passed in {@link PCollection} needs to be materialized using an indexed * format. */ boolean doesPCollectionRequireIndexedFormat(PCollection pcol) { @@ -1992,8 +2005,7 @@ boolean doesPCollectionRequireIndexedFormat(PCollection pcol) { } /** - * Marks the passed in {@link PCollection} as requiring to be materialized using - * an indexed + * Marks the passed in {@link PCollection} as requiring to be materialized using an indexed * format. */ void addPCollectionRequiringIndexedFormat(PCollection pcol) { @@ -2035,7 +2047,8 @@ void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) { // PubsubIO translations // ================================================================================ private static class StreamingPubsubIOReadOverrideFactory - implements PTransformOverrideFactory, PubsubUnboundedSource> { + implements PTransformOverrideFactory< + PBegin, PCollection, PubsubUnboundedSource> { @Override public PTransformReplacement> getReplacementTransform( @@ -2052,8 +2065,7 @@ public Map, ReplacementOutput> mapOutputs( } /** - * Suppress application of {@link PubsubUnboundedSource#expand} in streaming - * mode so that we can + * Suppress application of {@link PubsubUnboundedSource#expand} in streaming mode so that we can * instead defer to Windmill's implementation. */ private static class StreamingPubsubIORead @@ -2071,9 +2083,10 @@ public PubsubUnboundedSource getOverriddenTransform() { @Override public PCollection expand(PBegin input) { - Coder coder = transform.getNeedsMessageId() - ? new PubsubMessageWithAttributesAndMessageIdCoder() - : new PubsubMessageWithAttributesCoder(); + Coder coder = + transform.getNeedsMessageId() + ? new PubsubMessageWithAttributesAndMessageIdCoder() + : new PubsubMessageWithAttributesCoder(); return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED, coder); } @@ -2161,8 +2174,7 @@ public PubsubMessage apply(PubsubMessage input) { } /** - * Suppress application of {@link PubsubUnboundedSink#expand} in streaming mode - * so that we can + * Suppress application of {@link PubsubUnboundedSink#expand} in streaming mode so that we can * instead defer to Windmill's implementation. */ static class StreamingPubsubIOWrite extends PTransform, PDone> { @@ -2311,11 +2323,13 @@ public void translate(Impulse transform, TransformTranslator.TranslationContext } else { StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, "impulse"); - WindowedValues.FullWindowedValueCoder coder = WindowedValues.getFullCoder( - context.getOutput(transform).getCoder(), GlobalWindow.Coder.INSTANCE); + WindowedValues.FullWindowedValueCoder coder = + WindowedValues.getFullCoder( + context.getOutput(transform).getCoder(), GlobalWindow.Coder.INSTANCE); byte[] encodedImpulse; try { - encodedImpulse = encodeToByteArray(coder, WindowedValues.valueInGlobalWindow(new byte[0])); + encodedImpulse = + encodeToByteArray(coder, WindowedValues.valueInGlobalWindow(new byte[0])); } catch (Exception e) { throw new RuntimeException(e); } @@ -2347,13 +2361,10 @@ public Map, ReplacementOutput> mapOutputs( } /** - * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Unbounded - * Read.Unbounded} for + * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Unbounded Read.Unbounded} for * the Dataflow runner in streaming mode. * - *

- * In particular, if an UnboundedSource requires deduplication, then features of - * WindmillSink + *

In particular, if an UnboundedSource requires deduplication, then features of WindmillSink * are leveraged to do the deduplication. */ private static class StreamingUnboundedRead extends PTransform> { @@ -2443,7 +2454,8 @@ public PCollection expand(PCollection> input) { return input .apply( WithKeys.of( - (ValueWithRecordId value) -> Arrays.hashCode(value.getId()) % NUM_RESHARD_KEYS) + (ValueWithRecordId value) -> + Arrays.hashCode(value.getId()) % NUM_RESHARD_KEYS) .withKeyType(TypeDescriptors.integers())) // Reshuffle will dedup based on ids in ValueWithRecordId by passing the data // through @@ -2479,8 +2491,7 @@ public Map, ReplacementOutput> mapOutputs( } /** - * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded - * Read.Bounded} for the + * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded Read.Bounded} for the * Dataflow runner in streaming mode. */ private static class StreamingBoundedRead extends PTransform> { @@ -2501,8 +2512,7 @@ public final PCollection expand(PBegin input) { } /** - * A marker {@link DoFn} for writing the contents of a {@link PCollection} to a - * streaming {@link + * A marker {@link DoFn} for writing the contents of a {@link PCollection} to a streaming {@link * PCollectionView} backend implementation. */ public static class StreamingPCollectionViewWriterFn extends DoFn, T> { @@ -2592,12 +2602,19 @@ public Combine.GroupedValues getOriginalCombine() { } private static class PrimitiveCombineGroupedValuesOverrideFactory - implements - PTransformOverrideFactory>>, PCollection>, Combine.GroupedValues> { + implements PTransformOverrideFactory< + PCollection>>, + PCollection>, + Combine.GroupedValues> { @Override - public PTransformReplacement>>, PCollection>> getReplacementTransform( - AppliedPTransform>>, PCollection>, GroupedValues> transform) { + public PTransformReplacement>>, PCollection>> + getReplacementTransform( + AppliedPTransform< + PCollection>>, + PCollection>, + GroupedValues> + transform) { return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), new CombineGroupedValues<>( @@ -2638,8 +2655,10 @@ public Map, ReplacementOutput> mapOutputs( @VisibleForTesting static class StreamingShardedWriteFactory - implements - PTransformOverrideFactory, WriteFilesResult, WriteFiles> { + implements PTransformOverrideFactory< + PCollection, + WriteFilesResult, + WriteFiles> { // We pick 10 as a default, as it works well with the default number of workers // started @@ -2652,8 +2671,13 @@ static class StreamingShardedWriteFactory } @Override - public PTransformReplacement, WriteFilesResult> getReplacementTransform( - AppliedPTransform, WriteFilesResult, WriteFiles> transform) { + public PTransformReplacement, WriteFilesResult> + getReplacementTransform( + AppliedPTransform< + PCollection, + WriteFilesResult, + WriteFiles> + transform) { // By default, if numShards is not set WriteFiles will produce one file per // bundle. In // streaming, there are large numbers of small bundles, resulting in many tiny @@ -2675,9 +2699,11 @@ public PTransformReplacement, WriteFilesResult> } try { - List> sideInputs = WriteFilesTranslation.getDynamicDestinationSideInputs(transform); + List> sideInputs = + WriteFilesTranslation.getDynamicDestinationSideInputs(transform); FileBasedSink sink = WriteFilesTranslation.getSink(transform); - WriteFiles replacement = WriteFiles.to(sink).withSideInputs(sideInputs); + WriteFiles replacement = + WriteFiles.to(sink).withSideInputs(sideInputs); if (WriteFilesTranslation.isWindowedWrites(transform)) { replacement = replacement.withWindowedWrites(); } @@ -2729,8 +2755,7 @@ static String getDefaultContainerImageUrl(DataflowPipelineOptions options) { } /** - * Construct the default Dataflow container image name based on pipeline type - * and Java version. + * Construct the default Dataflow container image name based on pipeline type and Java version. */ static String getDefaultContainerImageNameForJob(DataflowPipelineOptions options) { Environments.JavaVersion javaVersion = Environments.getJavaVersion(); @@ -2744,8 +2769,7 @@ static String getDefaultContainerImageNameForJob(DataflowPipelineOptions options } /** - * Construct the default Dataflow container image name based on pipeline type - * and Java version. + * Construct the default Dataflow container image name based on pipeline type and Java version. */ static String getDefaultContainerVersion(DataflowPipelineOptions options) { DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); @@ -2796,8 +2820,7 @@ static void verifyStateSupportForWindowingStrategy(WindowingStrategy strategy) { } /** - * These are for dataflow-specific classes where we put fake stubs in the - * pipeline proto to pass + * These are for dataflow-specific classes where we put fake stubs in the pipeline proto to pass * validation. */ private static class DataflowPayloadTranslator @@ -2823,13 +2846,14 @@ public RunnerApi.FunctionSpec translate( } @SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) + "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) @AutoService(TransformPayloadTranslatorRegistrar.class) public static class DataflowTransformTranslator implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> getTransformPayloadTranslators() { + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { TransformPayloadTranslator dummyTranslator = new DataflowPayloadTranslator(); return ImmutableMap., TransformPayloadTranslator>builder() .put(CreateDataflowView.class, dummyTranslator)