diff --git a/CHANGES.md b/CHANGES.md index ce46b7087f8b..7a460962a33e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -67,6 +67,8 @@ * [Enrichment Transform](https://s.apache.org/enrichment-transform) along with GCP BigTable handler added to Python SDK ([#30001](https://github.com/apache/beam/pull/30001)). * Allow writing clustered and not time partitioned BigQuery tables (Java) ([#30094](https://github.com/apache/beam/pull/30094)). +* Merged sdks/java/fn-execution and runners/core-construction-java into the main SDK. These artifacts were never meant for users, but noting + that they no longer exist. These are steps to bring portability into the core SDK alongside all other core functionality. ## Breaking Changes diff --git a/build.gradle.kts b/build.gradle.kts index 7e4c6cdc71ab..67653e59c0d9 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -247,7 +247,6 @@ tasks.register("javaPreCommit") { dependsOn(":model:fn-execution:build") dependsOn(":model:job-management:build") dependsOn(":model:pipeline:build") - dependsOn(":runners:core-construction-java:build") dependsOn(":runners:core-java:build") dependsOn(":runners:direct-java:build") dependsOn(":runners:direct-java:needsRunnerTests") diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 0ca75c387c08..fe80b826c564 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2591,7 +2591,6 @@ class BeamModulePlugin implements Plugin { project.evaluationDependsOn(":sdks:python") project.evaluationDependsOn(config.expansionProjectPath) - project.evaluationDependsOn(":runners:core-construction-java") project.evaluationDependsOn(":sdks:java:extensions:python") // Setting up args to launch the expansion service @@ -2685,7 +2684,6 @@ class BeamModulePlugin implements Plugin { project.evaluationDependsOn(":sdks:python") project.evaluationDependsOn(":sdks:java:testing:expansion-service") - project.evaluationDependsOn(":runners:core-construction-java") project.evaluationDependsOn(":sdks:java:extensions:python") project.evaluationDependsOn(":sdks:go:test") @@ -2750,11 +2748,9 @@ class BeamModulePlugin implements Plugin { systemProperty "expansionPort", port systemProperty "semiPersistDir", config.semiPersistDir classpath = config.classpath + project.files( - project.project(":runners:core-construction-java").sourceSets.test.runtimeClasspath, project.project(":sdks:java:extensions:python").sourceSets.test.runtimeClasspath ) testClassesDirs = project.files( - project.project(":runners:core-construction-java").sourceSets.test.output.classesDirs, project.project(":sdks:java:extensions:python").sourceSets.test.output.classesDirs ) maxParallelForks config.numParallelTests @@ -2861,7 +2857,6 @@ class BeamModulePlugin implements Plugin { def config = it ? it as TransformServiceConfiguration : new TransformServiceConfiguration() project.evaluationDependsOn(":sdks:python") - project.evaluationDependsOn(":runners:core-construction-java") project.evaluationDependsOn(":sdks:java:extensions:python") project.evaluationDependsOn(":sdks:java:transform-service:app") diff --git a/playground/backend/internal/preparers/java_preparers.go b/playground/backend/internal/preparers/java_preparers.go index d65e07c76819..eee5d323ad99 100644 --- a/playground/backend/internal/preparers/java_preparers.go +++ b/playground/backend/internal/preparers/java_preparers.go @@ -38,7 +38,7 @@ const ( newLinePattern = "\n" javaPublicClassNamePattern = "public class (.*?) [{|implements(.*)]" pipelineNamePattern = `Pipeline\s([A-z|0-9_]*)\s=\sPipeline\.create` - graphSavePattern = "String dotString = org.apache.beam.runners.core.construction.renderer.PipelineDotRenderer.toDotString(%s);\n" + + graphSavePattern = "String dotString = org.apache.beam.sdk.util.construction.renderer.PipelineDotRenderer.toDotString(%s);\n" + " try (java.io.PrintWriter out = new java.io.PrintWriter(\"graph.dot\")) {\n " + " out.println(dotString);\n " + " } catch (java.io.FileNotFoundException e) {\n" + diff --git a/playground/backend/internal/preparers/java_preparers_test.go b/playground/backend/internal/preparers/java_preparers_test.go index 35b1d2d6e1e4..91f457c7112d 100644 --- a/playground/backend/internal/preparers/java_preparers_test.go +++ b/playground/backend/internal/preparers/java_preparers_test.go @@ -130,9 +130,9 @@ func TestGetJavaPreparers(t *testing.T) { } func Test_findPipelineObjectName(t *testing.T) { - code := "package org.apache.beam.examples;\n\n/*\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements. See the NOTICE file\n * distributed with this work for additional information\n * regarding copyright ownership. The ASF licenses this file\n * to you under the Apache License, Version 2.0 (the\n * \"License\"); you may not use this file except in compliance\n * with the License. You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// beam-playground:\n// name: Branching\n// description: Task from katas to branch out the numbers to two different transforms, one transform\n// is multiplying each number by 5 and the other transform is multiplying each number by 10.\n// multifile: false\n// categories:\n// - Branching\n// - Core Transforms\n\nimport static org.apache.beam.sdk.values.TypeDescriptors.integers;\n\nimport org.apache.beam.sdk.Pipeline;\nimport org.apache.beam.sdk.options.PipelineOptions;\nimport org.apache.beam.sdk.options.PipelineOptionsFactory;\nimport org.apache.beam.sdk.transforms.Create;\nimport org.apache.beam.sdk.transforms.MapElements;\nimport org.apache.beam.sdk.values.PCollection;\nimport org.apache.beam.runners.core.construction.renderer.PipelineDotRenderer;\n\npublic class Task {\n\n\n\n public static void main(String[] args) {\n PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();\n Pipeline pipeline = Pipeline.create(options);\n\n PCollection numbers =\n pipeline.apply(Create.of(1, 2, 3, 4, 5));\n\n PCollection mult5Results = applyMultiply5Transform(numbers);\n PCollection mult10Results = applyMultiply10Transform(numbers);\n\n mult5Results.apply(\"Log multiply 5\", Log.ofElements(\"Multiplied by 5: \"));\n mult10Results.apply(\"Log multiply 10\", Log.ofElements(\"Multiplied by 10: \"));\n\n String dotString = PipelineDotRenderer.toDotString(pipeline);\n System.out.println(dotString);\n pipeline.run();\n\n }\n\n static PCollection applyMultiply5Transform(PCollection input) {\n return input.apply(\"Multiply by 5\", MapElements.into(integers()).via(num -> num * 5));\n }\n\n static PCollection applyMultiply10Transform(PCollection input) {\n return input.apply(\"Multiply by 10\", MapElements.into(integers()).via(num -> num * 10));\n }\n\n}\n" + code := "package org.apache.beam.examples;\n\n/*\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements. See the NOTICE file\n * distributed with this work for additional information\n * regarding copyright ownership. The ASF licenses this file\n * to you under the Apache License, Version 2.0 (the\n * \"License\"); you may not use this file except in compliance\n * with the License. You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// beam-playground:\n// name: Branching\n// description: Task from katas to branch out the numbers to two different transforms, one transform\n// is multiplying each number by 5 and the other transform is multiplying each number by 10.\n// multifile: false\n// categories:\n// - Branching\n// - Core Transforms\n\nimport static org.apache.beam.sdk.values.TypeDescriptors.integers;\n\nimport org.apache.beam.sdk.Pipeline;\nimport org.apache.beam.sdk.options.PipelineOptions;\nimport org.apache.beam.sdk.options.PipelineOptionsFactory;\nimport org.apache.beam.sdk.transforms.Create;\nimport org.apache.beam.sdk.transforms.MapElements;\nimport org.apache.beam.sdk.values.PCollection;\nimport org.apache.beam.sdk.util.construction.renderer.construction.PipelineDotRenderer;\n\npublic class Task {\n\n\n\n public static void main(String[] args) {\n PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();\n Pipeline pipeline = Pipeline.create(options);\n\n PCollection numbers =\n pipeline.apply(Create.of(1, 2, 3, 4, 5));\n\n PCollection mult5Results = applyMultiply5Transform(numbers);\n PCollection mult10Results = applyMultiply10Transform(numbers);\n\n mult5Results.apply(\"Log multiply 5\", Log.ofElements(\"Multiplied by 5: \"));\n mult10Results.apply(\"Log multiply 10\", Log.ofElements(\"Multiplied by 10: \"));\n\n String dotString = PipelineDotRenderer.toDotString(pipeline);\n System.out.println(dotString);\n pipeline.run();\n\n }\n\n static PCollection applyMultiply5Transform(PCollection input) {\n return input.apply(\"Multiply by 5\", MapElements.into(integers()).via(num -> num * 5));\n }\n\n static PCollection applyMultiply10Transform(PCollection input) {\n return input.apply(\"Multiply by 10\", MapElements.into(integers()).via(num -> num * 10));\n }\n\n}\n" lc := createTempFileWithCode(code) - codeWithoutPipeline := "package org.apache.beam.examples;\n\n/*\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements. See the NOTICE file\n * distributed with this work for additional information\n * regarding copyright ownership. The ASF licenses this file\n * to you under the Apache License, Version 2.0 (the\n * \"License\"); you may not use this file except in compliance\n * with the License. You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// beam-playground:\n// name: Branching\n// description: Task from katas to branch out the numbers to two different transforms, one transform\n// is multiplying each number by 5 and the other transform is multiplying each number by 10.\n// multifile: false\n// categories:\n// - Branching\n// - Core Transforms\n\nimport static org.apache.beam.sdk.values.TypeDescriptors.integers;\n\nimport org.apache.beam.sdk.Pipeline;\nimport org.apache.beam.sdk.options.PipelineOptions;\nimport org.apache.beam.sdk.options.PipelineOptionsFactory;\nimport org.apache.beam.sdk.transforms.Create;\nimport org.apache.beam.sdk.transforms.MapElements;\nimport org.apache.beam.sdk.values.PCollection;\nimport org.apache.beam.runners.core.construction.renderer.PipelineDotRenderer;\n\npublic class Task {\n\n\n\n public static void main(String[] args) {\n PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();\n PCollection numbers =\n pipeline.apply(Create.of(1, 2, 3, 4, 5));\n\n PCollection mult5Results = applyMultiply5Transform(numbers);\n PCollection mult10Results = applyMultiply10Transform(numbers);\n\n mult5Results.apply(\"Log multiply 5\", Log.ofElements(\"Multiplied by 5: \"));\n mult10Results.apply(\"Log multiply 10\", Log.ofElements(\"Multiplied by 10: \"));\n\n String dotString = PipelineDotRenderer.toDotString(pipeline);\n System.out.println(dotString);\n pipeline.run();\n\n }\n\n static PCollection applyMultiply5Transform(PCollection input) {\n return input.apply(\"Multiply by 5\", MapElements.into(integers()).via(num -> num * 5));\n }\n\n static PCollection applyMultiply10Transform(PCollection input) {\n return input.apply(\"Multiply by 10\", MapElements.into(integers()).via(num -> num * 10));\n }\n\n}\n" + codeWithoutPipeline := "package org.apache.beam.examples;\n\n/*\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements. See the NOTICE file\n * distributed with this work for additional information\n * regarding copyright ownership. The ASF licenses this file\n * to you under the Apache License, Version 2.0 (the\n * \"License\"); you may not use this file except in compliance\n * with the License. You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// beam-playground:\n// name: Branching\n// description: Task from katas to branch out the numbers to two different transforms, one transform\n// is multiplying each number by 5 and the other transform is multiplying each number by 10.\n// multifile: false\n// categories:\n// - Branching\n// - Core Transforms\n\nimport static org.apache.beam.sdk.values.TypeDescriptors.integers;\n\nimport org.apache.beam.sdk.Pipeline;\nimport org.apache.beam.sdk.options.PipelineOptions;\nimport org.apache.beam.sdk.options.PipelineOptionsFactory;\nimport org.apache.beam.sdk.transforms.Create;\nimport org.apache.beam.sdk.transforms.MapElements;\nimport org.apache.beam.sdk.values.PCollection;\nimport org.apache.beam.sdk.util.construction.renderer.construction.PipelineDotRenderer;\n\npublic class Task {\n\n\n\n public static void main(String[] args) {\n PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();\n PCollection numbers =\n pipeline.apply(Create.of(1, 2, 3, 4, 5));\n\n PCollection mult5Results = applyMultiply5Transform(numbers);\n PCollection mult10Results = applyMultiply10Transform(numbers);\n\n mult5Results.apply(\"Log multiply 5\", Log.ofElements(\"Multiplied by 5: \"));\n mult10Results.apply(\"Log multiply 10\", Log.ofElements(\"Multiplied by 10: \"));\n\n String dotString = PipelineDotRenderer.toDotString(pipeline);\n System.out.println(dotString);\n pipeline.run();\n\n }\n\n static PCollection applyMultiply5Transform(PCollection input) {\n return input.apply(\"Multiply by 5\", MapElements.into(integers()).via(num -> num * 5));\n }\n\n static PCollection applyMultiply10Transform(PCollection input) {\n return input.apply(\"Multiply by 10\", MapElements.into(integers()).via(num -> num * 10));\n }\n\n}\n" lcWithoutPipeline := createTempFileWithCode(codeWithoutPipeline) path, _ := os.Getwd() defer os.RemoveAll(filepath.Join(path, "temp")) diff --git a/runners/core-construction-java/build.gradle b/runners/core-construction-java/build.gradle deleted file mode 100644 index 2c6fb402ae1d..000000000000 --- a/runners/core-construction-java/build.gradle +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * License); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an AS IS BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -plugins { id 'org.apache.beam.module' } -applyJavaNature( - classesTriggerCheckerBugs: [ - 'WindowIntoTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'FlattenTranslator': 'https://github.com/typetools/checker-framework/issues/3791', - 'WriteFilesTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'CreatePCollectionViewTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'ReshuffleTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'TestStreamTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'ImpulseTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'CombineTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'SplittableParDo': 'https://github.com/typetools/checker-framework/issues/3791', - 'ReadTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'GroupByKeyTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'ParDoTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - 'GroupIntoBatchesTranslation': 'https://github.com/typetools/checker-framework/issues/3791', - ], - automaticModuleName: 'org.apache.beam.runners.core.construction', -) - -description = "Apache Beam :: Runners :: Core Construction Java" -ext.summary = """Beam Runners Core provides utilities to aid runner authors interact with a Pipeline -prior to execution.""" - - -// Exclude tests that need a runner -test { - systemProperty "beamUseDummyRunner", "true" - useJUnit { - excludeCategories "org.apache.beam.sdk.testing.NeedsRunner" - } -} - -dependencies { - implementation project(path: ":model:pipeline", configuration: "shadow") - implementation project(path: ":model:job-management", configuration: "shadow") - implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(path: ":sdks:java:transform-service:launcher") - implementation library.java.vendored_grpc_1_60_1 - implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.classgraph - implementation library.java.jackson_core - implementation library.java.jackson_databind - implementation library.java.joda_time - implementation library.java.slf4j_api - implementation library.java.jackson_annotations - compileOnly library.java.error_prone_annotations - // Avro 1.8 leaks an older version of parameter that conflicts in runtime with the dependencies - // of some runners so we need to fix it to a more recent but still compatible version. - runtimeOnly "com.thoughtworks.paranamer:paranamer:2.8" - testImplementation library.java.junit - testImplementation library.java.mockito_core - testImplementation library.java.jackson_annotations - testImplementation library.java.jackson_dataformat_yaml - testImplementation project(path: ":model:fn-execution", configuration: "shadow") - testImplementation project(path: ":sdks:java:core", configuration: "testRuntimeMigration") - testImplementation project(path: ":sdks:java:extensions:avro") - testRuntimeOnly library.java.slf4j_jdk14 -} diff --git a/runners/core-java/build.gradle b/runners/core-java/build.gradle index 499cf06fcd6e..1e0899eacaf5 100644 --- a/runners/core-java/build.gradle +++ b/runners/core-java/build.gradle @@ -42,7 +42,6 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") - implementation project(":runners:core-construction-java") implementation library.java.vendored_guava_32_1_2_jre implementation library.java.joda_time implementation library.java.vendored_grpc_1_60_1 diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java index 5a0a85433d53..0759487565b0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.core; import java.util.Collection; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.transforms.DoFn; @@ -26,6 +25,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java index 855c8b2a786b..caa5565541a4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java @@ -17,19 +17,12 @@ */ package org.apache.beam.runners.core; -import static org.apache.beam.runners.core.construction.SplittableParDo.SPLITTABLE_PROCESS_URN; +import static org.apache.beam.sdk.util.construction.SplittableParDo.SPLITTABLE_PROCESS_URN; import com.google.auto.service.AutoService; import java.util.List; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation.RawPTransform; -import org.apache.beam.runners.core.construction.ReplacementOutputs; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDo.ProcessKeyedElements; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -53,6 +46,13 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation.RawPTransform; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDo.ProcessKeyedElements; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java index d6b2f94738cf..44d1b4f53071 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java @@ -19,7 +19,7 @@ import static org.apache.beam.model.pipeline.v1.MetricsApi.labelProps; import static org.apache.beam.model.pipeline.v1.MetricsApi.monitoringInfoSpec; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricName.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricName.java index adf3ad84f9e8..2be3aaec2987 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricName.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricName.java @@ -40,7 +40,6 @@ public class MonitoringInfoMetricName extends MetricName { private MonitoringInfoMetricName(String urn, Map labels) { checkArgument(!Strings.isNullOrEmpty(urn), "MonitoringInfoMetricName urn must be non-empty"); checkArgument(labels != null, "MonitoringInfoMetricName labels must be non-null"); - // TODO(ajamato): Move SimpleMonitoringInfoBuilder to :runners:core-construction-java // and ensure all necessary labels are set for the specific URN. this.urn = urn; this.labels = ImmutableMap.copyOf(labels); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index 56e373608697..d047a9133930 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -32,7 +32,6 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachineRunner; @@ -59,6 +58,7 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/direct-java/build.gradle b/runners/direct-java/build.gradle index fc2c549a370b..b7f74dc3e538 100644 --- a/runners/direct-java/build.gradle +++ b/runners/direct-java/build.gradle @@ -22,11 +22,11 @@ plugins { id 'org.apache.beam.module' } // Shade away runner execution utilities till because this causes ServiceLoader conflicts with // TransformPayloadTranslatorRegistrar amongst other runners. This only happens in the DirectRunner // because it is likely to appear on the classpath of another runner. -def dependOnProjects = [":runners:core-construction-java", +def dependOnProjects = [ ":runners:core-java", ":runners:local-java", ":runners:java-fn-execution", - ":sdks:java:extensions:avro" + ":sdks:java:core", ] applyJavaNature( @@ -50,7 +50,6 @@ description = "Apache Beam :: Runners :: Direct Java" * the following projects are evaluated before we evaluate this project. This is because * we are attempting to reference the "sourceSets.test.output" directly. */ -evaluationDependsOn(":runners:core-construction-java") evaluationDependsOn(":runners:core-java") evaluationDependsOn(":sdks:java:core") evaluationDependsOn(":examples:java") @@ -67,7 +66,6 @@ dependencies { dependOnProjects.each { implementation project(it) } - shadow project(path: ":sdks:java:core", configuration: "shadow") shadow library.java.vendored_grpc_1_60_1 shadow library.java.joda_time shadow library.java.slf4j_api @@ -79,12 +77,10 @@ dependencies { shadowTest project(path: ":sdks:java:core", configuration: "shadowTest") shadowTest project(path: ":runners:core-java", configuration: "testRuntimeMigration") shadowTest library.java.mockito_core - needsRunner project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") needsRunner project(path: ":runners:core-java", configuration: "testRuntimeMigration") needsRunner project(path: ":sdks:java:core", configuration: "shadowTest") needsRunner project(path: project.path, configuration: "shadow") needsRunner project(path: project.path, configuration: "shadowTest") - validatesRunner project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") validatesRunner project(path: ":runners:core-java", configuration: "testRuntimeMigration") validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest") validatesRunner project(path: project.path, configuration: "shadow") @@ -130,7 +126,6 @@ task needsRunnerTests(type: Test) { classpath = configurations.needsRunner // NOTE: We only add to the test class dirs to ensure that the direct runner // tests (i.e., those from this subproject) get scanned. - testClassesDirs += files(project(":runners:core-construction-java").sourceSets.test.output.classesDirs) testClassesDirs += files(project(":runners:core-java").sourceSets.test.output.classesDirs) testClassesDirs += files(project(":sdks:java:core").sourceSets.test.output.classesDirs) useJUnit { @@ -170,7 +165,6 @@ task validatesRunner(type: Test) { classpath = configurations.validatesRunner // NOTE: We only add to the test class dirs to ensure that the direct runner // tests (i.e., those from this subproject) get scanned. - testClassesDirs += files(project(":runners:core-construction-java").sourceSets.test.output.classesDirs) testClassesDirs += files(project(":runners:core-java").sourceSets.test.output.classesDirs) testClassesDirs += files(project(":sdks:java:core").sourceSets.test.output.classesDirs) useJUnit { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index 97eedb06bc3f..bbb1ae842444 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java @@ -26,8 +26,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.beam.runners.core.construction.ReadTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo.PrimitiveBoundedRead; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.options.PipelineOptions; @@ -36,6 +34,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.ReadTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo.PrimitiveBoundedRead; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java index 2bab79b1cb3d..d3f5c1a27018 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java @@ -19,9 +19,9 @@ import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java index a5066cbf52fb..00f246089493 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java @@ -21,9 +21,9 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index eb12ff0e3daa..adaae0b5494d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -24,7 +24,6 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.direct.DirectWriteViewVisitor.WriteView; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; @@ -32,6 +31,7 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java index a8f881433a3c..3078574b3bdb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java @@ -21,12 +21,12 @@ import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItemCoder; -import org.apache.beam.runners.core.construction.ForwardingPTransform; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.ForwardingPTransform; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java index b12b21a95a90..3400591f3664 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java @@ -17,12 +17,12 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index 8c0faedd857a..e27b6a618c3e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -28,9 +28,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory; import org.apache.beam.sdk.Pipeline; @@ -44,6 +41,9 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.common.ReflectHelpers; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java index ded87d72e6d0..ee71e33ff93f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java @@ -23,12 +23,12 @@ import java.util.Collections; import java.util.Map; import java.util.concurrent.Callable; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.core.metrics.MetricUpdates; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutableGraph.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutableGraph.java index 7a6dbd1c7c68..0cbb74d4e289 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutableGraph.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutableGraph.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.direct; import java.util.Collection; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; /** * The interface that enables querying of a graph of independently executable stages and the inputs diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 04b57ffd8514..c12a2ff44f8e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -30,7 +30,6 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; @@ -46,6 +45,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java index e88145c5bb7e..9a7b7f7a9b5d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java @@ -27,9 +27,6 @@ import java.util.Objects; import javax.annotation.Nonnull; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation.RawPTransform; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.runners.local.StructuralKey; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -50,6 +47,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation.RawPTransform; +import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index 7a11343396c4..22729d91720a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.ParDoTranslation; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.ParDoEvaluator.DoFnRunnerFactory; import org.apache.beam.runners.local.StructuralKey; @@ -28,6 +27,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index b16bb63883a5..dd9ef1cc3984 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -27,10 +27,6 @@ import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.runners.core.KeyedWorkItems; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.ReplacementOutputs; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -48,6 +44,10 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java index 20b984f19411..e88bbbb34787 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.direct; import java.util.Collection; -import org.apache.beam.runners.core.construction.ReadTranslation; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.ReadTranslation; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java index 560da714a4c6..7ac36e2fa8b8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java @@ -17,18 +17,18 @@ */ package org.apache.beam.runners.direct; -import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN; import static org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory.DIRECT_TEST_STREAM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import java.util.Collection; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 683163917726..85d0371a451e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -23,8 +23,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.ReplacementOutputs; -import org.apache.beam.runners.core.construction.TestStreamTranslation; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.testing.TestStream; @@ -36,6 +34,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; +import org.apache.beam.sdk.util.construction.TestStreamTranslation; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java index f28f75af5416..544222c8253f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java @@ -17,19 +17,19 @@ */ package org.apache.beam.runners.direct; -import static org.apache.beam.runners.core.construction.PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.CREATE_VIEW_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.READ_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.SplittableParDo.SPLITTABLE_PROCESS_URN; import static org.apache.beam.runners.direct.DirectGroupByKey.DIRECT_GABW_URN; import static org.apache.beam.runners.direct.DirectGroupByKey.DIRECT_GBKO_URN; import static org.apache.beam.runners.direct.DirectWriteViewVisitor.DIRECT_WRITE_VIEW_URN; import static org.apache.beam.runners.direct.MultiStepCombine.DIRECT_MERGE_ACCUMULATORS_EXTRACT_OUTPUT_URN; import static org.apache.beam.runners.direct.ParDoMultiOverrideFactory.DIRECT_STATEFUL_PAR_DO_URN; import static org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory.DIRECT_TEST_STREAM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.CREATE_VIEW_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.READ_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.SplittableParDo.SPLITTABLE_PROCESS_URN; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; @@ -38,14 +38,14 @@ import java.util.Collection; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory.DirectTestStream; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java index 65b49de8debf..a9ad4ecce7b6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java @@ -19,10 +19,10 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.runners.core.construction.SplittableParDo.PrimitiveUnboundedRead; import org.apache.beam.runners.local.StructuralKey; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.SplittableParDo.PrimitiveUnboundedRead; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.LoadingCache; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java index f362e7bdf1e9..bfcb0a591824 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java @@ -25,8 +25,6 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.runners.core.construction.ReadTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.direct.UnboundedReadDeduplicator.NeverDeduplicator; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; @@ -38,6 +36,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.ReadTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java index 5f0d5071acb3..1f5292b4e7c9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java @@ -18,13 +18,13 @@ package org.apache.beam.runners.direct; import java.util.Collection; -import org.apache.beam.runners.core.construction.WindowIntoTranslation; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.WindowIntoTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java index 1b77e8bc309f..e976180ae190 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java @@ -21,9 +21,6 @@ import java.io.Serializable; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.ReplacementOutputs; -import org.apache.beam.runners.core.construction.WriteFilesTranslation; import org.apache.beam.sdk.io.WriteFiles; import org.apache.beam.sdk.io.WriteFilesResult; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -35,6 +32,9 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; +import org.apache.beam.sdk.util.construction.WriteFilesTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index 1683ea8e142c..121c4325427a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -36,7 +36,6 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.CountDownLatch; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.direct.BoundedReadEvaluatorFactory.BoundedSourceShard; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; @@ -55,6 +54,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index 2d7b6308816a..c16fc3c410ae 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -33,7 +33,6 @@ import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.core.ReadyCheckingSideInputReader; import org.apache.beam.runners.core.StateInternals; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; @@ -62,6 +61,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index a5ca66dd5901..28c317b669f1 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -47,7 +47,6 @@ import java.util.NoSuchElementException; import java.util.Set; import java.util.concurrent.Executors; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.direct.UnboundedReadDeduplicator.NeverDeduplicator; import org.apache.beam.runners.direct.UnboundedReadEvaluatorFactory.UnboundedSourceShard; import org.apache.beam.sdk.Pipeline; @@ -74,6 +73,7 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.VarInt; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index 55303b3530d4..d8bfcb3f533a 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -147,7 +147,6 @@ dependencies { implementation library.java.vendored_guava_32_1_2_jre implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":runners:core-java") - implementation project(":runners:core-construction-java") implementation project(":runners:java-fn-execution") implementation project(":runners:java-job-service") implementation project(":sdks:java:extensions:google-cloud-platform-core") diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java index b0fba383d857..7861ec5371b9 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java @@ -21,8 +21,6 @@ import java.util.List; import java.util.Map; import org.apache.beam.runners.core.Concatenate; -import org.apache.beam.runners.core.construction.CreatePCollectionViewTranslation; -import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; @@ -30,6 +28,8 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PCollectionViews; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java index df3732770cbf..b415c9b10559 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java @@ -17,11 +17,11 @@ */ package org.apache.beam.runners.flink; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java index e0bef5e5d73a..76a711a65cd2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java @@ -17,11 +17,11 @@ */ package org.apache.beam.runners.flink; -import static org.apache.beam.runners.core.construction.ExecutableStageTranslation.generateNameFromStagePayload; import static org.apache.beam.runners.flink.translation.utils.FlinkPortableRunnerUtils.requiresTimeSortedInput; import static org.apache.beam.runners.fnexecution.translation.PipelineTranslatorUtils.createOutputMap; import static org.apache.beam.runners.fnexecution.translation.PipelineTranslatorUtils.getWindowingStrategy; import static org.apache.beam.runners.fnexecution.translation.PipelineTranslatorUtils.instantiateCoder; +import static org.apache.beam.sdk.util.construction.ExecutableStageTranslation.generateNameFromStagePayload; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import com.google.auto.service.AutoService; @@ -39,15 +39,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId; import org.apache.beam.runners.core.Concatenate; -import org.apache.beam.runners.core.construction.NativeTransforms; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.flink.translation.functions.FlinkExecutableStageContextFactory; import org.apache.beam.runners.flink.translation.functions.FlinkExecutableStageFunction; import org.apache.beam.runners.flink.translation.functions.FlinkExecutableStagePruningFunction; @@ -69,6 +60,15 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.util.construction.NativeTransforms; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 91ca8e8fe107..8f37178d24f1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -27,10 +27,6 @@ import java.util.Map; import java.util.Map.Entry; import org.apache.beam.runners.core.Concatenate; -import org.apache.beam.runners.core.construction.CreatePCollectionViewTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.ReadTranslation; import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction; import org.apache.beam.runners.flink.translation.functions.FlinkExplodeWindowsFunction; @@ -68,6 +64,10 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.ReadTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java index 7c1bc87ced03..52e298918ea8 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java @@ -26,9 +26,9 @@ import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java index 07c340fea1a4..bbb3cc67ca4a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java @@ -17,17 +17,17 @@ */ package org.apache.beam.runners.flink; -import static org.apache.beam.runners.core.construction.resources.PipelineResources.detectClassPathResourcesToStage; +import static org.apache.beam.sdk.util.construction.resources.PipelineResources.detectClassPathResourcesToStage; import java.util.UUID; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.runners.jobsubmission.JobInvoker; import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator; import org.apache.beam.runners.jobsubmission.PortablePipelineRunner; import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java index 12ed3603264a..029eff25a825 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java @@ -20,11 +20,11 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import java.util.Map; -import org.apache.beam.runners.core.construction.resources.PipelineResources; import org.apache.beam.runners.core.metrics.MetricsPusher; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.metrics.MetricsOptions; +import org.apache.beam.sdk.util.construction.resources.PipelineResources; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.RuntimeExecutionMode; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java index 269f21030210..eb21887a3e12 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.flink; -import static org.apache.beam.runners.core.construction.resources.PipelineResources.detectClassPathResourcesToStage; import static org.apache.beam.runners.fnexecution.translation.PipelineTranslatorUtils.hasUnboundedPCollections; +import static org.apache.beam.sdk.util.construction.resources.PipelineResources.detectClassPathResourcesToStage; import java.util.List; import java.util.Map; @@ -26,13 +26,6 @@ import org.apache.beam.model.jobmanagement.v1.ArtifactApi; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; -import org.apache.beam.runners.core.construction.graph.ProtoOverrides; -import org.apache.beam.runners.core.construction.graph.SplittableParDoExpander; -import org.apache.beam.runners.core.construction.graph.TrivialNativeTransformExpander; import org.apache.beam.runners.core.metrics.MetricsPusher; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.jobsubmission.PortablePipelineJarUtils; @@ -44,6 +37,13 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.SdkHarnessOptions; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; +import org.apache.beam.sdk.util.construction.graph.ProtoOverrides; +import org.apache.beam.sdk.util.construction.graph.SplittableParDoExpander; +import org.apache.beam.sdk.util.construction.graph.TrivialNativeTransformExpander; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.flink.api.common.JobExecutionResult; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java index 295ad2f98a2f..6945bead80bb 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java @@ -21,8 +21,6 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.graph.ProjectionPushdownOptimizer; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineRunner; @@ -33,6 +31,8 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.graph.ProjectionPushdownOptimizer; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.runtime.jobgraph.JobGraph; import org.slf4j.Logger; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java index ffc7da97cd02..3ed00a3c5ef2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.flink; -import static org.apache.beam.runners.core.construction.PTransformTranslation.WRITE_FILES_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.WRITE_FILES_TRANSFORM_URN; import java.io.IOException; import java.nio.ByteBuffer; @@ -28,11 +28,6 @@ import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ReplacementOutputs; -import org.apache.beam.runners.core.construction.UnconsumedReads; -import org.apache.beam.runners.core.construction.WriteFilesTranslation; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkKeyUtils; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; @@ -49,6 +44,11 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; +import org.apache.beam.sdk.util.construction.UnconsumedReads; +import org.apache.beam.sdk.util.construction.WriteFilesTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java index 7077ea4c3d1a..58d7d76fe548 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java @@ -18,10 +18,10 @@ package org.apache.beam.runners.flink; import static java.lang.String.format; -import static org.apache.beam.runners.core.construction.ExecutableStageTranslation.generateNameFromStagePayload; import static org.apache.beam.runners.fnexecution.translation.PipelineTranslatorUtils.createOutputMap; import static org.apache.beam.runners.fnexecution.translation.PipelineTranslatorUtils.getWindowingStrategy; import static org.apache.beam.runners.fnexecution.translation.PipelineTranslatorUtils.instantiateCoder; +import static org.apache.beam.sdk.util.construction.ExecutableStageTranslation.generateNameFromStagePayload; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -40,18 +40,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.SystemReduceFn; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.NativeTransforms; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ReadTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.RunnerPCollectionView; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.TestStreamTranslation; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.flink.translation.functions.FlinkExecutableStageContextFactory; import org.apache.beam.runners.flink.translation.functions.ImpulseSourceFunction; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; @@ -90,6 +78,18 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.NativeTransforms; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ReadTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.RunnerPCollectionView; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.TestStreamTranslation; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PCollectionViews; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index f2b8d64a0893..b44e157f4a1b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.flink; import static java.lang.String.format; -import static org.apache.beam.runners.core.construction.SplittableParDo.SPLITTABLE_PROCESS_URN; +import static org.apache.beam.sdk.util.construction.SplittableParDo.SPLITTABLE_PROCESS_URN; import com.google.auto.service.AutoService; import java.io.IOException; @@ -32,12 +32,6 @@ import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; import org.apache.beam.runners.core.SystemReduceFn; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.ReadTranslation; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; @@ -84,6 +78,12 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.ReadTranslation; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java index 0a89bd18172b..10ba64a77148 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java @@ -22,13 +22,13 @@ import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java index 69ad58253b8e..46458eccb83c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java @@ -21,12 +21,12 @@ import java.util.List; import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDoNaiveBounded; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** {@link PTransform} overrides for Flink runner. */ diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index f437daf86e71..fb1724c72b29 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -23,7 +23,6 @@ import java.util.Optional; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; @@ -37,6 +36,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java index 7693fb94490d..40a104fdfb21 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java @@ -33,10 +33,6 @@ import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateTags; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.fnexecution.control.BundleCheckpointHandler; @@ -62,6 +58,10 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; @@ -76,7 +76,7 @@ /** * Flink operator that passes its input DataSet through an SDK-executed {@link - * org.apache.beam.runners.core.construction.graph.ExecutableStage}. + * org.apache.beam.sdk.util.construction.graph.ExecutableStage}. * *

The output of this operation is a multiplexed DataSet whose elements are tagged with a union * coder. The coder's tags are determined by the output coder map. The resulting data set should be diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java index 467358c154db..30556d75ff07 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java @@ -17,11 +17,11 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java index b1b95c6b58e4..2359f45e531e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -18,13 +18,13 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java index 787b1729cb72..81e61cd587aa 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java @@ -17,11 +17,11 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index 0839812c27b1..5295c9949e56 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -18,13 +18,13 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index 13998693de7f..f9c80203fba1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -18,13 +18,13 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java index 16eac410f278..e6169dbd2142 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java @@ -33,7 +33,6 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; @@ -50,6 +49,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index 9f4da7cea8e5..828d066cff16 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java @@ -19,9 +19,9 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index 2195ecdf1ab7..bf3ba7bc368f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -19,13 +19,13 @@ import java.io.EOFException; import java.io.IOException; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkPortableRunnerUtils.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkPortableRunnerUtils.java index 359a0c60eb7e..a31b0ac9adaf 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkPortableRunnerUtils.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkPortableRunnerUtils.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.flink.translation.utils; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; /** diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java index 9cf71d4e78e9..23e36fb653ff 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java @@ -20,12 +20,12 @@ import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index a1b8bced7a1d..558de8dc7c0f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.List; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil; @@ -30,6 +29,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.RichInputFormat; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 63f5ede00242..a77f5e95112d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -54,7 +54,6 @@ import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; @@ -84,6 +83,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.NoopLock; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index 1bdb57b8f989..0c27c25b93db 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -58,11 +58,6 @@ import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternalsFactory; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.UserStateReference; import org.apache.beam.runners.flink.translation.functions.FlinkExecutableStageContextFactory; import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; import org.apache.beam.runners.flink.translation.utils.Locker; @@ -102,6 +97,11 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.UserStateReference; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java index 204247b1d836..82ecb418480c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToByteBufferKeySelector.java @@ -18,10 +18,10 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import java.nio.ByteBuffer; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfByteBufferKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfByteBufferKeySelector.java index 8c6f10abf448..8ff382adc938 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfByteBufferKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfByteBufferKeySelector.java @@ -18,10 +18,10 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import java.nio.ByteBuffer; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java index 64ea6ca26d4d..1cf08f22afcb 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java @@ -19,10 +19,10 @@ import java.nio.ByteBuffer; import org.apache.beam.runners.core.KeyedWorkItem; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java index d43723964844..16060cbbebe4 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java @@ -18,10 +18,10 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; import java.nio.ByteBuffer; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index 961d31a75370..fc369b898a07 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -21,8 +21,6 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil; @@ -39,6 +37,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueWithRecordId; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSource.java index 0b9fdd9dcd7c..8a84aace803e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/FlinkSource.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map; import java.util.function.Function; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.utils.SerdeUtils; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.bounded.FlinkBoundedSource; @@ -29,6 +28,7 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.unbounded.FlinkUnboundedSource; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java index ab9a6cc03cd5..c20df4431ab8 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.bounded; import javax.annotation.Nullable; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java index 8ef2edfa606e..3fddb39da3f8 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.unbounded; import javax.annotation.Nullable; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceReader; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java index 312f8812ff1c..7bafeef8e861 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java @@ -26,13 +26,13 @@ import java.util.function.Function; import java.util.function.Supplier; import org.apache.beam.runners.core.DoFnRunner; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; import org.apache.beam.runners.flink.translation.utils.Locker; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.state.ListState; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java index da876823a446..5d5fe285b30a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java @@ -27,7 +27,6 @@ import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateTag; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.ListCoder; @@ -48,6 +47,7 @@ import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CombineContextFactory; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java index 205270c22332..fe32fbd94c6f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -32,7 +32,6 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.StateTag; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkKeyUtils; import org.apache.beam.sdk.coders.Coder; @@ -58,6 +57,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CombineContextFactory; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java index 676e35d4bc0f..9d898ed53a89 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java @@ -43,9 +43,6 @@ import java.util.Collection; import java.util.List; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.resources.PipelineResources; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.TextIO; @@ -56,6 +53,9 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.resources.PipelineResources; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.flink.api.java.ExecutionEnvironment; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java index da8c560690a6..b834d253c4eb 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java @@ -24,7 +24,6 @@ import java.util.Collections; import java.util.HashMap; import org.apache.beam.repackaged.core.org.apache.commons.lang3.SerializationUtils; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -37,6 +36,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.ExecutionConfig; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java index 91d6aca3507c..f402d9bda689 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java @@ -27,8 +27,6 @@ import java.util.concurrent.Executors; import org.apache.beam.model.jobmanagement.v1.JobApi; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -55,6 +53,8 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.util.FilePatternMatchingShardedFile; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java index fef5df402171..6ba30a8e020a 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java @@ -30,8 +30,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.GenerateSequence; @@ -49,6 +47,8 @@ import org.apache.beam.sdk.transforms.InferableFunction; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java index d5d34b59214b..733bf536634c 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java @@ -27,8 +27,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.bounded.FlinkBoundedSource; import org.apache.beam.sdk.Pipeline; @@ -42,6 +40,8 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.resourcehints.ResourceHints; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.PValues; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java index b502e1129ee2..22a9ce4f39ab 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java @@ -25,11 +25,11 @@ import java.util.Collection; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.core.construction.resources.PipelineResources; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.construction.resources.PipelineResources; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java index 4da94e5a13a0..2334113a2570 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java @@ -23,8 +23,6 @@ import java.util.concurrent.TimeUnit; import org.apache.beam.model.jobmanagement.v1.JobApi.JobState; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -40,6 +38,8 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableStateExecutionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableStateExecutionTest.java index aa40ec0e9f3a..d3eade4944d4 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableStateExecutionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableStateExecutionTest.java @@ -23,8 +23,6 @@ import java.util.concurrent.TimeUnit; import org.apache.beam.model.jobmanagement.v1.JobApi.JobState; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.VarIntCoder; @@ -39,6 +37,8 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java index 594bf55d73de..ffa764a03d86 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java @@ -29,8 +29,6 @@ import java.util.concurrent.TimeUnit; import org.apache.beam.model.jobmanagement.v1.JobApi.JobState; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -49,6 +47,8 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourcePortableTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourcePortableTest.java index 193f291ab9fd..bf07fe4cafc2 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourcePortableTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourcePortableTest.java @@ -30,10 +30,6 @@ import java.util.stream.Collectors; import org.apache.beam.model.jobmanagement.v1.JobApi.JobState; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; @@ -48,6 +44,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java index 8bd31f0bb9d9..a7183df67b44 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java @@ -24,8 +24,6 @@ import java.util.Collection; import java.util.HashSet; import java.util.Set; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.TestCountingSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper; import org.apache.beam.sdk.io.BoundedSource; @@ -33,6 +31,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.api.TimeCharacteristic; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java index 10e20a6d47d3..74ceaac48231 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java @@ -20,9 +20,9 @@ import java.util.Collections; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateInternalsTest; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index a2d6f5027abb..570076749468 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -28,7 +28,6 @@ import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals; import org.apache.beam.sdk.coders.CoderException; @@ -37,6 +36,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.typeutils.GenericTypeInfo; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java index d00c3d821e86..c7f922fe070e 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.flink.translation.functions; -import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; import static org.hamcrest.Matchers.is; import static org.mockito.Matchers.any; import static org.mockito.Mockito.doThrow; @@ -32,7 +32,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.fnexecution.control.BundleCheckpointHandler; import org.apache.beam.runners.fnexecution.control.BundleFinalizationHandler; @@ -50,6 +49,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializerTest.java index 3c4e43bd339f..6dae02bacaa2 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializerTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializerTest.java @@ -24,11 +24,11 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.flink.api.common.typeutils.ComparatorTestBase; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.junit.Test; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index 17cc16cc76e0..34a80116460d 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -50,7 +50,6 @@ import org.apache.beam.runners.core.StatefulDoFnRunner; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; @@ -84,6 +83,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java index 0ad9ef694a20..4c67a33a533f 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; -import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; import static org.apache.beam.runners.flink.translation.wrappers.streaming.StreamRecordStripper.stripStreamRecordFromWindowedValue; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; @@ -61,8 +61,6 @@ import org.apache.beam.runners.core.StateTags; import org.apache.beam.runners.core.StatefulDoFnRunner; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate; import org.apache.beam.runners.flink.streaming.FlinkStateInternalsTest; @@ -98,6 +96,8 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.NoopLock; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java index 8fab1bc6c167..9a4b864a8cd3 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java @@ -32,7 +32,6 @@ import java.nio.ByteBuffer; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.SystemReduceFn; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.MultiOutputOutputManagerFactory; import org.apache.beam.sdk.coders.Coder; @@ -47,6 +46,7 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java index 0634695b1c46..92fcc58a21bf 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java @@ -39,7 +39,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.stream.LongStream; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.flink.metrics.MetricGroupWrapper; @@ -52,6 +51,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java index 58f6fd8484bf..d5a09b06bd5c 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java @@ -23,12 +23,12 @@ import java.util.Collections; import java.util.List; import org.apache.beam.runners.core.DoFnRunner; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.flink.api.common.state.ListState; import org.apache.flink.runtime.state.OperatorStateBackend; diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 34297ecba510..27b1ca83a9b9 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -92,7 +92,6 @@ dependencies { // io-kafka is only used in PTransform override so it is optional provided project(":sdks:java:io:kafka") implementation project(":sdks:java:io:google-cloud-platform") - implementation project(":runners:core-construction-java") implementation library.java.avro implementation library.java.bigdataoss_util implementation library.java.commons_codec @@ -124,7 +123,6 @@ dependencies { testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(path: ":sdks:java:extensions:avro") testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core", configuration: "testRuntimeMigration") - testImplementation project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:extensions:python", configuration: "testRuntimeMigration") testImplementation library.java.google_cloud_dataflow_java_proto_library_all testImplementation library.java.jackson_dataformat_yaml diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java index 838b00f02c51..0478a2c5f0a1 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java @@ -20,8 +20,6 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.runners.dataflow.BatchViewOverrides.GroupByKeyAndSortValuesOnly; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.coders.Coder; @@ -41,6 +39,8 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 903298458184..da3b3e7b6163 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -51,12 +51,6 @@ import java.util.Map.Entry; import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.TransformInputs; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.dataflow.BatchViewOverrides.GroupByKeyAndSortValuesOnly; import org.apache.beam.runners.dataflow.DataflowRunner.CombineGroupedValues; import org.apache.beam.runners.dataflow.PrimitiveParDoSingleFactory.ParDoSingle; @@ -98,6 +92,12 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.TransformInputs; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; 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 8ea366d4a997..a454945c3d01 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 @@ -18,12 +18,12 @@ package org.apache.beam.runners.dataflow; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.beam.runners.core.construction.resources.PipelineResources.detectClassPathResourcesToStage; import static org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.ENABLE_CUSTOM_PUBSUB_SINK; import static org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.ENABLE_CUSTOM_PUBSUB_SOURCE; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; +import static org.apache.beam.sdk.util.construction.resources.PipelineResources.detectClassPathResourcesToStage; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects.firstNonNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; @@ -59,26 +59,6 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.BeamUrns; -import org.apache.beam.runners.core.construction.DeduplicatedFlattenFactory; -import org.apache.beam.runners.core.construction.EmptyFlattenAsCreateFactory; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.External; -import org.apache.beam.runners.core.construction.ExternalTranslationOptions; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.ReplacementOutputs; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource; -import org.apache.beam.runners.core.construction.UnconsumedReads; -import org.apache.beam.runners.core.construction.WriteFilesTranslation; -import org.apache.beam.runners.core.construction.graph.ProjectionPushdownOptimizer; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification; import org.apache.beam.runners.dataflow.StreamingViewOverrides.StreamingCreatePCollectionViewFactory; import org.apache.beam.runners.dataflow.TransformTranslator.StepTranslationContext; @@ -153,6 +133,26 @@ import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ReflectHelpers; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.DeduplicatedFlattenFactory; +import org.apache.beam.sdk.util.construction.EmptyFlattenAsCreateFactory; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.External; +import org.apache.beam.sdk.util.construction.ExternalTranslationOptions; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDoNaiveBounded; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource; +import org.apache.beam.sdk.util.construction.UnconsumedReads; +import org.apache.beam.sdk.util.construction.WriteFilesTranslation; +import org.apache.beam.sdk.util.construction.graph.ProjectionPushdownOptimizer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java index b7ba74c2cb8a..8d300fc8bb6c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java @@ -21,8 +21,6 @@ import java.util.List; import java.util.Map; import java.util.UUID; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -37,6 +35,8 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.ShardedKey; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java index 140858d88c04..3141c49bde0f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java @@ -17,12 +17,12 @@ */ package org.apache.beam.runners.dataflow; -import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.ParDoTranslation.registerCoderOrThrow; -import static org.apache.beam.runners.core.construction.ParDoTranslation.translateTimeDomain; -import static org.apache.beam.runners.core.construction.ParDoTranslation.translateTimerFamilySpec; import static org.apache.beam.sdk.transforms.reflect.DoFnSignatures.getStateSpecOrThrow; import static org.apache.beam.sdk.transforms.reflect.DoFnSignatures.getTimerSpecOrThrow; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.ParDoTranslation.registerCoderOrThrow; +import static org.apache.beam.sdk.util.construction.ParDoTranslation.translateTimeDomain; +import static org.apache.beam.sdk.util.construction.ParDoTranslation.translateTimerFamilySpec; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import com.google.auto.service.AutoService; @@ -34,15 +34,6 @@ import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.DisplayData; -import org.apache.beam.runners.core.construction.ForwardingPTransform; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation.ParDoLikeTimerFamilySpecs; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -61,6 +52,15 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature.TimerDeclaration; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.construction.ForwardingPTransform; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation.ParDoLikeTimerFamilySpecs; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PCollectionViews; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReadTranslator.java index 821d149f620d..7ee60a6dd546 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReadTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReadTranslator.java @@ -24,11 +24,11 @@ import com.google.api.services.dataflow.model.SourceMetadata; import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.dataflow.internal.CustomSources; import org.apache.beam.runners.dataflow.util.PropertyNames; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; /** Translator for the {@code Read} {@code PTransform} for the Dataflow back-end. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java index 403e61857e79..65bdeb11b888 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java @@ -18,14 +18,14 @@ package org.apache.beam.runners.dataflow; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.DoFn.RequiresStableInput; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reshuffle; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java index cf4d6f436120..53a94c4855b8 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java @@ -17,8 +17,6 @@ */ package org.apache.beam.runners.dataflow; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.DoFn; @@ -31,6 +29,8 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentityWindowFn; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java index 4105b29d0b90..a52b1bb9833f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java @@ -18,14 +18,14 @@ package org.apache.beam.runners.dataflow; import java.util.Map; -import org.apache.beam.runners.core.construction.ForwardingPTransform; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.ReplacementOutputs; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.construction.ForwardingPTransform; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/StreamingViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/StreamingViewOverrides.java index 633c0ec08c99..fa69d2c2a496 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/StreamingViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/StreamingViewOverrides.java @@ -19,8 +19,6 @@ import java.util.ArrayList; import java.util.List; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.runners.dataflow.DataflowRunner.StreamingPCollectionViewWriterFn; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -32,6 +30,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; +import org.apache.beam.sdk.util.construction.PTransformReplacements; +import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PCollectionViews; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java index bb91c99875fa..c4f79500fd27 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.OutputReference; import org.apache.beam.sdk.Pipeline; @@ -27,6 +26,7 @@ import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/AvroCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/AvroCoderCloudObjectTranslator.java index 9f5913f051b5..e8d0eaac35fe 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/AvroCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/AvroCoderCloudObjectTranslator.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.dataflow.util; import org.apache.avro.Schema; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.avro.io.AvroDatumFactory; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; +import org.apache.beam.sdk.util.construction.SdkComponents; /** A {@link CloudObjectTranslator} for {@link AvroCoder}. */ @SuppressWarnings({ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslator.java index 3638eaf68678..8abb784b1272 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslator.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.util; -import org.apache.beam.runners.core.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.SdkComponents; /** * A translator that takes an object and creates a {@link CloudObject} which can be converted back diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java index d158b8d5a7a5..62351739228f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java @@ -23,7 +23,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CustomCoder; @@ -45,6 +44,7 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java index 1376f21cd15d..208b46141f03 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java @@ -23,8 +23,6 @@ import java.util.Map; import java.util.ServiceLoader; import java.util.Set; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CustomCoder; @@ -36,6 +34,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index df6a3255f5b6..3e9ca0d600b0 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -43,7 +43,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.sdk.extensions.gcp.storage.GcsCreateOptions; import org.apache.beam.sdk.extensions.gcp.util.BackOffAdapter; import org.apache.beam.sdk.io.FileSystems; @@ -52,6 +51,7 @@ import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.MoreFutures; +import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.HashCode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.Hashing; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java index 115c7883e77e..926c14936ede 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java @@ -19,10 +19,10 @@ import java.io.IOException; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; /** Translator for row coders. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java index f611b95358dc..a1b92f1ba436 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java @@ -21,7 +21,6 @@ import java.util.UUID; import javax.annotation.Nullable; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaCoder; @@ -30,6 +29,7 @@ import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SerializableCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SerializableCoderCloudObjectTranslator.java index be902caf1f42..45c13043075f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SerializableCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SerializableCoderCloudObjectTranslator.java @@ -20,8 +20,8 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.io.Serializable; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.util.construction.SdkComponents; /** A {@link CloudObjectTranslator} for {@link SerializableCoder}. */ @SuppressWarnings({ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 37e50a2d0ebb..06a700bcd4b4 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -64,11 +64,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.DockerPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions; @@ -120,6 +115,11 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; 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 51af766511fc..df6c6e3be454 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 @@ -93,15 +93,6 @@ import org.apache.beam.model.expansion.v1.ExpansionApi; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.BeamUrns; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.ExpansionServiceClient; -import org.apache.beam.runners.core.construction.ExpansionServiceClientFactory; -import org.apache.beam.runners.core.construction.External; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.runners.dataflow.DataflowRunner.StreamingShardedWriteFactory; import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; @@ -164,6 +155,15 @@ import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.ShardedKey; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.ExpansionServiceClient; +import org.apache.beam.sdk.util.construction.ExpansionServiceClientFactory; +import org.apache.beam.sdk.util.construction.External; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValues; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java index f49a288ac70f..68c0260a9bdd 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java @@ -34,7 +34,6 @@ import java.util.List; import java.util.Objects; import java.util.Set; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -66,6 +65,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/GCSUploadMain.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/GCSUploadMain.java index e940d40f0a8e..1cb69e90cbfe 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/GCSUploadMain.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/GCSUploadMain.java @@ -21,10 +21,10 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.HashCode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.Hashing; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Files; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java index 5364b8d52b42..f9f68f4e7409 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java @@ -64,7 +64,6 @@ import java.util.List; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.dataflow.util.PackageUtil.PackageAttributes; import org.apache.beam.runners.dataflow.util.PackageUtil.StagedFile; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; @@ -81,6 +80,7 @@ import org.apache.beam.sdk.util.FastNanoClockAndSleeper; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.ZipFiles; +import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle index 067b80a1e56c..124c11026dfc 100644 --- a/runners/google-cloud-dataflow-java/worker/build.gradle +++ b/runners/google-cloud-dataflow-java/worker/build.gradle @@ -114,7 +114,6 @@ applyJavaNature( relocate("org.apache.beam.model.fnexecution.v1", getWorkerRelocatedPath("org.apache.beam.model.fnexecution.v1")) dependencies { - include(project(":runners:core-construction-java")) include(project(":runners:core-java")) } relocate("org.apache.beam.runners.core", getWorkerRelocatedPath("org.apache.beam.runners.core")) @@ -187,7 +186,6 @@ dependencies { implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(":runners:core-construction-java") implementation project(":runners:core-java") implementation project(":runners:java-fn-execution") implementation project(path: ":runners:google-cloud-dataflow-java:worker:windmill", configuration: "shadow") diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java index df692d83b3f4..72338ba0c2ad 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java @@ -31,8 +31,6 @@ import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.runners.core.StateInternalsFactory; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.dataflow.util.CloudObject; import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.runners.dataflow.util.PropertyNames; @@ -53,6 +51,8 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/RunnerHarnessCoderCloudObjectTranslatorRegistrar.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/RunnerHarnessCoderCloudObjectTranslatorRegistrar.java index cb90a320ceb8..f07c6bbd7962 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/RunnerHarnessCoderCloudObjectTranslatorRegistrar.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/RunnerHarnessCoderCloudObjectTranslatorRegistrar.java @@ -22,7 +22,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecordCoder; import org.apache.beam.runners.dataflow.util.CloudObject; import org.apache.beam.runners.dataflow.util.CloudObjectTranslator; @@ -37,6 +36,7 @@ import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowViaWindowSetFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowViaWindowSetFn.java index dd7d04f0013d..bfe51b31b116 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowViaWindowSetFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowViaWindowSetFn.java @@ -26,13 +26,13 @@ import org.apache.beam.runners.core.StateInternalsFactory; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.SystemReduceFn; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.runners.dataflow.worker.util.BatchGroupAlsoByWindowFn; import org.apache.beam.runners.dataflow.worker.util.StreamingGroupAlsoByWindowFn; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java index 08515a1fb33e..aaa6cd6d6d36 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java @@ -25,9 +25,9 @@ import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.dataflow.worker.status.DebugCapture.Capturable; import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor; +import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.servlet.ServletHandler; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java index b9d9cf2901ac..67995cdb0ded 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java @@ -29,12 +29,12 @@ import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java index 84b391055df9..f784deb60dd2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java @@ -48,7 +48,6 @@ import javax.management.MalformedObjectNameException; import javax.management.ObjectName; import javax.management.ReflectionException; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; @@ -57,6 +56,7 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.SdkHarnessOptions; +import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TimerOrElement.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TimerOrElement.java index f54957ff0700..051ad2c659d4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TimerOrElement.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TimerOrElement.java @@ -25,7 +25,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.runners.dataflow.util.CloudObject; import org.apache.beam.runners.dataflow.util.CloudObjectTranslator; import org.apache.beam.runners.dataflow.util.CloudObjects; @@ -34,6 +33,7 @@ import org.apache.beam.runners.dataflow.util.Structs; import org.apache.beam.runners.dataflow.worker.WindmillKeyedWorkItem.FakeKeyedWorkItemCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 1035fada0ff9..abd2cbbac6ef 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -87,9 +87,6 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.dataflow.internal.CustomSources; import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; @@ -160,6 +157,9 @@ import org.apache.beam.sdk.util.VarInt; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.ValueWithRecordId; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index 65cb937c7f06..66c100757426 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -74,10 +74,6 @@ import java.util.logging.Level; import java.util.logging.LogManager; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.core.metrics.ExecutionStateSampler; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator; import org.apache.beam.runners.dataflow.DataflowRunner; @@ -115,6 +111,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java index b8f916efdfe7..d24a57dc4eee 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java @@ -34,7 +34,7 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.runners.core.construction.Environments; +import org.apache.beam.sdk.util.construction.Environments; import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Rule; diff --git a/runners/java-fn-execution/build.gradle b/runners/java-fn-execution/build.gradle index 5e19e4cba102..732f3526f129 100644 --- a/runners/java-fn-execution/build.gradle +++ b/runners/java-fn-execution/build.gradle @@ -24,19 +24,16 @@ description = "Apache Beam :: Runners :: Java Fn Execution" dependencies { implementation library.java.vendored_guava_32_1_2_jre - implementation project(":runners:core-construction-java") implementation project(":runners:core-java") compileOnly project(":sdks:java:harness") implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(":runners:core-construction-java") implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api implementation project(path: ":model:job-management", configuration: "shadow") implementation library.java.joda_time testImplementation project(":sdks:java:harness") - testImplementation project(":runners:core-construction-java") testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(path: ":runners:core-java", configuration: "testRuntimeMigration") testImplementation library.java.junit diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java index 0fa4082b4861..7f0bbe54288b 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java @@ -25,11 +25,11 @@ import org.apache.beam.model.jobmanagement.v1.ArtifactApi; import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.ArtifactResolver; -import org.apache.beam.runners.core.construction.BeamUrns; -import org.apache.beam.runners.core.construction.DefaultArtifactResolver; import org.apache.beam.sdk.fn.server.FnService; import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.util.construction.ArtifactResolver; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.DefaultArtifactResolver; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultExecutableStageContext.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultExecutableStageContext.java index ffe6e031a22e..7d3271a59b2b 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultExecutableStageContext.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultExecutableStageContext.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.fnexecution.control; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; /** Implementation of a {@link ExecutableStageContext}. */ public class DefaultExecutableStageContext implements ExecutableStageContext, AutoCloseable { diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java index 019028f5b936..da5da6c9d603 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java @@ -35,11 +35,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardEnvironments; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardRunnerProtocols; -import org.apache.beam.runners.core.construction.BeamUrns; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.ExecutableProcessBundleDescriptor; import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.TimerSpec; @@ -70,6 +65,11 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.NoopLock; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ExecutableStageContext.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ExecutableStageContext.java index 662a2a7093cc..e6712e8fa852 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ExecutableStageContext.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ExecutableStageContext.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.fnexecution.control; import java.io.Serializable; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; /** The context required in order to execute {@link ExecutableStage stages}. */ public interface ExecutableStageContext extends AutoCloseable { diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactory.java index 6f459331fbc0..eee60d266568 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactory.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.fnexecution.control; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; /** * A factory that has all job-scoped information, and can be combined with stage-scoped information diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java index c7e0cd884c4e..6e43c91e1d49 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.fnexecution.control; -import static org.apache.beam.runners.core.construction.SyntheticComponents.uniqueId; +import static org.apache.beam.sdk.util.construction.SyntheticComponents.uniqueId; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; @@ -37,14 +37,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; -import org.apache.beam.runners.core.construction.graph.SideInputReference; -import org.apache.beam.runners.core.construction.graph.TimerReference; -import org.apache.beam.runners.core.construction.graph.UserStateReference; import org.apache.beam.runners.fnexecution.data.RemoteInputDestination; import org.apache.beam.runners.fnexecution.wire.ByteStringCoder; import org.apache.beam.runners.fnexecution.wire.LengthPrefixUnknownCoders; @@ -57,6 +49,14 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.graph.SideInputReference; +import org.apache.beam.sdk.util.construction.graph.TimerReference; +import org.apache.beam.sdk.util.construction.graph.UserStateReference; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactory.java index c18180dfe3ab..54fccbceadab 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactory.java @@ -24,13 +24,13 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.sdk.function.ThrowingFunction; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java index 0a85c4adeb22..afe15e3cd0f2 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.fnexecution.control; import java.util.Map; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; /** diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java index c9faf8508aa3..682c45e30795 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java @@ -42,7 +42,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitRequest.DesiredSplit; import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitResponse; import org.apache.beam.model.pipeline.v1.Endpoints; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.TimerSpec; import org.apache.beam.runners.fnexecution.data.FnDataService; import org.apache.beam.runners.fnexecution.data.RemoteInputDestination; @@ -59,6 +58,7 @@ import org.apache.beam.sdk.fn.data.LogicalEndpoint; import org.apache.beam.sdk.fn.data.TimerEndpoint; import org.apache.beam.sdk.util.MoreFutures; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.slf4j.Logger; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactory.java index 3a71bcfae72f..9ca69fe622be 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactory.java @@ -23,8 +23,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.ExecutableProcessBundleDescriptor; import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.TimerSpec; import org.apache.beam.runners.fnexecution.data.GrpcDataService; @@ -36,6 +34,8 @@ import org.apache.beam.sdk.fn.IdGenerator; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.fn.server.GrpcFnServer; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java index 32803de86cf5..1dec1d4339d8 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java @@ -21,8 +21,8 @@ /** * A bundle factory scoped to a particular {@link - * org.apache.beam.runners.core.construction.graph.ExecutableStage}, which has all of the resources - * it needs to provide new {@link RemoteBundle RemoteBundles}. + * org.apache.beam.sdk.util.construction.graph.ExecutableStage}, which has all of the resources it + * needs to provide new {@link RemoteBundle RemoteBundles}. * *

Closing a StageBundleFactory signals that the stage has completed and any resources bound to * its lifetime can be cleaned up. diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactory.java index bbc75d94839f..d99a733a4eb4 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactory.java @@ -26,11 +26,11 @@ import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.TimerSpec; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java index a1713c69d920..241012a65c47 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java @@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.BeamUrns; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.control.ControlClientPool; import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService; @@ -40,6 +39,7 @@ import org.apache.beam.sdk.options.ManualDockerEnvironmentOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.RemoteEnvironmentOptions; +import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java index 2a3db0b4e051..72fa991c1f73 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java @@ -30,7 +30,6 @@ import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardRunnerProtocols; -import org.apache.beam.runners.core.construction.BeamUrns; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.control.ControlClientPool; import org.apache.beam.runners.fnexecution.control.ControlClientPool.Source; @@ -45,6 +44,7 @@ import org.apache.beam.sdk.fn.server.ServerFactory; import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java index 461ccceb5a52..3edc30f5122a 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java @@ -24,7 +24,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnExternalWorkerPoolGrpc; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.BeamUrns; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.control.ControlClientPool; import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService; @@ -35,6 +34,7 @@ import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.fn.server.ServerFactory; +import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.slf4j.Logger; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java index fe026e5e5659..17b85e97c423 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java @@ -21,7 +21,6 @@ import java.util.concurrent.TimeoutException; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.BeamUrns; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.control.ControlClientPool; import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService; @@ -32,6 +31,7 @@ import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.RemoteEnvironmentOptions; +import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.slf4j.Logger; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java index dba4af3bc346..b26d3a643ef1 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java @@ -25,9 +25,6 @@ import java.util.List; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.SideInputReference; import org.apache.beam.runners.fnexecution.state.StateRequestHandler; import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.IterableSideInputHandler; import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler; @@ -36,6 +33,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.SideInputReference; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java index fbd3230310f8..e6eb5bfaaae5 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java @@ -30,10 +30,6 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode; import org.apache.beam.runners.fnexecution.control.TimerReceiverFactory; import org.apache.beam.runners.fnexecution.wire.WireCoders; import org.apache.beam.sdk.coders.Coder; @@ -41,6 +37,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/StreamingSideInputHandlerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/StreamingSideInputHandlerFactory.java index 3b649759099d..463ab98feb94 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/StreamingSideInputHandlerFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/StreamingSideInputHandlerFactory.java @@ -25,8 +25,6 @@ import java.util.HashMap; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.SideInputReference; import org.apache.beam.runners.fnexecution.state.StateRequestHandler; import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.IterableSideInputHandler; import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler; @@ -34,6 +32,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.SideInputReference; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java index 1e0d74d50ade..bde97f0b9d19 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java @@ -22,9 +22,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Coder; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; -import org.apache.beam.runners.core.construction.ModelCoders; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.LengthPrefixCoder; +import org.apache.beam.sdk.util.construction.ModelCoders; /** Utilities for replacing or wrapping unknown coders with {@link LengthPrefixCoder}. */ public class LengthPrefixUnknownCoders { diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java index de328d5a9642..f0dc2f65a0ea 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java @@ -17,19 +17,19 @@ */ package org.apache.beam.runners.fnexecution.wire; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.io.IOException; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.SyntheticComponents; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.SyntheticComponents; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; /** Helpers to construct coders for gRPC port reads and writes. */ public class WireCoders { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java index 799644f66b15..63b5385916db 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java @@ -47,9 +47,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.data.GrpcDataService; import org.apache.beam.runners.fnexecution.environment.EnvironmentFactory; @@ -69,6 +66,9 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java index 108412a9dafb..21d7550c38b9 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java @@ -26,18 +26,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.ModelCoderRegistrar; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.FusedPipeline; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.ProtoOverrides; -import org.apache.beam.runners.core.construction.graph.SplittableParDoExpander; -import org.apache.beam.runners.core.construction.graph.TimerReference; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -51,12 +39,22 @@ import org.apache.beam.sdk.state.TimerSpec; import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.ProcessContext; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.ModelCoderRegistrar; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.FusedPipeline; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.ProtoOverrides; +import org.apache.beam.sdk.util.construction.graph.SplittableParDoExpander; +import org.apache.beam.sdk.util.construction.graph.TimerReference; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java index fff089b8c96c..9bdd089c0cdc 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java @@ -70,14 +70,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitResponse.ChannelSplit; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.FusedPipeline; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; -import org.apache.beam.runners.core.construction.graph.ProtoOverrides; -import org.apache.beam.runners.core.construction.graph.SplittableParDoExpander; import org.apache.beam.runners.core.metrics.DistributionData; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns; @@ -138,6 +130,14 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.FusedPipeline; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.graph.ProtoOverrides; +import org.apache.beam.sdk.util.construction.graph.SplittableParDoExpander; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -1840,23 +1840,21 @@ public void onWindowExpiration( RemoteOutputReceiver.of( (Coder>) remoteOutputCoder.getValue(), outputContents::add)); } - Map, Collection>> + Map, Collection>> timerValues = new HashMap<>(); - Map< - KV, - RemoteOutputReceiver>> + Map, RemoteOutputReceiver>> timerReceivers = new HashMap<>(); for (Map.Entry> transformTimerSpecs : descriptor.getTimerSpecs().entrySet()) { for (ProcessBundleDescriptors.TimerSpec timerSpec : transformTimerSpecs.getValue().values()) { KV key = KV.of(timerSpec.transformId(), timerSpec.timerId()); - List> outputContents = + List> outputContents = Collections.synchronizedList(new ArrayList<>()); timerValues.put(key, outputContents); timerReceivers.put( key, RemoteOutputReceiver.of( - (Coder>) timerSpec.coder(), + (Coder>) timerSpec.coder(), outputContents::add)); } } @@ -2245,9 +2243,9 @@ private KV byteValueOf(String key, long value) throws CoderExcep return KV.of(key, CoderUtils.encodeToByteArray(BigEndianLongCoder.of(), value)); } - private org.apache.beam.runners.core.construction.Timer timerForTest( + private org.apache.beam.sdk.util.construction.Timer timerForTest( String key, long fireTimestamp, long holdTimestamp) { - return org.apache.beam.runners.core.construction.Timer.of( + return org.apache.beam.sdk.util.construction.Timer.of( key, "", Collections.singletonList(GlobalWindow.INSTANCE), diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java index 97f8eeac2f73..774bdf27cef0 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java @@ -60,8 +60,6 @@ import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.fnexecution.EmbeddedSdkHarness; import org.apache.beam.runners.fnexecution.control.SdkHarnessClient.BundleProcessor; import org.apache.beam.runners.fnexecution.control.SdkHarnessClient.BundleProcessor.ActiveBundle; @@ -86,6 +84,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow.Coder; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java index 61090e9ffdf3..8796f7097499 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java @@ -33,10 +33,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; import org.apache.beam.runners.fnexecution.data.GrpcDataService; import org.apache.beam.runners.fnexecution.environment.EnvironmentFactory; import org.apache.beam.runners.fnexecution.environment.RemoteEnvironment; @@ -48,6 +44,10 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.After; import org.junit.Before; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java index 16d32e46f010..a77d1a2392cb 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java @@ -33,7 +33,6 @@ import java.util.List; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.control.ControlClientPool; import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService; @@ -44,6 +43,7 @@ import org.apache.beam.sdk.options.ManualDockerEnvironmentOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.RemoteEnvironmentOptions; +import org.apache.beam.sdk.util.construction.Environments; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java index 4ba993251739..a71086891fa4 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java @@ -31,7 +31,6 @@ import java.util.Collections; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler; import org.apache.beam.runners.fnexecution.provisioning.StaticGrpcProvisionService; import org.apache.beam.sdk.fn.IdGenerator; @@ -39,6 +38,7 @@ import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.RemoteEnvironmentOptions; +import org.apache.beam.sdk.util.construction.Environments; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java index 9bb9eb616117..1e65b57ef56f 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java @@ -23,8 +23,8 @@ import static org.mockito.Mockito.verify; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler; +import org.apache.beam.sdk.util.construction.Environments; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java index 2515a4afd644..16a74446aeaf 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlersTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.fnexecution.state; -import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -32,11 +32,11 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.junit.Rule; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java index a5f23db5b29e..f9a19dacc3df 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.fnexecution.translation; -import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; +import static org.apache.beam.sdk.util.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -31,11 +31,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.ImmutableExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.SideInputReference; import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.IterableSideInputHandler; import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler; import org.apache.beam.sdk.coders.KvCoder; @@ -47,6 +42,11 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.ImmutableExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.SideInputReference; import org.apache.beam.sdk.values.KV; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java index 7a2d54b55064..0a17ee639544 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.fnexecution.wire; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects.firstNonNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; @@ -57,10 +57,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardCoders; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; -import org.apache.beam.runners.core.construction.CoderTranslator; -import org.apache.beam.runners.core.construction.ModelCoderRegistrar; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.ByteCoder; import org.apache.beam.sdk.coders.Coder; @@ -85,6 +81,10 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.ShardedKey; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; +import org.apache.beam.sdk.util.construction.CoderTranslator; +import org.apache.beam.sdk.util.construction.ModelCoderRegistrar; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java index 86917e1c6bd4..061c75284f85 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java @@ -24,9 +24,6 @@ import java.io.OutputStream; import java.util.Collection; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -35,6 +32,9 @@ import org.apache.beam.sdk.coders.LengthPrefixCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Rule; diff --git a/runners/java-job-service/build.gradle b/runners/java-job-service/build.gradle index e3d26ece4765..e4286374ef10 100644 --- a/runners/java-job-service/build.gradle +++ b/runners/java-job-service/build.gradle @@ -24,12 +24,10 @@ description = "Apache Beam :: Runners :: Java Job Service" dependencies { implementation library.java.vendored_guava_32_1_2_jre - implementation project(":runners:core-construction-java") implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(":sdks:java:expansion-service") - implementation project(":runners:core-construction-java") implementation project(":runners:java-fn-execution") implementation library.java.jackson_core implementation library.java.jackson_databind diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java index 99ae19e503be..1392b30af38c 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java @@ -49,14 +49,14 @@ import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.graph.PipelineValidator; import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService; import org.apache.beam.sdk.fn.server.FnService; import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver; import org.apache.beam.sdk.function.ThrowingConsumer; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.graph.PipelineValidator; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java index affc4cc0b09b..db488499d4f9 100644 --- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java +++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java @@ -40,12 +40,12 @@ import org.apache.beam.model.jobmanagement.v1.JobApi; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.sdk.io.ClassLoaderFileSystem; import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.MessageOrBuilder; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.util.JsonFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java index 4793ffcbc8af..c887b624f17a 100644 --- a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java +++ b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java @@ -28,11 +28,11 @@ import java.util.concurrent.Executors; import org.apache.beam.model.jobmanagement.v1.JobApi; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.metrics.MetricResults; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; diff --git a/runners/jet/build.gradle b/runners/jet/build.gradle index 4b87fe940407..2b6bf2bcdff4 100644 --- a/runners/jet/build.gradle +++ b/runners/jet/build.gradle @@ -27,7 +27,6 @@ description = "Apache Beam :: Runners :: Hazelcast Jet" evaluationDependsOn(":sdks:java:core") evaluationDependsOn(":runners:core-java") -evaluationDependsOn(":runners:core-construction-java") project.ext { jet_version = '4.0' @@ -43,7 +42,6 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":runners:core-java") implementation project(path: ":model:pipeline", configuration: "shadow") - implementation project(":runners:core-construction-java") implementation "com.hazelcast.jet:hazelcast-jet:$jet_version" implementation library.java.joda_time implementation library.java.vendored_guava_32_1_2_jre @@ -51,19 +49,16 @@ dependencies { testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(path: ":runners:core-java", configuration: "testRuntimeMigration") - testImplementation project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") testImplementation library.java.joda_time testImplementation "com.hazelcast.jet:hazelcast-jet-core:$jet_version:tests" testImplementation "com.hazelcast:hazelcast:$hazelcast_version:tests" needsRunner project(path: ":sdks:java:core", configuration: "shadowTest") needsRunner project(path: ":runners:core-java", configuration: "testRuntimeMigration") - needsRunner project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") needsRunner project(path: project.path, configuration: "testRuntimeMigration") validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest") validatesRunner project(path: ":runners:core-java", configuration: "testRuntimeMigration") - validatesRunner project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") validatesRunner project(path: project.path, configuration: "testRuntimeMigration") } @@ -124,7 +119,6 @@ task needsRunnerTests(type: Test) { systemProperty "beamTestPipelineOptions", JsonOutput.toJson(["--runner=TestJetRunner"]) classpath = configurations.needsRunner - testClassesDirs += files(project(":runners:core-construction-java").sourceSets.test.output.classesDirs) testClassesDirs += files(project(":runners:core-java").sourceSets.test.output.classesDirs) testClassesDirs += files(project(":sdks:java:core").sourceSets.test.output.classesDirs) useJUnit { diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetGraphVisitor.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetGraphVisitor.java index bad7b19be80c..bc0d645e363d 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetGraphVisitor.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetGraphVisitor.java @@ -19,11 +19,11 @@ import com.hazelcast.jet.core.DAG; import java.util.function.Function; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.PValue; /** Logic that specifies how to apply translations when traversing the nodes of a Beam pipeline. */ diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetRunner.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetRunner.java index 620136da5221..c27f9a0f0611 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetRunner.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetRunner.java @@ -31,10 +31,6 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Function; import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded; -import org.apache.beam.runners.core.construction.UnconsumedReads; import org.apache.beam.runners.core.metrics.MetricUpdates; import org.apache.beam.runners.jet.metrics.JetMetricsContainer; import org.apache.beam.sdk.Pipeline; @@ -43,6 +39,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDoNaiveBounded; +import org.apache.beam.sdk.util.construction.UnconsumedReads; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java index fc9e1e6ccb13..45550dc445c8 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java @@ -26,11 +26,6 @@ import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.CreatePCollectionViewTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.ReadTranslation; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.jet.processors.AssignWindowP; import org.apache.beam.runners.jet.processors.BoundedSourceP; import org.apache.beam.runners.jet.processors.FlattenP; @@ -51,6 +46,11 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.ReadTranslation; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java index 08ac6dc91ee5..62aeae335ded 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.jet; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java index aa19fb2fa444..38cb96b13a07 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java @@ -34,8 +34,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import javax.annotation.Nonnull; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -48,6 +46,8 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java index c5bd4ecaf2ef..3af66c3bd73a 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java @@ -47,7 +47,6 @@ import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.core.SideInputReader; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.jet.DAGBuilder; import org.apache.beam.runners.jet.JetPipelineOptions; import org.apache.beam.runners.jet.Utils; @@ -60,6 +59,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java index 4da880e618b1..30601d19e42c 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java @@ -31,12 +31,12 @@ import java.util.List; import java.util.function.Function; import javax.annotation.Nonnull; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.jet.Utils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for reading from a bounded Beam diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ParDoP.java index 004a0222ee29..b6bb029d2e30 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ParDoP.java @@ -28,11 +28,11 @@ import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java index 3b3d79193622..718d60ce8a99 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java @@ -35,13 +35,13 @@ import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java index 38a75d47e703..a6666b26eb6a 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java @@ -31,12 +31,12 @@ import java.util.List; import java.util.function.Function; import javax.annotation.Nonnull; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.jet.Utils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for reading from an unbounded Beam diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java index 1d2d39058ba1..cec527902c58 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java @@ -41,8 +41,6 @@ import org.apache.beam.runners.core.ReduceFnRunner; import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.runners.jet.Utils; @@ -54,6 +52,8 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestJetRunner.java b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestJetRunner.java index 0f34db0b2eba..b2a5188a37c2 100644 --- a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestJetRunner.java +++ b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestJetRunner.java @@ -28,7 +28,6 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineRunner; @@ -40,6 +39,7 @@ import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/portability/java/build.gradle b/runners/portability/java/build.gradle index 1cb61bc30cc8..45709a0dbd1e 100644 --- a/runners/portability/java/build.gradle +++ b/runners/portability/java/build.gradle @@ -44,7 +44,6 @@ dependencies { implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre - testImplementation project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") testImplementation library.java.hamcrest testImplementation library.java.junit testImplementation library.java.mockito_core diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java index b657b8bb49a1..000efb7430af 100644 --- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java +++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.portability; -import static org.apache.beam.runners.core.construction.resources.PipelineResources.detectClassPathResourcesToStage; +import static org.apache.beam.sdk.util.construction.resources.PipelineResources.detectClassPathResourcesToStage; import java.util.Arrays; import java.util.List; @@ -33,11 +33,6 @@ import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc.JobServiceBlockingStub; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.DefaultArtifactResolver; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService; import org.apache.beam.runners.portability.CloseableResource.CloseException; @@ -50,6 +45,11 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.sdk.util.construction.DefaultArtifactResolver; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/runners/samza/build.gradle b/runners/samza/build.gradle index fbbec322c87b..8e2cc2a5eb7d 100644 --- a/runners/samza/build.gradle +++ b/runners/samza/build.gradle @@ -46,7 +46,6 @@ dependencies { implementation library.java.vendored_guava_32_1_2_jre implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":runners:core-java") - implementation project(":runners:core-construction-java") implementation project(":runners:java-fn-execution") implementation project(":runners:java-job-service") implementation library.java.jackson_annotations diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java index 9c3bdcf4c887..2a45f20060e3 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java @@ -20,12 +20,12 @@ import java.util.UUID; import javax.annotation.Nullable; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.runners.jobsubmission.JobInvoker; import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator; import org.apache.beam.runners.jobsubmission.PortablePipelineRunner; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java index 6bfec1e3cdff..fb974b79ab1b 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java @@ -18,17 +18,17 @@ package org.apache.beam.runners.samza; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; -import org.apache.beam.runners.core.construction.graph.ProtoOverrides; -import org.apache.beam.runners.core.construction.graph.SplittableParDoExpander; -import org.apache.beam.runners.core.construction.graph.TrivialNativeTransformExpander; -import org.apache.beam.runners.core.construction.renderer.PipelineDotRenderer; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.jobsubmission.PortablePipelineResult; import org.apache.beam.runners.jobsubmission.PortablePipelineRunner; import org.apache.beam.runners.samza.translation.SamzaPortablePipelineTranslator; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; +import org.apache.beam.sdk.util.construction.graph.ProtoOverrides; +import org.apache.beam.sdk.util.construction.graph.SplittableParDoExpander; +import org.apache.beam.sdk.util.construction.graph.TrivialNativeTransformExpander; +import org.apache.beam.sdk.util.construction.renderer.PipelineDotRenderer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java index 26745114b43d..111fd684ff63 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java @@ -24,8 +24,6 @@ import java.util.ServiceLoader; import java.util.Set; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.renderer.PipelineDotRenderer; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.jobsubmission.PortablePipelineResult; import org.apache.beam.runners.samza.translation.ConfigBuilder; @@ -44,6 +42,8 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.renderer.PipelineDotRenderer; import org.apache.beam.sdk.values.PValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.samza.application.StreamApplication; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java index eb2b423a1171..2b25033dd95f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java @@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.serialization.Base64Serializer; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.runners.samza.metrics.FnWithMetricsWrapper; @@ -43,6 +42,7 @@ import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.Partition; import org.apache.samza.SamzaException; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java index 3b53894954d7..3fd356796d4a 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java @@ -35,7 +35,6 @@ import java.util.function.Function; import java.util.stream.Collectors; import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.serialization.Base64Serializer; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.runners.samza.metrics.FnWithMetricsWrapper; @@ -47,6 +46,7 @@ import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.Partition; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOpFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOpFactory.java index 503e9049b98e..a4112a510459 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOpFactory.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOpFactory.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.samza.metrics; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.samza.runtime.Op; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.checkerframework.checker.nullness.qual.NonNull; /** diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java index b963f75f71d5..bb396ae0456b 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java @@ -37,7 +37,6 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.control.ExecutableStageContext; import org.apache.beam.runners.fnexecution.control.StageBundleFactory; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; @@ -54,6 +53,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java index fcac505f8cf9..f4e3641aeb9e 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java @@ -34,7 +34,6 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.control.ExecutableStageContext; import org.apache.beam.runners.fnexecution.control.StageBundleFactory; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; @@ -50,6 +49,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java index 758a341b2a5f..7625704de8ee 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java @@ -39,9 +39,6 @@ import org.apache.beam.runners.core.StatefulDoFnRunner; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode; import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory; import org.apache.beam.runners.fnexecution.control.RemoteBundle; import org.apache.beam.runners.fnexecution.control.StageBundleFactory; @@ -61,6 +58,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStateRequestHandlers.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStateRequestHandlers.java index f7ef0e5764ba..b891f746197b 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStateRequestHandlers.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStateRequestHandlers.java @@ -27,7 +27,6 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.StateTags; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors; import org.apache.beam.runners.fnexecution.control.StageBundleFactory; import org.apache.beam.runners.fnexecution.state.StateRequestHandler; @@ -39,6 +38,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java index 2ba60a168b99..e4ea4794b94c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java @@ -39,8 +39,6 @@ import org.apache.beam.runners.core.StateInternalsFactory; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateTag; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.UserStateReference; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.runners.samza.state.SamzaMapState; import org.apache.beam.runners.samza.state.SamzaSetState; @@ -66,6 +64,8 @@ import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.UserStateReference; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Ints; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java index cf164b145b63..fc0b5ff7a51c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java @@ -34,8 +34,6 @@ import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.core.serialization.Base64Serializer; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -45,6 +43,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java index c9832ecbabd3..7cf502a92715 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java @@ -32,7 +32,6 @@ import java.util.Map; import java.util.UUID; import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.serialization.Base64Serializer; import org.apache.beam.runners.samza.SamzaExecutionEnvironment; import org.apache.beam.runners.samza.SamzaPipelineOptions; @@ -41,6 +40,7 @@ import org.apache.beam.runners.samza.runtime.SamzaStoreStateInternals; import org.apache.beam.runners.samza.util.ConfigUtils; import org.apache.beam.runners.samza.util.PortableConfigUtils; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.config.ApplicationConfig; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/FlattenPCollectionsTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/FlattenPCollectionsTranslator.java index 03fd613f85e9..fe840ee86461 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/FlattenPCollectionsTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/FlattenPCollectionsTranslator.java @@ -24,13 +24,13 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.samza.runtime.Op; import org.apache.beam.runners.samza.runtime.OpAdapter; import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java index f88c36c171c8..527522e66728 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java @@ -24,8 +24,6 @@ import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.runners.core.SystemReduceFn; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.runners.samza.runtime.DoFnOp; import org.apache.beam.runners.samza.runtime.GroupByKeyOp; @@ -47,6 +45,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ImpulseTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ImpulseTranslator.java index 97ee9ab502b6..35d8f1f39962 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ImpulseTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ImpulseTranslator.java @@ -17,12 +17,12 @@ */ package org.apache.beam.runners.samza.translation; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.runners.samza.util.SamzaPipelineTranslatorUtils; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.samza.operators.KV; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java index 2c998417b403..e8668f5244b3 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java @@ -32,10 +32,6 @@ import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.RunnerPCollectionView; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.runners.samza.runtime.DoFnOp; import org.apache.beam.runners.samza.runtime.Op; @@ -61,6 +57,10 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.RunnerPCollectionView; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PortableTranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PortableTranslationContext.java index f64c8dc616e3..8821ee62928d 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PortableTranslationContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PortableTranslationContext.java @@ -23,10 +23,10 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.graph.PipelineNode; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.runners.samza.runtime.OpMessage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.samza.application.descriptors.StreamApplicationDescriptor; import org.apache.samza.operators.KV; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java index e1530ead60ad..59caecdc41c2 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.samza.translation; import java.util.Map; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.core.serialization.Base64Serializer; import org.apache.beam.runners.samza.adapter.BoundedSourceSystem; import org.apache.beam.runners.samza.adapter.UnboundedSourceSystem; @@ -30,6 +29,7 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java index e82020238ce6..0bfe935df6fc 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java @@ -19,9 +19,6 @@ import com.google.auto.service.AutoService; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.NativeTransforms; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.runners.samza.util.SamzaCoders; import org.apache.beam.runners.samza.util.WindowUtils; @@ -30,6 +27,9 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.NativeTransforms; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.samza.operators.MessageStream; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java index 74eb6d71e7f0..e90e63317708 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java @@ -23,14 +23,14 @@ import java.util.HashMap; import java.util.Map; import java.util.ServiceLoader; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.samza.metrics.SamzaMetricOpFactory; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.PValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java index a5ec212ed3de..150b1ce90902 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java @@ -23,11 +23,11 @@ import java.util.ServiceLoader; import java.util.Set; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.samza.SamzaPipelineOptions; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishView.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishView.java index a3ebbffef9a8..3c09474eda2c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishView.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishView.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.samza.translation; import java.util.List; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTransformOverride.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTransformOverride.java index 839f4d6145ca..89848147cef4 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTransformOverride.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTransformOverride.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.samza.translation; import org.apache.beam.runners.core.Concatenate; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java index 06324617b1f5..ef96bbf97aac 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java @@ -20,10 +20,6 @@ import java.io.IOException; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.TestStreamTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.core.serialization.Base64Serializer; import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.runners.samza.util.SamzaPipelineTranslatorUtils; @@ -33,6 +29,10 @@ import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.TestStreamTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTransformOverrides.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTransformOverrides.java index b7063df11ae6..3eb169337329 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTransformOverrides.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTransformOverrides.java @@ -18,12 +18,12 @@ package org.apache.beam.runners.samza.translation; import java.util.List; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded; -import org.apache.beam.runners.core.construction.UnsupportedOverrideFactory; import org.apache.beam.sdk.runners.PTransformOverride; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDoNaiveBounded; +import org.apache.beam.sdk.util.construction.UnsupportedOverrideFactory; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** {@link org.apache.beam.sdk.transforms.PTransform} overrides for Samza runner. */ diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java index 0ed7252cf73d..9a0e9a78da79 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.samza.runtime.DoFnOp; import org.apache.beam.runners.samza.runtime.KvToKeyedWorkItemOp; import org.apache.beam.runners.samza.runtime.OpAdapter; @@ -36,6 +35,7 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TransformConfigGenerator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TransformConfigGenerator.java index b6ac2ba4388b..aa956e4b28e0 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TransformConfigGenerator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TransformConfigGenerator.java @@ -19,10 +19,10 @@ import java.util.Collections; import java.util.Map; -import org.apache.beam.runners.core.construction.graph.PipelineNode; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; /** Generates config for a BEAM PTransform (regular java api or portable api). */ public interface TransformConfigGenerator> { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TransformTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TransformTranslator.java index 0f55f6dd3b29..477b5e3a91e2 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TransformTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TransformTranslator.java @@ -17,10 +17,10 @@ */ package org.apache.beam.runners.samza.translation; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; /** Interface of Samza translator for BEAM {@link PTransform}. */ public interface TransformTranslator> { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java index 49402393cf70..ae6015c5f00d 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java @@ -27,8 +27,6 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.samza.SamzaPipelineOptions; import org.apache.beam.runners.samza.metrics.SamzaMetricOpFactory; import org.apache.beam.runners.samza.metrics.SamzaTransformMetricRegistry; @@ -41,6 +39,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java index 32eeaa18a5f6..6482a686ec0b 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java @@ -18,15 +18,15 @@ package org.apache.beam.runners.samza.translation; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.samza.runtime.OpAdapter; import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.runners.samza.runtime.WindowAssignOp; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.samza.operators.MessageStream; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/DoFnUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/DoFnUtils.java index de212367c514..6fd6a6370a1e 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/DoFnUtils.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/DoFnUtils.java @@ -19,8 +19,8 @@ import java.util.Set; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.commons.collections.CollectionUtils; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java index 99f71f53860b..4ab1bcc8b0ee 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java @@ -19,9 +19,9 @@ import java.io.IOException; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.graph.PipelineNode; import org.apache.beam.runners.fnexecution.wire.WireCoders; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.PCollection; /** Utilities for pipeline translation. */ diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/StateUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/StateUtils.java index 0ebe89ca56c8..1db9db74074e 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/StateUtils.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/StateUtils.java @@ -18,9 +18,9 @@ package org.apache.beam.runners.samza.util; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; /** Utils for determining stateful operators. */ public class StateUtils { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java index 07aa2a4cb994..b4531d078933 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java @@ -19,12 +19,12 @@ import java.io.IOException; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode; import org.apache.beam.runners.fnexecution.wire.WireCoders; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.WindowingStrategy; /** Utils for window operations. */ diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/util/DoFnUtilsTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/util/DoFnUtilsTest.java index ea74f078c75d..cef1a463ff96 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/util/DoFnUtilsTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/util/DoFnUtilsTest.java @@ -21,14 +21,14 @@ import java.io.Serializable; import java.util.Objects; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/util/WindowUtilsTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/util/WindowUtilsTest.java index a76811b05102..9c614175e5de 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/util/WindowUtilsTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/util/WindowUtilsTest.java @@ -20,8 +20,6 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -31,6 +29,8 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java index 024c854fe814..96717f29e87f 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java @@ -24,8 +24,6 @@ import java.util.concurrent.Future; import java.util.concurrent.ThreadFactory; import javax.annotation.Nullable; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.graph.ProjectionPushdownOptimizer; import org.apache.beam.runners.core.metrics.MetricsPusher; import org.apache.beam.runners.core.metrics.NoOpMetricsSink; import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsAccumulator; @@ -42,6 +40,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PipelineOptionsValidator; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.graph.ProjectionPushdownOptimizer; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.spark.SparkEnv$; import org.apache.spark.metrics.MetricsSystem; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java index 351da747b43c..1b0b7d4f09f0 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java @@ -32,8 +32,6 @@ import java.util.Set; import java.util.function.Supplier; import javax.annotation.Nullable; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.SparkCommonPipelineOptions; import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.SideInputValues; import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider; @@ -48,6 +46,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java index 6e2d21bfb948..f2045aa5c194 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java @@ -29,8 +29,6 @@ import java.util.HashMap; import javax.annotation.Nullable; import org.apache.beam.repackaged.core.org.apache.commons.lang3.ArrayUtils; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.resources.PipelineResources; import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.SideInputValues; import org.apache.beam.sdk.coders.BigDecimalCoder; @@ -76,6 +74,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.resources.PipelineResources; import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkTransformOverrides.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkTransformOverrides.java index b12816b9b19b..a60aa59efd61 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkTransformOverrides.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkTransformOverrides.java @@ -18,13 +18,13 @@ package org.apache.beam.runners.spark.structuredstreaming.translation; import java.util.List; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded; -import org.apache.beam.runners.core.construction.UnsupportedOverrideFactory; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDoNaiveBounded; +import org.apache.beam.sdk.util.construction.UnsupportedOverrideFactory; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** {@link PTransform} overrides for Spark runner. */ diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java index 58a596d1c267..2a01af5fb76e 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Map; import java.util.function.Supplier; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.spark.structuredstreaming.translation.PipelineTranslator.TranslationState; import org.apache.beam.runners.spark.structuredstreaming.translation.PipelineTranslator.UnresolvedTranslation; import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.SideInputValues; @@ -36,6 +35,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java index 48caa938994f..6b548eff88c2 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java @@ -25,7 +25,6 @@ import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.core.SideInputReader; -import org.apache.beam.runners.core.construction.ParDoTranslation; import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsAccumulator; import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.CachedSideInputReader; import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.NoOpStepContext; @@ -39,6 +38,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java index 71a67c76fa86..c4a18801ccba 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java @@ -19,7 +19,6 @@ import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.spark.structuredstreaming.translation.PipelineTranslator; import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; import org.apache.beam.sdk.Pipeline; @@ -32,6 +31,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java index de9e1d375e3e..e395c3f35908 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java @@ -19,13 +19,13 @@ import java.io.IOException; import java.util.function.Supplier; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.spark.structuredstreaming.io.BoundedDatasetFactory; import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.spark.sql.Encoder; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java index fda067afb2fa..b348eca0c436 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java @@ -30,13 +30,13 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index 5e74ccd2b195..d775cfd4d6e9 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -159,7 +159,6 @@ spark.components = [ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(":runners:core-construction-java") implementation project(":runners:core-java") implementation project(":runners:java-fn-execution") implementation project(":runners:java-job-service") @@ -423,8 +422,6 @@ tasks.register("validatesStructuredStreamingRunnerBatch", Test) { excludeTestsMatching 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenMultipleCoders' // SDF excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testLifecycleMethodsBounded' - // https://github.com/apache/beam/issues/29972 - excludeTestsMatching 'org.apache.beam.sdk.transforms.CombineTest$CombineWithContextTests.testHotKeyCombineWithSideInputs' } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkCommonPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkCommonPipelineOptions.java index 10e560f88bca..3c91b3547b7f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkCommonPipelineOptions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkCommonPipelineOptions.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.spark; -import org.apache.beam.runners.core.construction.resources.PipelineResources; import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.options.ApplicationNameOptions; @@ -27,6 +26,7 @@ import org.apache.beam.sdk.options.FileStagingOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.util.construction.resources.PipelineResources; /** * Spark runner {@link PipelineOptions} handles Spark execution-related configurations, such as the diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java index dca208c7a1ed..3125dedd1a36 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java @@ -19,13 +19,13 @@ import java.util.UUID; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.jobsubmission.JobInvocation; import org.apache.beam.runners.jobsubmission.JobInvoker; import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator; import org.apache.beam.runners.jobsubmission.PortablePipelineRunner; import org.apache.beam.sdk.options.PortablePipelineOptions; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkNativePipelineVisitor.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkNativePipelineVisitor.java index 143eab8be85f..638823c8834a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkNativePipelineVisitor.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkNativePipelineVisitor.java @@ -22,13 +22,13 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkPipelineTranslator; import org.apache.beam.runners.spark.translation.TransformEvaluator; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java index e914f0b462d8..d741de12d779 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java @@ -28,13 +28,6 @@ import org.apache.beam.model.jobmanagement.v1.ArtifactApi; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; -import org.apache.beam.runners.core.construction.graph.ProtoOverrides; -import org.apache.beam.runners.core.construction.graph.SplittableParDoExpander; -import org.apache.beam.runners.core.construction.graph.TrivialNativeTransformExpander; import org.apache.beam.runners.core.metrics.MetricsPusher; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.jobsubmission.PortablePipelineJarUtils; @@ -52,6 +45,13 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.metrics.MetricsOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; +import org.apache.beam.sdk.util.construction.graph.ProtoOverrides; +import org.apache.beam.sdk.util.construction.graph.SplittableParDoExpander; +import org.apache.beam.sdk.util.construction.graph.TrivialNativeTransformExpander; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index f5720365b260..034e0632df49 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -26,9 +26,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.TransformInputs; -import org.apache.beam.runners.core.construction.graph.ProjectionPushdownOptimizer; import org.apache.beam.runners.core.metrics.MetricsPusher; import org.apache.beam.runners.spark.metrics.MetricsAccumulator; import org.apache.beam.runners.spark.metrics.SparkBeamMetricSource; @@ -53,6 +50,9 @@ import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.TransformInputs; +import org.apache.beam.sdk.util.construction.graph.ProjectionPushdownOptimizer; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java index 31e3e34ee0d7..402d81c63660 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java @@ -19,7 +19,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkContextFactory; import org.apache.beam.runners.spark.translation.SparkPipelineTranslator; @@ -30,6 +29,7 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.joda.time.Duration; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkTransformOverrides.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkTransformOverrides.java index 95a0080ce0bc..52cf7ba6f2fb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkTransformOverrides.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkTransformOverrides.java @@ -18,13 +18,13 @@ package org.apache.beam.runners.spark; import java.util.List; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded; -import org.apache.beam.runners.core.construction.UnsupportedOverrideFactory; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDoNaiveBounded; +import org.apache.beam.sdk.util.construction.UnsupportedOverrideFactory; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** {@link PTransform} overrides for Spark runner. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java index e5e344669070..2f0b5bc917f8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java @@ -19,10 +19,10 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.spark.api.java.JavaSparkContext$; import org.apache.spark.rdd.RDD; import org.apache.spark.streaming.StreamingContext; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java index c846ac20a626..d0f06fb248f6 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java @@ -26,7 +26,6 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.metrics.MetricsAccumulator; import org.apache.beam.runners.spark.metrics.MetricsContainerStepMapAccumulator; @@ -36,6 +35,7 @@ import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.spark.Dependency; import org.apache.spark.HashPartitioner; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java index 34ef3331ae49..450ab7fed0eb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java @@ -22,7 +22,6 @@ import java.io.Serializable; import java.util.Collections; import java.util.Iterator; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; @@ -42,6 +41,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext$; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java index 930dd4d15e25..bc7ed5bbbac1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java @@ -30,8 +30,6 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.metrics.CounterCell; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; @@ -52,6 +50,8 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java index b72481128f53..f448ab0c5e75 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java @@ -23,7 +23,6 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.EmptyCheckpointMark; @@ -38,6 +37,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Stopwatch; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index d6895216c2f8..49a2f1d80755 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -25,8 +25,6 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; @@ -36,6 +34,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index df36d24531a6..4ee9fd5d2a11 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -29,7 +29,6 @@ import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.metrics.MetricsContainerStepMapAccumulator; import org.apache.beam.runners.spark.util.CachedSideInputReader; import org.apache.beam.runners.spark.util.SideInputBroadcast; @@ -41,6 +40,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java index 3ef215834dec..e37941a35557 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java @@ -37,12 +37,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId; import org.apache.beam.runners.core.SystemReduceFn; -import org.apache.beam.runners.core.construction.NativeTransforms; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.metrics.MetricsAccumulator; @@ -57,6 +51,12 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.util.construction.NativeTransforms; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java index 1075ae0d2a7d..a4c5da770a3c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java @@ -37,7 +37,6 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; import org.apache.beam.runners.core.SideInputReader; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.runners.spark.util.SparkSideInputReader; import org.apache.beam.sdk.coders.Coder; @@ -50,6 +49,7 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java index a3d3186ad479..7f4a9af0f743 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java @@ -34,9 +34,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.InMemoryTimerInternals; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.Timer; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.runners.fnexecution.control.BundleProgressHandler; import org.apache.beam.runners.fnexecution.control.ExecutableStageContext; @@ -63,6 +60,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.broadcast.Broadcast; @@ -71,7 +71,7 @@ /** * Spark function that passes its input through an SDK-executed {@link - * org.apache.beam.runners.core.construction.graph.ExecutableStage}. + * org.apache.beam.sdk.util.construction.graph.ExecutableStage}. * *

The output of this operation is a multiplexed {@link Dataset} whose elements are tagged with a * union coder. The coder's tags are determined by {@link SparkExecutableStageFunction#outputMap}. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java index 5b6704b76183..b8007750061d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java @@ -30,13 +30,13 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java index dea82d8e3da0..523dcbad0823 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java @@ -34,11 +34,6 @@ import java.util.Set; import java.util.concurrent.LinkedBlockingQueue; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; @@ -55,6 +50,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkTranslationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkTranslationContext.java index e28b9412bf47..e0e6333c9112 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkTranslationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkTranslationContext.java @@ -22,11 +22,11 @@ import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.spark.api.java.JavaSparkContext; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 878447e7f8d5..5dc553faab5b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -27,11 +27,6 @@ import java.util.Iterator; import java.util.Map; import org.apache.beam.runners.core.SystemReduceFn; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.SourceRDD; @@ -66,6 +61,11 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDoNaiveBounded; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 266b67798a22..599d45f9e9cc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -31,11 +31,6 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.ConsoleIO; import org.apache.beam.runners.spark.io.CreateStream; @@ -77,6 +72,11 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java index 9cb4b44c897c..0c62224ccb20 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java @@ -26,7 +26,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineWithContext; @@ -40,6 +39,7 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java index 2068e272051a..cc6d9ea69756 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.spark.translation; -import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.mockito.Matchers.any; @@ -37,8 +37,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; import org.apache.beam.runners.fnexecution.control.BundleCheckpointHandler; @@ -57,6 +55,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Before; diff --git a/runners/twister2/build.gradle b/runners/twister2/build.gradle index 2db020a7536a..36a044737d82 100644 --- a/runners/twister2/build.gradle +++ b/runners/twister2/build.gradle @@ -37,7 +37,6 @@ dependencies { exclude group: 'com.esotericsoftware.kryo', module: 'kryo' } implementation project(path: ":model:pipeline", configuration: "shadow") - implementation project(path: ":runners:core-construction-java") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.jackson_annotations implementation library.java.joda_time diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java index 5a390b4741f4..c101358cd9f7 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java @@ -43,11 +43,6 @@ import java.util.logging.Logger; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded; -import org.apache.beam.runners.core.construction.resources.PipelineResources; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineRunner; @@ -55,6 +50,11 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PTransformOverride; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SplittableParDo; +import org.apache.beam.sdk.util.construction.SplittableParDoNaiveBounded; +import org.apache.beam.sdk.util.construction.resources.PipelineResources; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java index a5b03cd62865..7102e8f8ef0e 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java @@ -26,11 +26,11 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java index 079e89bf194b..d48dfb5cc4db 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java @@ -27,10 +27,6 @@ import java.util.NoSuchElementException; import java.util.logging.Logger; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.ReadTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.twister2.Twister2TranslationContext; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; @@ -38,6 +34,10 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.ReadTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; /** Twister2 wrapper for Bounded Source. */ @SuppressWarnings({ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/Twister2BatchPipelineTranslator.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/Twister2BatchPipelineTranslator.java index aad5b0ae59fa..ceb3b9bfcf72 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/Twister2BatchPipelineTranslator.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/Twister2BatchPipelineTranslator.java @@ -20,7 +20,6 @@ import java.util.HashMap; import java.util.Map; import java.util.logging.Logger; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.twister2.Twister2BatchTranslationContext; import org.apache.beam.runners.twister2.Twister2PipelineOptions; import org.apache.beam.runners.twister2.translators.batch.AssignWindowTranslatorBatch; @@ -32,6 +31,7 @@ import org.apache.beam.runners.twister2.translators.batch.ReadSourceTranslatorBatch; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.checkerframework.checker.nullness.qual.Nullable; /** Twister pipeline translator for batch pipelines. */ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java index 38c38cc3547e..3b17bdac9e91 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java @@ -19,7 +19,6 @@ import edu.iu.dsc.tws.api.tset.sets.batch.BatchTSet; import java.io.IOException; -import org.apache.beam.runners.core.construction.CreatePCollectionViewTranslation; import org.apache.beam.runners.twister2.Twister2BatchTranslationContext; import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.runners.twister2.translators.functions.ByteToElemFunction; @@ -34,6 +33,7 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java index 1a452e8b372b..97aeff608d7a 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java @@ -25,7 +25,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.ParDoTranslation; import org.apache.beam.runners.twister2.Twister2BatchTranslationContext; import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.runners.twister2.translators.functions.DoFnFunction; @@ -37,6 +36,7 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java index eef0a13f5141..14e858fe5108 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java @@ -20,12 +20,12 @@ import edu.iu.dsc.tws.tset.env.BatchTSetEnvironment; import edu.iu.dsc.tws.tset.env.TSetEnvironment; import edu.iu.dsc.tws.tset.sets.batch.SourceTSet; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.twister2.Twister2BatchTranslationContext; import org.apache.beam.runners.twister2.translation.wrappers.Twister2BoundedSource; import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; /** Source translator. */ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java index c2f078adc73f..6559c903bbee 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java @@ -25,15 +25,15 @@ import java.util.Iterator; import java.util.logging.Logger; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.twister2.utils.Twister2AssignContext; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; /** Assign Windows function. */ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java index 30522c89521b..4f47785add38 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java @@ -36,12 +36,6 @@ import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.StepContext; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.twister2.Twister2TranslationContext; import org.apache.beam.runners.twister2.utils.NoOpStepContext; import org.apache.beam.runners.twister2.utils.Twister2SideInputReader; @@ -56,6 +50,12 @@ import org.apache.beam.sdk.util.DoFnWithExecutionInformation; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.SerializablePipelineOptions; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java index 9929b100bd9d..4c3dd604577b 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java @@ -36,11 +36,6 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.TriggerTranslation; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.options.PipelineOptions; @@ -48,6 +43,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TriggerTranslation; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle index 9de7e0da1eb3..ff6568787bdc 100644 --- a/sdks/java/core/build.gradle +++ b/sdks/java/core/build.gradle @@ -77,13 +77,15 @@ dependencies { shadow project(path: ":model:pipeline", configuration: "shadow") shadow project(path: ":model:fn-execution", configuration: "shadow") shadow project(path: ":model:job-management", configuration: "shadow") + shadow project(path: ":sdks:java:transform-service:launcher") shadow library.java.vendored_grpc_1_60_1 shadow library.java.vendored_guava_32_1_2_jre shadow library.java.byte_buddy - implementation library.java.antlr_runtime - implementation library.java.commons_compress - implementation library.java.commons_lang3 + shadow library.java.antlr_runtime + shadow library.java.commons_compress + shadow library.java.commons_lang3 testImplementation library.java.mockito_inline + shadow library.java.classgraph shadow library.java.jsr305 shadow library.java.error_prone_annotations shadow library.java.jackson_core @@ -102,6 +104,7 @@ dependencies { provided library.java.everit_json_schema shadowTest library.java.everit_json_schema provided library.java.junit + testImplementation "com.github.stefanbirkner:system-rules:1.19.0" provided library.java.hamcrest provided 'io.airlift:aircompressor:0.18' provided 'com.facebook.presto.hadoop:hadoop-apache2:3.2.0-1' diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ArtifactResolver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ArtifactResolver.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ArtifactResolver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ArtifactResolver.java index 08d024552f21..85995fa13670 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ArtifactResolver.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ArtifactResolver.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.List; import java.util.Optional; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/BeamUrns.java similarity index 96% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/BeamUrns.java index a5bbc0103363..05bb2b0e0a00 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/BeamUrns.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ProtocolMessageEnum; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslation.java index 999aa620b8e8..5a0d8c3cae25 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.io.IOException; import java.util.ArrayList; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslator.java similarity index 93% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslator.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslator.java index 842c25c18804..3d89c4c7ff4a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslator.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.List; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; /** * An interface that translates coders to components and back. diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslatorRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslatorRegistrar.java similarity index 96% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslatorRegistrar.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslatorRegistrar.java index 551ac6fd0c3d..b69d0290de52 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslatorRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslatorRegistrar.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Map; import org.apache.beam.sdk.coders.Coder; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java similarity index 95% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java index c175595baaf8..1f07aba8a303 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.util.Collections; import java.util.List; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -143,7 +142,7 @@ public byte[] getPayload(WindowedValue.ParamWindowedValueCoder from) { @Override public WindowedValue.ParamWindowedValueCoder fromComponents( - List> components, byte[] payload, TranslationContext context) { + List> components, byte[] payload, CoderTranslation.TranslationContext context) { return WindowedValue.ParamWindowedValueCoder.fromComponents(components, payload); } }; @@ -163,7 +162,7 @@ public byte[] getPayload(RowCoder from) { @Override public RowCoder fromComponents( - List> components, byte[] payload, TranslationContext context) { + List> components, byte[] payload, CoderTranslation.TranslationContext context) { checkArgument( components.isEmpty(), "Expected empty component list, but received: " + components); Schema schema; @@ -225,7 +224,7 @@ public abstract static class SimpleStructuredCoderTranslator> implements CoderTranslator { @Override public final T fromComponents( - List> components, byte[] payload, TranslationContext context) { + List> components, byte[] payload, CoderTranslation.TranslationContext context) { return fromComponents(components); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CombineTranslation.java similarity index 89% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CombineTranslation.java index d502ba1514db..932b59266402 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CombineTranslation.java @@ -15,11 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; - -import static org.apache.beam.runners.core.construction.PTransformTranslation.COMBINE_GLOBALLY_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.COMBINE_GROUPED_VALUES_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import java.io.IOException; @@ -27,7 +23,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; @@ -49,20 +44,21 @@ * RunnerApi.CombinePayload} protos. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class CombineTranslation { static final String JAVA_SERIALIZED_COMBINE_FN_URN = "beam:combinefn:javasdk:v1"; - /** A {@link TransformPayloadTranslator} for {@link Combine.PerKey}. */ + /** A {@link PTransformTranslation.TransformPayloadTranslator} for {@link Combine.PerKey}. */ public static class CombinePerKeyPayloadTranslator implements PTransformTranslation.TransformPayloadTranslator> { private CombinePerKeyPayloadTranslator() {} @Override public String getUrn() { - return COMBINE_PER_KEY_TRANSFORM_URN; + return PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN; } @Override @@ -102,14 +98,14 @@ private static Coder extractAccumulatorCoder( } } - /** A {@link TransformPayloadTranslator} for {@link Combine.Globally}. */ + /** A {@link PTransformTranslation.TransformPayloadTranslator} for {@link Combine.Globally}. */ public static class CombineGloballyPayloadTranslator implements PTransformTranslation.TransformPayloadTranslator> { private CombineGloballyPayloadTranslator() {} @Override public String getUrn() { - return COMBINE_GLOBALLY_TRANSFORM_URN; + return PTransformTranslation.COMBINE_GLOBALLY_TRANSFORM_URN; } @Override @@ -159,14 +155,16 @@ static CombinePayload payloadForCombineGlobally( } } - /** A {@link TransformPayloadTranslator} for {@link Combine.GroupedValues}. */ + /** + * A {@link PTransformTranslation.TransformPayloadTranslator} for {@link Combine.GroupedValues}. + */ public static class CombineGroupedValuesPayloadTranslator implements PTransformTranslation.TransformPayloadTranslator> { private CombineGroupedValuesPayloadTranslator() {} @Override public String getUrn() { - return COMBINE_GROUPED_VALUES_TRANSFORM_URN; + return PTransformTranslation.COMBINE_GROUPED_VALUES_TRANSFORM_URN; } @Override @@ -210,15 +208,18 @@ private static Coder extractAccumulatorCoder( } /** - * Registers {@link TransformPayloadTranslator TransformPayloadTranslators} for {@link Combine - * Combines}. + * Registers {@link PTransformTranslation.TransformPayloadTranslator TransformPayloadTranslators} + * for {@link Combine Combines}. */ @AutoService(TransformPayloadTranslatorRegistrar.class) public static class Registrar implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators() { - return ImmutableMap., TransformPayloadTranslator>builder() + return ImmutableMap + ., PTransformTranslation.TransformPayloadTranslator>builder() .put(Combine.Globally.class, new CombineGloballyPayloadTranslator()) .put(Combine.GroupedValues.class, new CombineGroupedValuesPayloadTranslator()) .put(Combine.PerKey.class, new CombinePerKeyPayloadTranslator()) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CreatePCollectionViewTranslation.java similarity index 95% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CreatePCollectionViewTranslation.java index d27599c1ae07..1a2b217dc7aa 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CreatePCollectionViewTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -25,13 +25,13 @@ import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; @@ -45,6 +45,7 @@ */ @Deprecated @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class CreatePCollectionViewTranslation { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DeduplicatedFlattenFactory.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DeduplicatedFlattenFactory.java index 03f3d42a6491..57e9763b941a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DeduplicatedFlattenFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.HashMap; import java.util.Map; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultArtifactResolver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DefaultArtifactResolver.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultArtifactResolver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DefaultArtifactResolver.java index 7b20c2e6ca19..4edd54a0c66c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultArtifactResolver.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DefaultArtifactResolver.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.ArrayList; import java.util.List; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DefaultExpansionServiceClientFactory.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DefaultExpansionServiceClientFactory.java index 106b8b39cbee..f18b8da5062c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DefaultExpansionServiceClientFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DisplayDataTranslation.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DisplayDataTranslation.java index d0e22d5e1df2..2d112eac70fc 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/DisplayDataTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/EmptyFlattenAsCreateFactory.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/EmptyFlattenAsCreateFactory.java index cbfb1f9c0842..89eef31e92a4 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/EmptyFlattenAsCreateFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Environments.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Environments.java index cc1f8cbd0701..161a57de1395 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Environments.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.fasterxml.jackson.core.Base64Variants; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExecutableStageTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExecutableStageTranslation.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExecutableStageTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExecutableStageTranslation.java index 822418c0ff4d..9669f462e0c5 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExecutableStageTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExecutableStageTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -26,8 +26,8 @@ import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.LinkedHashMultimap; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExpansionServiceClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExpansionServiceClient.java similarity index 95% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExpansionServiceClient.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExpansionServiceClient.java index 52ed0280cf7c..095b03e0dc5a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExpansionServiceClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExpansionServiceClient.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import org.apache.beam.model.expansion.v1.ExpansionApi; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExpansionServiceClientFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExpansionServiceClientFactory.java similarity index 95% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExpansionServiceClientFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExpansionServiceClientFactory.java index fdc78f6d5385..94196c41ab6c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExpansionServiceClientFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExpansionServiceClientFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import org.apache.beam.model.pipeline.v1.Endpoints; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/External.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/External.java index b3f21d80b371..b7c3551e64dc 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/External.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslation.java index eb4c9f8a0bee..a96370622025 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslationOptions.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptions.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslationOptions.java index 4b3ef24ca1d2..5ed14faf31ee 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslationOptions.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.List; import org.apache.beam.sdk.options.Default; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptionsRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslationOptionsRegistrar.java similarity index 96% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptionsRegistrar.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslationOptionsRegistrar.java index 6296f4c83775..3902f7669730 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslationOptionsRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ExternalTranslationOptionsRegistrar.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import org.apache.beam.sdk.annotations.Internal; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/FlattenTranslator.java similarity index 92% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/FlattenTranslator.java index 201a65e6233c..3bdf17c33005 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/FlattenTranslator.java @@ -15,23 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import java.util.Collections; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.Window.Assign; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; /** * Utility methods for translating a {@link Assign} to and from {@link RunnerApi} representations. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class FlattenTranslator implements TransformPayloadTranslator> { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ForwardingPTransform.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ForwardingPTransform.java index 3e49900843cc..4ded8de37f47 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ForwardingPTransform.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Map; import org.apache.beam.sdk.coders.CannotProvideCoderException; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupByKeyTranslation.java similarity index 84% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupByKeyTranslation.java index 183fa7ffcdc9..d08a48d0e5e6 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupByKeyTranslation.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import java.util.Collections; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; @@ -32,11 +31,13 @@ * representations. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class GroupByKeyTranslation { - static class GroupByKeyTranslator implements TransformPayloadTranslator> { + static class GroupByKeyTranslator + implements PTransformTranslation.TransformPayloadTranslator> { @Override public String getUrn() { return PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; @@ -53,7 +54,9 @@ public FunctionSpec translate( @AutoService(TransformPayloadTranslatorRegistrar.class) public static class Registrar implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators() { return Collections.singletonMap(GroupByKey.class, new GroupByKeyTranslator()); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslation.java similarity index 86% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslation.java index 7c81afd8ae07..499c7fd21f51 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslation.java @@ -15,20 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.GroupIntoBatchesPayload; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.GroupIntoBatches; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) public class GroupIntoBatchesTranslation { @@ -37,7 +37,7 @@ public class GroupIntoBatchesTranslation { * representation. */ static class GroupIntoBatchesTranslator - implements TransformPayloadTranslator> { + implements PTransformTranslation.TransformPayloadTranslator> { @Override public String getUrn() { return PTransformTranslation.GROUP_INTO_BATCHES_URN; @@ -59,7 +59,8 @@ public RunnerApi.FunctionSpec translate( * protobuf representation. */ static class ShardedGroupIntoBatchesTranslator - implements TransformPayloadTranslator.WithShardedKey> { + implements PTransformTranslation.TransformPayloadTranslator< + GroupIntoBatches.WithShardedKey> { @Override public String getUrn() { return PTransformTranslation.GROUP_INTO_BATCHES_WITH_SHARDED_KEY_URN; @@ -90,9 +91,12 @@ private static GroupIntoBatchesPayload getPayloadFromParameters( @AutoService(TransformPayloadTranslatorRegistrar.class) public static class Registrar implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators() { - return ImmutableMap., TransformPayloadTranslator>builder() + return ImmutableMap + ., PTransformTranslation.TransformPayloadTranslator>builder() .put(GroupIntoBatches.class, new GroupIntoBatchesTranslator()) .put(GroupIntoBatches.WithShardedKey.class, new ShardedGroupIntoBatchesTranslator()) .build(); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ImpulseTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ImpulseTranslation.java similarity index 91% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ImpulseTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ImpulseTranslation.java index 25f0cd7749b5..0dece5effba1 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ImpulseTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ImpulseTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import java.io.IOException; @@ -23,15 +23,16 @@ import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; /** * Utility methods for translating a {@link Impulse} to and from {@link RunnerApi} representations. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class ImpulseTranslation { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java index 01a69f441290..7fde7bd17484 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoders.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoders.java index e37b52638be8..d7a4a41eab0c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoders.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/NativeTransforms.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/NativeTransforms.java index 19127acfca39..964eabbd5fb1 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/NativeTransforms.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import java.util.Iterator; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PCollectionTranslation.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PCollectionTranslation.java index badf288cb6a4..3b816732283d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PCollectionTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.io.IOException; import org.apache.beam.model.pipeline.v1.RunnerApi; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PCollectionViewTranslation.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PCollectionViewTranslation.java index 180934fb8b3b..f0ea7f877dd9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PCollectionViewTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformMatchers.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformMatchers.java index aec269b5929d..747f3b32cefc 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformMatchers.java @@ -15,9 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; - -import static org.apache.beam.runners.core.construction.PTransformTranslation.WRITE_FILES_TRANSFORM_URN; +package org.apache.beam.sdk.util.construction; import java.io.IOException; import java.util.HashSet; @@ -527,7 +525,7 @@ public String toString() { public static PTransformMatcher writeWithRunnerDeterminedSharding() { return application -> { - if (WRITE_FILES_TRANSFORM_URN.equals( + if (PTransformTranslation.WRITE_FILES_TRANSFORM_URN.equals( PTransformTranslation.urnForTransformOrNull(application.getTransform()))) { try { return WriteFilesTranslation.isRunnerDeterminedSharding((AppliedPTransform) application); @@ -535,7 +533,7 @@ public static PTransformMatcher writeWithRunnerDeterminedSharding() { throw new RuntimeException( String.format( "Transform with URN %s failed to parse: %s", - WRITE_FILES_TRANSFORM_URN, application.getTransform()), + PTransformTranslation.WRITE_FILES_TRANSFORM_URN, application.getTransform()), exc); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformReplacements.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformReplacements.java index 7dbc0b964f19..5d84209ec759 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformReplacements.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java index 14f6ae7e4b48..3167c2f22b0d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PTransformTranslation.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.io.IOException; @@ -35,8 +35,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms.CombineComponents; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms.SplittableParDoComponents; -import org.apache.beam.runners.core.construction.ExternalTranslation.ExternalTranslator; -import org.apache.beam.runners.core.construction.ParDoTranslation.ParDoTranslator; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.RowCoder; @@ -48,6 +46,8 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.common.ReflectHelpers.ObjectsClassComparator; +import org.apache.beam.sdk.util.construction.ExternalTranslation.ExternalTranslator; +import org.apache.beam.sdk.util.construction.ParDoTranslation.ParDoTranslator; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ParDoTranslation.java similarity index 92% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ParDoTranslation.java index eb4291dc003a..23906c733ae3 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ParDoTranslation.java @@ -15,15 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; - -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; -import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN; +package org.apache.beam.sdk.util.construction; + import static org.apache.beam.sdk.transforms.reflect.DoFnSignatures.getStateSpecOrThrow; import static org.apache.beam.sdk.transforms.reflect.DoFnSignatures.getTimerSpecOrThrow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -47,8 +40,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardRequirements; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardUserStateTypes; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformTranslator; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -87,6 +78,7 @@ /** Utilities for interacting with {@link ParDo} instances and {@link ParDoPayload} protos. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class ParDoTranslation { @@ -131,23 +123,24 @@ public class ParDoTranslation { static { checkState( REQUIRES_STATEFUL_PROCESSING_URN.equals( - getUrn(StandardRequirements.Enum.REQUIRES_STATEFUL_PROCESSING))); + BeamUrns.getUrn(StandardRequirements.Enum.REQUIRES_STATEFUL_PROCESSING))); checkState( REQUIRES_BUNDLE_FINALIZATION_URN.equals( - getUrn(StandardRequirements.Enum.REQUIRES_BUNDLE_FINALIZATION))); + BeamUrns.getUrn(StandardRequirements.Enum.REQUIRES_BUNDLE_FINALIZATION))); checkState( - REQUIRES_STABLE_INPUT_URN.equals(getUrn(StandardRequirements.Enum.REQUIRES_STABLE_INPUT))); + REQUIRES_STABLE_INPUT_URN.equals( + BeamUrns.getUrn(StandardRequirements.Enum.REQUIRES_STABLE_INPUT))); checkState( REQUIRES_TIME_SORTED_INPUT_URN.equals( - getUrn(StandardRequirements.Enum.REQUIRES_TIME_SORTED_INPUT))); + BeamUrns.getUrn(StandardRequirements.Enum.REQUIRES_TIME_SORTED_INPUT))); checkState( REQUIRES_SPLITTABLE_DOFN_URN.equals( - getUrn(StandardRequirements.Enum.REQUIRES_SPLITTABLE_DOFN))); + BeamUrns.getUrn(StandardRequirements.Enum.REQUIRES_SPLITTABLE_DOFN))); checkState( REQUIRES_ON_WINDOW_EXPIRATION_URN.equals( - getUrn(StandardRequirements.Enum.REQUIRES_ON_WINDOW_EXPIRATION))); - checkState(BAG_USER_STATE.equals(getUrn(StandardUserStateTypes.Enum.BAG))); - checkState(MULTIMAP_USER_STATE.equals(getUrn(StandardUserStateTypes.Enum.MULTIMAP))); + BeamUrns.getUrn(StandardRequirements.Enum.REQUIRES_ON_WINDOW_EXPIRATION))); + checkState(BAG_USER_STATE.equals(BeamUrns.getUrn(StandardUserStateTypes.Enum.BAG))); + checkState(MULTIMAP_USER_STATE.equals(BeamUrns.getUrn(StandardUserStateTypes.Enum.MULTIMAP))); } /** The URN for an unknown Java {@link DoFn}. */ @@ -157,10 +150,11 @@ public class ParDoTranslation { /** The URN for an unknown Java {@link WindowMappingFn}. */ public static final String CUSTOM_JAVA_WINDOW_MAPPING_FN_URN = "beam:windowmappingfn:javasdk:0.1"; - /** A {@link TransformPayloadTranslator} for {@link ParDo}. */ - public static class ParDoTranslator implements TransformTranslator> { + /** A {@link PTransformTranslation.TransformPayloadTranslator} for {@link ParDo}. */ + public static class ParDoTranslator + implements PTransformTranslation.TransformTranslator> { - public static TransformTranslator create() { + public static PTransformTranslation.TransformTranslator create() { return new ParDoTranslator(); } @@ -168,7 +162,7 @@ private ParDoTranslator() {} @Override public String getUrn(ParDo.MultiOutput transform) { - return PAR_DO_TRANSFORM_URN; + return PTransformTranslation.PAR_DO_TRANSFORM_URN; } @Override @@ -191,7 +185,7 @@ public RunnerApi.PTransform translate( ParDoPayload payload = translateParDo(appliedParDo, components); builder.setSpec( RunnerApi.FunctionSpec.newBuilder() - .setUrn(PAR_DO_TRANSFORM_URN) + .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN) .setPayload(payload.toByteString()) .build()); builder.setEnvironmentId(components.getEnvironmentIdFor(appliedParDo.getResourceHints())); @@ -526,12 +520,16 @@ public static List> getSideInputs(AppliedPTransform public static RunnerApi.PCollection getMainInput( RunnerApi.PTransform ptransform, Components components) throws IOException { checkArgument( - PAR_DO_TRANSFORM_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_PAIR_WITH_RESTRICTION_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_PROCESS_ELEMENTS_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.equals( + PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN.equals( + ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN.equals( + ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN.equals( + ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN.equals( + ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.equals( ptransform.getSpec().getUrn()), "Unexpected payload type %s", ptransform.getSpec().getUrn()); @@ -543,12 +541,16 @@ public static RunnerApi.PCollection getMainInput( public static String getMainInputName(RunnerApi.PTransformOrBuilder ptransform) throws IOException { checkArgument( - PAR_DO_TRANSFORM_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_PAIR_WITH_RESTRICTION_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_PROCESS_ELEMENTS_URN.equals(ptransform.getSpec().getUrn()) - || SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.equals( + PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN.equals( + ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN.equals( + ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN.equals( + ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN.equals( + ptransform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.equals( ptransform.getSpec().getUrn()), "Unexpected payload type %s", ptransform.getSpec().getUrn()); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PipelineOptionsTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PipelineOptionsTranslation.java index 840466d6b96b..cd6ab7dd414a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PipelineOptionsTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.fasterxml.jackson.core.TreeNode; import com.fasterxml.jackson.core.type.TypeReference; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PipelineTranslation.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PipelineTranslation.java index 688d7a808647..828b4f7ad83c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PipelineTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.io.IOException; import java.util.ArrayList; @@ -27,11 +27,11 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.graph.PipelineValidator; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy.Node; +import org.apache.beam.sdk.util.construction.graph.PipelineValidator; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ListMultimap; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PrimitiveCreate.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PrimitiveCreate.java index d91ddbc079bc..2186c0fe1f05 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/PrimitiveCreate.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Map; import org.apache.beam.sdk.coders.Coder; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReadTranslation.java similarity index 91% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReadTranslation.java index 71cdb18ccf11..acf9edd8ac4d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReadTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -27,7 +27,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.IsBounded; import org.apache.beam.model.pipeline.v1.RunnerApi.ReadPayload; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; @@ -46,6 +45,7 @@ * ReadPayload} protos. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class ReadTranslation { @@ -143,11 +143,14 @@ public static PCollection.IsBounded sourceIsBounded(AppliedPTransform t } } - /** A {@link TransformPayloadTranslator} for {@link SplittableParDo.PrimitiveUnboundedRead}. */ + /** + * A {@link PTransformTranslation.TransformPayloadTranslator} for {@link + * SplittableParDo.PrimitiveUnboundedRead}. + */ public static class UnboundedReadPayloadTranslator implements PTransformTranslation.TransformPayloadTranslator< SplittableParDo.PrimitiveUnboundedRead> { - public static TransformPayloadTranslator create() { + public static PTransformTranslation.TransformPayloadTranslator create() { return new UnboundedReadPayloadTranslator(); } @@ -170,11 +173,14 @@ public FunctionSpec translate( } } - /** A {@link TransformPayloadTranslator} for {@link SplittableParDo.PrimitiveBoundedRead}. */ + /** + * A {@link PTransformTranslation.TransformPayloadTranslator} for {@link + * SplittableParDo.PrimitiveBoundedRead}. + */ public static class BoundedReadPayloadTranslator implements PTransformTranslation.TransformPayloadTranslator< SplittableParDo.PrimitiveBoundedRead> { - public static TransformPayloadTranslator create() { + public static PTransformTranslation.TransformPayloadTranslator create() { return new BoundedReadPayloadTranslator(); } @@ -201,9 +207,12 @@ public FunctionSpec translate( @AutoService(TransformPayloadTranslatorRegistrar.class) public static class Registrar implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators() { - return ImmutableMap., TransformPayloadTranslator>builder() + return ImmutableMap + ., PTransformTranslation.TransformPayloadTranslator>builder() .put(SplittableParDo.PrimitiveUnboundedRead.class, new UnboundedReadPayloadTranslator()) .put(SplittableParDo.PrimitiveBoundedRead.class, new BoundedReadPayloadTranslator()) .build(); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RehydratedComponents.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RehydratedComponents.java index f1bbc521a096..f79696214368 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RehydratedComponents.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; @@ -26,7 +26,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.values.PCollection; @@ -85,7 +84,9 @@ public Coder load(String id) throws Exception { RunnerApi.@Nullable Coder coder = components.getCodersOrDefault(id, null); checkState(coder != null, "No coder with id '%s' in serialized components", id); return CoderTranslation.fromProto( - coder, RehydratedComponents.this, TranslationContext.DEFAULT); + coder, + RehydratedComponents.this, + CoderTranslation.TranslationContext.DEFAULT); } }); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReplacementOutputs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReplacementOutputs.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReplacementOutputs.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReplacementOutputs.java index 6ff0d8c0ec03..308ae1a2d891 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReplacementOutputs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReplacementOutputs.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReshuffleTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReshuffleTranslation.java similarity index 84% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReshuffleTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReshuffleTranslation.java index bd91673f3818..017793915641 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReshuffleTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ReshuffleTranslation.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import java.util.Collections; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Reshuffle; @@ -32,11 +31,13 @@ * representations. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class ReshuffleTranslation { - static class ReshuffleTranslator implements TransformPayloadTranslator> { + static class ReshuffleTranslator + implements PTransformTranslation.TransformPayloadTranslator> { @Override public String getUrn() { return PTransformTranslation.RESHUFFLE_URN; @@ -53,7 +54,9 @@ public FunctionSpec translate( @AutoService(TransformPayloadTranslatorRegistrar.class) public static class Registrar implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators() { return Collections.singletonMap(Reshuffle.class, new ReshuffleTranslator()); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java index 1e9af091dee3..3af8836c4622 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Map; import java.util.Objects; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SdkComponents.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SdkComponents.java index dc24c5540920..cad527ad28b9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SdkComponents.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SerializablePipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SerializablePipelineOptions.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SerializablePipelineOptions.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SerializablePipelineOptions.java index e2a1257cbaa6..05ac9b9e4327 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SerializablePipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SerializablePipelineOptions.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SingleInputOutputOverrideFactory.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SingleInputOutputOverrideFactory.java index fae483db9cf2..d5282f800667 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SingleInputOutputOverrideFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Map; import org.apache.beam.sdk.runners.PTransformOverrideFactory; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDo.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDo.java index 5ea2c4968dd9..3873d154a884 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDo.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -29,11 +29,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput; import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.ParDoTranslation.ParDoLike; -import org.apache.beam.runners.core.construction.ParDoTranslation.ParDoLikeTimerFamilySpecs; -import org.apache.beam.runners.core.construction.ReadTranslation.BoundedReadPayloadTranslator; -import org.apache.beam.runners.core.construction.ReadTranslation.UnboundedReadPayloadTranslator; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -65,6 +60,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.NameUtils; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.ParDoTranslation.ParDoLike; +import org.apache.beam.sdk.util.construction.ParDoTranslation.ParDoLikeTimerFamilySpecs; +import org.apache.beam.sdk.util.construction.ReadTranslation.BoundedReadPayloadTranslator; +import org.apache.beam.sdk.util.construction.ReadTranslation.UnboundedReadPayloadTranslator; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -99,6 +99,7 @@ * ParDo.of(splittable DoFn)}, but not for direct use by pipeline writers. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class SplittableParDo diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDoNaiveBounded.java similarity index 96% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDoNaiveBounded.java index ad6b51539742..edae34fbecf9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDoNaiveBounded.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.util.Collection; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.core.construction.SplittableParDo.ProcessKeyedElements; import org.apache.beam.sdk.fn.splittabledofn.RestrictionTrackers; import org.apache.beam.sdk.fn.splittabledofn.RestrictionTrackers.ClaimObserver; import org.apache.beam.sdk.options.PipelineOptions; @@ -60,18 +59,21 @@ /** * Utility transforms and overrides for running bounded splittable DoFn's naively, by implementing - * {@link ProcessKeyedElements} using a simple {@link Reshuffle} and {@link ParDo}. + * {@link SplittableParDo.ProcessKeyedElements} using a simple {@link Reshuffle} and {@link ParDo}. */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public class SplittableParDoNaiveBounded { - /** Overrides a {@link ProcessKeyedElements} into {@link SplittableProcessNaive}. */ + /** + * Overrides a {@link SplittableParDo.ProcessKeyedElements} into {@link SplittableProcessNaive}. + */ public static class OverrideFactory implements PTransformOverrideFactory< PCollection>>, PCollectionTuple, - ProcessKeyedElements> { + SplittableParDo.ProcessKeyedElements< + InputT, OutputT, RestrictionT, WatermarkEstimatorStateT>> { @Override public PTransformReplacement< PCollection>>, PCollectionTuple> @@ -79,7 +81,8 @@ public static class OverrideFactory>>, PCollectionTuple, - ProcessKeyedElements> + SplittableParDo.ProcessKeyedElements< + InputT, OutputT, RestrictionT, WatermarkEstimatorStateT>> transform) { checkArgument( DoFnSignatures.signatureForDoFn(transform.getTransform().getFn()).isBoundedPerElement() @@ -100,11 +103,14 @@ public Map, ReplacementOutput> mapOutputs( static class SplittableProcessNaive< InputT, OutputT, RestrictionT, PositionT, WatermarkEstimatorStateT> extends PTransform>>, PCollectionTuple> { - private final ProcessKeyedElements + private final SplittableParDo.ProcessKeyedElements< + InputT, OutputT, RestrictionT, WatermarkEstimatorStateT> original; SplittableProcessNaive( - ProcessKeyedElements original) { + SplittableParDo.ProcessKeyedElements< + InputT, OutputT, RestrictionT, WatermarkEstimatorStateT> + original) { this.original = original; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SyntheticComponents.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SyntheticComponents.java similarity index 96% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SyntheticComponents.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SyntheticComponents.java index 5f0834911a1b..e28846eeb9d0 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SyntheticComponents.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SyntheticComponents.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.function.Predicate; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TestStreamTranslation.java similarity index 92% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TestStreamTranslation.java index 4fcf32bc7486..154c3c87a1ce 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TestStreamTranslation.java @@ -15,10 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; - -import static org.apache.beam.runners.core.construction.PTransformTranslation.TEST_STREAM_TRANSFORM_URN; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import java.io.IOException; @@ -27,7 +24,6 @@ import java.util.List; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestStream; @@ -37,6 +33,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.joda.time.Duration; import org.joda.time.Instant; @@ -45,6 +42,7 @@ * representations. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class TestStreamTranslation { @@ -82,8 +80,8 @@ public static TestStream getTestStream( SdkComponents sdkComponents = SdkComponents.create(application.getPipeline().getOptions()); RunnerApi.PTransform transformProto = PTransformTranslation.toProto(application, sdkComponents); - checkArgument( - TEST_STREAM_TRANSFORM_URN.equals(transformProto.getSpec().getUrn()), + Preconditions.checkArgument( + PTransformTranslation.TEST_STREAM_TRANSFORM_URN.equals(transformProto.getSpec().getUrn()), "Attempt to get %s from a transform with wrong URN %s", TestStream.class.getSimpleName(), transformProto.getSpec().getUrn()); @@ -166,10 +164,11 @@ static TestStream.Event eventFromProto( } /** A translator registered to translate {@link TestStream} objects to protobuf representation. */ - static class TestStreamTranslator implements TransformPayloadTranslator> { + static class TestStreamTranslator + implements PTransformTranslation.TransformPayloadTranslator> { @Override public String getUrn() { - return TEST_STREAM_TRANSFORM_URN; + return PTransformTranslation.TEST_STREAM_TRANSFORM_URN; } @Override @@ -182,7 +181,7 @@ public RunnerApi.FunctionSpec translate( private RunnerApi.FunctionSpec translateTyped( final TestStream testStream, SdkComponents components) throws IOException { return RunnerApi.FunctionSpec.newBuilder() - .setUrn(TEST_STREAM_TRANSFORM_URN) + .setUrn(PTransformTranslation.TEST_STREAM_TRANSFORM_URN) .setPayload(payloadForTestStream(testStream, components).toByteString()) .build(); } @@ -191,7 +190,9 @@ private RunnerApi.FunctionSpec translateTyped( @AutoService(TransformPayloadTranslatorRegistrar.class) public static class Registrar implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators() { return Collections.singletonMap(TestStream.class, new TestStreamTranslator()); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Timer.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Timer.java index 42c5ce8cb8ef..08707d6e7331 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Timer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.value.AutoValue; import java.io.IOException; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformInputs.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformInputs.java index 52fcd3fe031e..a9efbd34ce17 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformInputs.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformPayloadTranslatorRegistrar.java similarity index 82% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformPayloadTranslatorRegistrar.java index 3e6cbc148197..bbcd5644c68a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformPayloadTranslatorRegistrar.java @@ -15,10 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.transforms.PTransform; /** A registrar of TransformPayloadTranslator. */ @@ -26,6 +25,8 @@ "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public interface TransformPayloadTranslatorRegistrar { - Map, ? extends TransformPayloadTranslator> + Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators(); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java index 6e2bce503be1..d14dd81283de 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -40,12 +40,12 @@ import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher; import org.apache.beam.sdk.util.ReleaseInfo; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TriggerTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TriggerTranslation.java index d4b5da958f31..d4ead4ce8946 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TriggerTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSource.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSource.java index 53fad782da96..430cec0d2479 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnconsumedReads.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnconsumedReads.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnconsumedReads.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnconsumedReads.java index b1afed1ef55d..fafd385708b1 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnconsumedReads.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnconsumedReads.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.HashSet; import java.util.Set; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnknownCoderWrapper.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnknownCoderWrapper.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnknownCoderWrapper.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnknownCoderWrapper.java index 0476cd3a0e3a..e72b99f2fd8e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnknownCoderWrapper.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnknownCoderWrapper.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.io.IOException; import java.io.InputStream; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnsupportedOverrideFactory.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnsupportedOverrideFactory.java index 18bbae7b54bb..faaae92ed420 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnsupportedOverrideFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Map; import org.apache.beam.sdk.runners.AppliedPTransform; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WindowIntoTranslation.java similarity index 90% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WindowIntoTranslation.java index 3beeb043309d..6aece50d858e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WindowIntoTranslation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -26,7 +26,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.Window; @@ -39,11 +38,13 @@ * representations. */ @SuppressWarnings({ + "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class WindowIntoTranslation { - static class WindowAssignTranslator implements TransformPayloadTranslator> { + static class WindowAssignTranslator + implements PTransformTranslation.TransformPayloadTranslator> { @Override public String getUrn() { @@ -105,10 +106,10 @@ public static WindowIntoPayload getWindowIntoPayload(AppliedPTransform getWindowIntoPayload(application).getWindowFn()); } - /** A {@link TransformPayloadTranslator} for {@link Window}. */ + /** A {@link PTransformTranslation.TransformPayloadTranslator} for {@link Window}. */ public static class WindowIntoPayloadTranslator implements PTransformTranslation.TransformPayloadTranslator> { - public static TransformPayloadTranslator create() { + public static PTransformTranslation.TransformPayloadTranslator create() { return new WindowIntoPayloadTranslator(); } @@ -134,7 +135,9 @@ public FunctionSpec translate( @AutoService(TransformPayloadTranslatorRegistrar.class) public static class Registrar implements TransformPayloadTranslatorRegistrar { @Override - public Map, ? extends TransformPayloadTranslator> + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators() { return Collections.singletonMap(Window.Assign.class, new WindowIntoPayloadTranslator()); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WindowingStrategyTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WindowingStrategyTranslation.java index 7266b4125cae..f46c43ca5bcf 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WindowingStrategyTranslation.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import java.io.IOException; import java.io.Serializable; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WriteFilesTranslation.java similarity index 96% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WriteFilesTranslation.java index a2ad432ce643..8d7dcb1582fd 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/WriteFilesTranslation.java @@ -15,9 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; -import static org.apache.beam.runners.core.construction.PTransformTranslation.WRITE_FILES_TRANSFORM_URN; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; @@ -32,7 +31,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput; import org.apache.beam.model.pipeline.v1.RunnerApi.WriteFilesPayload; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.WriteFiles; import org.apache.beam.sdk.io.WriteFilesResult; @@ -241,7 +239,7 @@ public FunctionSpec getSpec() { @Override public FunctionSpec migrate(SdkComponents components) throws IOException { return FunctionSpec.newBuilder() - .setUrn(WRITE_FILES_TRANSFORM_URN) + .setUrn(PTransformTranslation.WRITE_FILES_TRANSFORM_URN) .setPayload(payloadForWriteFilesLike(this, components).toByteString()) .build(); } @@ -294,10 +292,11 @@ public boolean isRunnerDeterminedSharding() { } } - static class WriteFilesTranslator implements TransformPayloadTranslator> { + static class WriteFilesTranslator + implements PTransformTranslation.TransformPayloadTranslator> { @Override public String getUrn() { - return WRITE_FILES_TRANSFORM_URN; + return PTransformTranslation.WRITE_FILES_TRANSFORM_URN; } @Override @@ -315,7 +314,7 @@ public FunctionSpec translate( @AutoService(TransformPayloadTranslatorRegistrar.class) public static class Registrar implements TransformPayloadTranslatorRegistrar { @Override - public Map, TransformPayloadTranslator> + public Map, PTransformTranslation.TransformPayloadTranslator> getTransformPayloadTranslators() { return Collections.singletonMap(WriteFiles.CONCRETE_CLASS, new WriteFilesTranslator()); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ExecutableStage.java similarity index 86% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ExecutableStage.java index 21d252dff459..b8399e3bcf9d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ExecutableStage.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import java.util.Collection; import java.util.Collections; @@ -33,8 +33,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; /** * A combination of PTransforms that can be executed within a single SDK harness. @@ -56,7 +54,7 @@ public interface ExecutableStage { * Return the {@link Components} required to execute this {@link ExecutableStage}. * *

This must contain all of the transforms returned by {@link #getTransforms()} and the closure - * of all components that those {@link PTransformNode transforms} reference. + * of all components that those {@link PipelineNode.PTransformNode transforms} reference. */ RunnerApi.Components getComponents(); @@ -80,12 +78,12 @@ public interface ExecutableStage { Collection getWireCoderSettings(); /** - * Returns the root {@link PCollectionNode} of this {@link ExecutableStage}. This {@link - * ExecutableStage} executes by reading elements from a Remote gRPC Read Node. + * Returns the root {@link PipelineNode.PCollectionNode} of this {@link ExecutableStage}. This + * {@link ExecutableStage} executes by reading elements from a Remote gRPC Read Node. * *

TODO(BEAM-4658): Add timers as input PCollections to executable stages. */ - PCollectionNode getInputPCollection(); + PipelineNode.PCollectionNode getInputPCollection(); /** * Returns a set of descriptors that will be accessed by this {@link ExecutableStage} as side @@ -106,18 +104,19 @@ public interface ExecutableStage { Collection getTimers(); /** - * Returns the leaf {@link PCollectionNode PCollections} of this {@link ExecutableStage}. + * Returns the leaf {@link PipelineNode.PCollectionNode PCollections} of this {@link + * ExecutableStage}. * - *

All of these {@link PCollectionNode PCollections} are consumed by a {@link PTransformNode - * PTransform} which is not contained within this executable stage, and must be materialized at - * execution time by a Remote gRPC Write Transform. + *

All of these {@link PipelineNode.PCollectionNode PCollections} are consumed by a {@link + * PipelineNode.PTransformNode PTransform} which is not contained within this executable stage, + * and must be materialized at execution time by a Remote gRPC Write Transform. * *

TODO(BEAM-4658): Add timers as output PCollections to executable stages. */ - Collection getOutputPCollections(); + Collection getOutputPCollections(); /** Get the transforms that perform processing within this {@link ExecutableStage}. */ - Collection getTransforms(); + Collection getTransforms(); /** * Returns a composite {@link PTransform} which is equivalent to this {@link ExecutableStage} as @@ -129,8 +128,8 @@ public interface ExecutableStage { *

  • The only {@link PCollection PCollections} in the {@link PTransform#getInputsMap()} is the * result of {@link #getInputPCollection()} and {@link #getSideInputs()}. *
  • The output {@link PCollection PCollections} in the values of {@link - * PTransform#getOutputsMap()} are the {@link PCollectionNode PCollections} returned by - * {@link #getOutputPCollections()}. + * PTransform#getOutputsMap()} are the {@link PipelineNode.PCollectionNode PCollections} + * returned by {@link #getOutputPCollections()}. *
  • The {@link PTransform#getSpec()} contains an {@link ExecutableStagePayload} with inputs * and outputs equal to the PTransform's inputs and outputs, and transforms equal to the * result of {@link #getTransforms}. @@ -147,7 +146,7 @@ default PTransform toPTransform(String uniqueName) { payload.addAllWireCoderSettings(getWireCoderSettings()); // Populate inputs and outputs of the stage payload and outer PTransform simultaneously. - PCollectionNode input = getInputPCollection(); + PipelineNode.PCollectionNode input = getInputPCollection(); pt.putInputs("input", getInputPCollection().getId()); payload.setInput(input.getId()); @@ -178,7 +177,7 @@ default PTransform toPTransform(String uniqueName) { } int outputIndex = 0; - for (PCollectionNode output : getOutputPCollections()) { + for (PipelineNode.PCollectionNode output : getOutputPCollections()) { pt.putOutputs(String.format("materialized_%d", outputIndex), output.getId()); payload.addOutputs(output.getId()); outputIndex++; @@ -186,7 +185,7 @@ default PTransform toPTransform(String uniqueName) { // Inner PTransforms of this stage are hidden from the outer pipeline and only belong in the // stage payload. - for (PTransformNode transform : getTransforms()) { + for (PipelineNode.PTransformNode transform : getTransforms()) { payload.addTransforms(transform.getId()); } payload.setComponents( @@ -196,7 +195,9 @@ default PTransform toPTransform(String uniqueName) { .putAllTransforms( getTransforms().stream() .collect( - Collectors.toMap(PTransformNode::getId, PTransformNode::getTransform)))); + Collectors.toMap( + PipelineNode.PTransformNode::getId, + PipelineNode.PTransformNode::getTransform)))); pt.setSpec( FunctionSpec.newBuilder() @@ -221,7 +222,7 @@ static ExecutableStage fromPayload(ExecutableStagePayload payload) { Environment environment = payload.getEnvironment(); Collection wireCoderSettings = payload.getWireCoderSettingsList(); - PCollectionNode input = + PipelineNode.PCollectionNode input = PipelineNode.pCollection( payload.getInput(), components.getPcollectionsOrThrow(payload.getInput())); List sideInputs = @@ -236,11 +237,11 @@ static ExecutableStage fromPayload(ExecutableStagePayload payload) { payload.getTimersList().stream() .map(timerId -> TimerReference.fromTimerId(timerId, components)) .collect(Collectors.toList()); - List transforms = + List transforms = payload.getTransformsList().stream() .map(id -> PipelineNode.pTransform(id, components.getTransformsOrThrow(id))) .collect(Collectors.toList()); - List outputs = + List outputs = payload.getOutputsList().stream() .map(id -> PipelineNode.pCollection(id, components.getPcollectionsOrThrow(id))) .collect(Collectors.toList()); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/FieldAccessVisitor.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitor.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/FieldAccessVisitor.java index ac9d505ae865..c378ef3db7f9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/FieldAccessVisitor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import java.util.HashMap; import java.util.Map; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/FusedPipeline.java similarity index 92% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/FusedPipeline.java index a8f8743c9ff9..1f385def476c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/FusedPipeline.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import com.google.auto.value.AutoValue; import java.util.HashMap; @@ -28,8 +28,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.SyntheticComponents; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.SyntheticComponents; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; /** A {@link Pipeline} which has been separated into collections of executable components. */ @@ -38,7 +37,7 @@ public abstract class FusedPipeline { static FusedPipeline of( Components components, Set environmentalStages, - Set runnerStages, + Set runnerStages, Set requirements) { return new AutoValue_FusedPipeline(components, environmentalStages, runnerStages, requirements); } @@ -49,7 +48,7 @@ static FusedPipeline of( public abstract Set getFusedStages(); /** The {@link PTransform PTransforms} that a runner is responsible for executing. */ - public abstract Set getRunnerExecutedTransforms(); + public abstract Set getRunnerExecutedTransforms(); /** The {@link PTransform PTransforms} that a runner is responsible for executing. */ public abstract Set getRequirements(); @@ -70,7 +69,7 @@ public RunnerApi.Pipeline toPipeline() { Sets.union( executableStageTransforms.keySet(), getRunnerExecutedTransforms().stream() - .map(PTransformNode::getId) + .map(PipelineNode.PTransformNode::getId) .collect(Collectors.toSet())); // Augment the initial transforms with all of the executable transforms. @@ -82,7 +81,7 @@ public RunnerApi.Pipeline toPipeline() { .getTopologicallyOrderedTransforms() .spliterator(), false) - .map(PTransformNode::getId) + .map(PipelineNode.PTransformNode::getId) .collect(Collectors.toList()); Pipeline res = Pipeline.newBuilder() diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyPCollectionFusers.java similarity index 86% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyPCollectionFusers.java index 0405bfb2ccd5..9a36bfffeb82 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyPCollectionFusers.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -27,10 +27,8 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; @@ -127,10 +125,10 @@ class GreedyPCollectionFusers { /** Returns true if the PTransform node for the given input PCollection can be fused across. */ public static boolean canFuse( - PTransformNode transformNode, + PipelineNode.PTransformNode transformNode, Environment environment, - PCollectionNode candidate, - Collection stagePCollections, + PipelineNode.PCollectionNode candidate, + Collection stagePCollections, QueryablePipeline pipeline) { return URN_FUSIBILITY_CHECKERS .getOrDefault(transformNode.getTransform().getSpec().getUrn(), DEFAULT_FUSIBILITY_CHECKER) @@ -142,7 +140,9 @@ public static boolean canFuse( * environment. */ public static boolean isCompatible( - PTransformNode left, PTransformNode right, QueryablePipeline pipeline) { + PipelineNode.PTransformNode left, + PipelineNode.PTransformNode right, + QueryablePipeline pipeline) { CompatibilityChecker leftChecker = URN_COMPATIBILITY_CHECKERS.getOrDefault( left.getTransform().getSpec().getUrn(), DEFAULT_COMPATIBILITY_CHECKER); @@ -160,23 +160,26 @@ public static boolean isCompatible( private interface FusibilityChecker { /** - * Determine if a {@link PTransformNode} can be fused into an existing {@link ExecutableStage}. + * Determine if a {@link PipelineNode.PTransformNode} can be fused into an existing {@link + * ExecutableStage}. */ boolean canFuse( - PTransformNode transformNode, + PipelineNode.PTransformNode transformNode, Environment environment, - @SuppressWarnings("unused") PCollectionNode candidate, - Collection stagePCollections, + @SuppressWarnings("unused") PipelineNode.PCollectionNode candidate, + Collection stagePCollections, QueryablePipeline pipeline); } private interface CompatibilityChecker { /** - * Determine if two {@link PTransformNode PTransforms} can be fused into a new stage. This - * determines sibling fusion for new {@link ExecutableStage stages}. + * Determine if two {@link PipelineNode.PTransformNode PTransforms} can be fused into a new + * stage. This determines sibling fusion for new {@link ExecutableStage stages}. */ boolean isCompatible( - PTransformNode newNode, PTransformNode otherNode, QueryablePipeline pipeline); + PipelineNode.PTransformNode newNode, + PipelineNode.PTransformNode otherNode, + QueryablePipeline pipeline); } /** @@ -187,10 +190,10 @@ boolean isCompatible( * contain data for the side input window that contains the element. */ private static boolean canFuseParDo( - PTransformNode parDo, + PipelineNode.PTransformNode parDo, Environment environment, - PCollectionNode candidate, - Collection stagePCollections, + PipelineNode.PCollectionNode candidate, + Collection stagePCollections, QueryablePipeline pipeline) { Optional env = pipeline.getEnvironment(parDo); checkArgument( @@ -233,7 +236,9 @@ private static boolean canFuseParDo( } private static boolean parDoCompatibility( - PTransformNode parDo, PTransformNode other, QueryablePipeline pipeline) { + PipelineNode.PTransformNode parDo, + PipelineNode.PTransformNode other, + QueryablePipeline pipeline) { // Implicitly true if we are attempting to fuse against oneself. This case comes up for // PCollections representing timers since they create a self-loop in the graph. return parDo.equals(other) @@ -262,10 +267,10 @@ private static boolean parDoCompatibility( * A WindowInto can be fused into a stage if it executes in the same Environment as that stage. */ private static boolean canFuseCompatibleEnvironment( - PTransformNode operation, + PipelineNode.PTransformNode operation, Environment environment, - @SuppressWarnings("unused") PCollectionNode candidate, - @SuppressWarnings("unused") Collection stagePCollections, + @SuppressWarnings("unused") PipelineNode.PCollectionNode candidate, + @SuppressWarnings("unused") Collection stagePCollections, QueryablePipeline pipeline) { // WindowInto transforms may not have an environment Optional operationEnvironment = pipeline.getEnvironment(operation); @@ -273,7 +278,9 @@ private static boolean canFuseCompatibleEnvironment( } private static boolean compatibleEnvironments( - PTransformNode left, PTransformNode right, QueryablePipeline pipeline) { + PipelineNode.PTransformNode left, + PipelineNode.PTransformNode right, + QueryablePipeline pipeline) { return pipeline.getEnvironment(left).equals(pipeline.getEnvironment(right)); } @@ -326,26 +333,26 @@ private static boolean compatibleEnvironments( * */ private static boolean canAlwaysFuse( - @SuppressWarnings("unused") PTransformNode flatten, + @SuppressWarnings("unused") PipelineNode.PTransformNode flatten, @SuppressWarnings("unused") Environment environment, - @SuppressWarnings("unused") PCollectionNode candidate, - @SuppressWarnings("unused") Collection stagePCollections, + @SuppressWarnings("unused") PipelineNode.PCollectionNode candidate, + @SuppressWarnings("unused") Collection stagePCollections, @SuppressWarnings("unused") QueryablePipeline pipeline) { return true; } private static boolean cannotFuse( - @SuppressWarnings("unused") PTransformNode cannotFuse, + @SuppressWarnings("unused") PipelineNode.PTransformNode cannotFuse, @SuppressWarnings("unused") Environment environment, - @SuppressWarnings("unused") PCollectionNode candidate, - @SuppressWarnings("unused") Collection stagePCollections, + @SuppressWarnings("unused") PipelineNode.PCollectionNode candidate, + @SuppressWarnings("unused") Collection stagePCollections, @SuppressWarnings("unused") QueryablePipeline pipeline) { return false; } private static boolean noCompatibility( - @SuppressWarnings("unused") PTransformNode self, - @SuppressWarnings("unused") PTransformNode other, + @SuppressWarnings("unused") PipelineNode.PTransformNode self, + @SuppressWarnings("unused") PipelineNode.PTransformNode other, @SuppressWarnings("unused") QueryablePipeline pipeline) { // TODO: There is performance to be gained if the output of a flatten is fused into a stage // where its output is wholly consumed after a fusion break. This requires slightly more @@ -356,10 +363,10 @@ private static boolean noCompatibility( // Things with unknown URNs either execute within their own stage or are executed by the runner. // In either case, assume the system is capable of executing the expressed transform private static boolean unknownTransformFusion( - PTransformNode transform, + PipelineNode.PTransformNode transform, @SuppressWarnings("unused") Environment environment, - @SuppressWarnings("unused") PCollectionNode candidate, - @SuppressWarnings("unused") Collection stagePCollections, + @SuppressWarnings("unused") PipelineNode.PCollectionNode candidate, + @SuppressWarnings("unused") Collection stagePCollections, @SuppressWarnings("unused") QueryablePipeline pipeline) { LOG.debug( "Unknown {} {} will not fuse into an existing {}", @@ -372,8 +379,8 @@ private static boolean unknownTransformFusion( // Things with unknown URNs either execute within their own stage or are executed by the runner. // In either case, assume the system is capable of executing the expressed transform private static boolean unknownTransformCompatibility( - PTransformNode transform, - @SuppressWarnings("unused") PTransformNode other, + PipelineNode.PTransformNode transform, + @SuppressWarnings("unused") PipelineNode.PTransformNode other, @SuppressWarnings("unused") QueryablePipeline pipeline) { LOG.debug( "Unknown {} {} will not root a {} with other {}", diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyPipelineFuser.java similarity index 77% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyPipelineFuser.java index 0d53ca7c18b7..a8aaeff07329 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyPipelineFuser.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; @@ -39,10 +39,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.OutputDeduplicator.DeduplicationResult; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ComparisonChain; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -68,9 +65,9 @@ private GreedyPipelineFuser(Pipeline p) { // Validate that the original pipeline is well-formed. PipelineValidator.validate(p); this.pipeline = QueryablePipeline.forPrimitivesIn(p.getComponents()); - Set unfusedRootNodes = new LinkedHashSet<>(); + Set unfusedRootNodes = new LinkedHashSet<>(); NavigableSet rootConsumers = new TreeSet<>(); - for (PTransformNode pTransformNode : pipeline.getRootTransforms()) { + for (PipelineNode.PTransformNode pTransformNode : pipeline.getRootTransforms()) { // This will usually be a single node, the downstream of an Impulse, but may be of any size DescendantConsumers descendants = getRootConsumers(pTransformNode); unfusedRootNodes.addAll(descendants.getUnfusedNodes()); @@ -101,32 +98,35 @@ public static FusedPipeline fuse(Pipeline p) { * PTransform), where the PTransform consumes input elements on a per-element basis from the * PCollection, represented by a {@link CollectionConsumer}. A sibling set is a collection of * siblings which can execute within a single {@link ExecutableStage}, determined by {@link - * GreedyPCollectionFusers#isCompatible(PTransformNode, PTransformNode, QueryablePipeline)}. + * GreedyPCollectionFusers#isCompatible(PipelineNode.PTransformNode, PipelineNode.PTransformNode, + * QueryablePipeline)}. * *

    While a pending sibling set exists: * *

      *
    • Retrieve a pending sibling set from the front of the queue. *
    • If the pending sibling set has already been created, continue. Each materialized {@link - * PTransformNode} can be consumed by any number of {@link ExecutableStage stages}, but each - * {@link PTransformNode} may only be present in a single stage rooted at a single {@link - * PCollectionNode}, otherwise it will process elements of that {@link PCollectionNode} - * multiple times. + * PipelineNode.PTransformNode} can be consumed by any number of {@link ExecutableStage + * stages}, but each {@link PipelineNode.PTransformNode} may only be present in a single + * stage rooted at a single {@link PipelineNode.PCollectionNode}, otherwise it will process + * elements of that {@link PipelineNode.PCollectionNode} multiple times. *
    • Create a {@link GreedyStageFuser} with those siblings as the initial consuming transforms * of the stage - *
    • For each materialized {@link PCollectionNode}, find all of the descendant in-environment - * consumers. See {@link #getDescendantConsumers(PCollectionNode)} for details. + *
    • For each materialized {@link PipelineNode.PCollectionNode}, find all of the descendant + * in-environment consumers. See {@link + * #getDescendantConsumers(PipelineNode.PCollectionNode)} for details. *
    • Construct all of the sibling sets from the descendant in-environment consumers, and add * them to the queue of sibling sets. *
    */ private FusedPipeline fusePipeline( - Collection initialUnfusedTransforms, + Collection initialUnfusedTransforms, NavigableSet> initialConsumers, Set requirements) { Map consumedCollectionsAndTransforms = new HashMap<>(); Set stages = new LinkedHashSet<>(); - Set unfusedTransforms = new LinkedHashSet<>(initialUnfusedTransforms); + Set unfusedTransforms = + new LinkedHashSet<>(initialUnfusedTransforms); Queue> pendingSiblingSets = new ArrayDeque<>(initialConsumers); while (!pendingSiblingSets.isEmpty()) { @@ -138,7 +138,7 @@ private FusedPipeline fusePipeline( checkState( siblingSet.equals(candidateSiblings) || siblingSet.isEmpty(), "Inconsistent collection of siblings reported for a %s. Initial attempt missed %s", - PCollectionNode.class.getSimpleName(), + PipelineNode.PCollectionNode.class.getSimpleName(), siblingSet); if (siblingSet.isEmpty()) { LOG.debug("Filtered out duplicate stage root {}", candidateSiblings); @@ -152,7 +152,7 @@ private FusedPipeline fusePipeline( consumedCollectionsAndTransforms.put(sibling, stage); } stages.add(stage); - for (PCollectionNode materializedOutput : stage.getOutputPCollections()) { + for (PipelineNode.PCollectionNode materializedOutput : stage.getOutputPCollections()) { // Get all of the descendant consumers of each materialized PCollection, and add them to the // queue of pending siblings. DescendantConsumers descendantConsumers = getDescendantConsumers(materializedOutput); @@ -164,7 +164,7 @@ private FusedPipeline fusePipeline( } } // TODO: Figure out where to store this. - DeduplicationResult deduplicated = + OutputDeduplicator.DeduplicationResult deduplicated = OutputDeduplicator.ensureSingleProducer(pipeline, stages, unfusedTransforms); // TODO: Stages can be fused with each other, if doing so does not introduce duplicate paths // for an element to take through the Pipeline. Compatible siblings can generally be fused, @@ -187,7 +187,7 @@ private FusedPipeline fusePipeline( requirements); } - private DescendantConsumers getRootConsumers(PTransformNode rootNode) { + private DescendantConsumers getRootConsumers(PipelineNode.PTransformNode rootNode) { checkArgument( rootNode.getTransform().getInputsCount() == 0, "Transform %s is not at the root of the graph (consumes %s)", @@ -202,11 +202,11 @@ private DescendantConsumers getRootConsumers(PTransformNode rootNode) { PTransformTranslation.READ_TRANSFORM_URN, rootNode.getId(), pipeline.getEnvironment(rootNode)); - Set unfused = new HashSet<>(); + Set unfused = new HashSet<>(); unfused.add(rootNode); NavigableSet environmentNodes = new TreeSet<>(); // Walk down until the first environments are found, and fuse them as appropriate. - for (PCollectionNode output : pipeline.getOutputPCollections(rootNode)) { + for (PipelineNode.PCollectionNode output : pipeline.getOutputPCollections(rootNode)) { DescendantConsumers descendants = getDescendantConsumers(output); unfused.addAll(descendants.getUnfusedNodes()); environmentNodes.addAll(descendants.getFusibleConsumers()); @@ -215,43 +215,46 @@ private DescendantConsumers getRootConsumers(PTransformNode rootNode) { } /** - * Retrieve all descendant {@link PTransformNode PTransforms} which are executed within an {@link - * Environment}, such that there is a path between this input {@link PCollectionNode} and the - * descendant {@link PTransformNode} with no intermediate {@link PTransformNode} which executes - * within an environment. + * Retrieve all descendant {@link PipelineNode.PTransformNode PTransforms} which are executed + * within an {@link Environment}, such that there is a path between this input {@link + * PipelineNode.PCollectionNode} and the descendant {@link PipelineNode.PTransformNode} with no + * intermediate {@link PipelineNode.PTransformNode} which executes within an environment. * *

    This occurs as follows: * *

      - *
    • For each consumer of the input {@link PCollectionNode}: + *
    • For each consumer of the input {@link PipelineNode.PCollectionNode}: *
        - *
      • If that {@link PTransformNode} executes within an environment, add it to the - * collection of descendants - *
      • If that {@link PTransformNode} does not execute within an environment, for each - * output {@link PCollectionNode} that that {@link PTransformNode} produces, add the - * result of recursively applying this method to that {@link PCollectionNode}. + *
      • If that {@link PipelineNode.PTransformNode} executes within an environment, add it + * to the collection of descendants + *
      • If that {@link PipelineNode.PTransformNode} does not execute within an environment, + * for each output {@link PipelineNode.PCollectionNode} that that {@link + * PipelineNode.PTransformNode} produces, add the result of recursively applying this + * method to that {@link PipelineNode.PCollectionNode}. *
      *
    * - *

    As {@link PCollectionNode PCollections} output by a {@link PTransformNode} that executes - * within an {@link Environment} are not recursively inspected, {@link PTransformNode PTransforms} - * reachable only via a path including that node as an intermediate node cannot be returned as a - * descendant consumer of the original {@link PCollectionNode}. + *

    As {@link PipelineNode.PCollectionNode PCollections} output by a {@link + * PipelineNode.PTransformNode} that executes within an {@link Environment} are not recursively + * inspected, {@link PipelineNode.PTransformNode PTransforms} reachable only via a path including + * that node as an intermediate node cannot be returned as a descendant consumer of the original + * {@link PipelineNode.PCollectionNode}. */ - private DescendantConsumers getDescendantConsumers(PCollectionNode inputPCollection) { - Set unfused = new HashSet<>(); + private DescendantConsumers getDescendantConsumers( + PipelineNode.PCollectionNode inputPCollection) { + Set unfused = new HashSet<>(); NavigableSet downstreamConsumers = new TreeSet<>(); - for (PTransformNode consumer : pipeline.getPerElementConsumers(inputPCollection)) { + for (PipelineNode.PTransformNode consumer : pipeline.getPerElementConsumers(inputPCollection)) { if (pipeline.getEnvironment(consumer).isPresent()) { // The base case: this descendant consumes elements from downstreamConsumers.add(CollectionConsumer.of(inputPCollection, consumer)); } else { LOG.debug( "Adding {} {} to the set of runner-executed transforms", - PTransformNode.class.getSimpleName(), + PipelineNode.PTransformNode.class.getSimpleName(), consumer.getId()); unfused.add(consumer); - for (PCollectionNode output : pipeline.getOutputPCollections(consumer)) { + for (PipelineNode.PCollectionNode output : pipeline.getOutputPCollections(consumer)) { // Recurse to all of the ouput PCollections of this PTransform. DescendantConsumers descendants = getDescendantConsumers(output); unfused.addAll(descendants.getUnfusedNodes()); @@ -265,11 +268,11 @@ private DescendantConsumers getDescendantConsumers(PCollectionNode inputPCollect @AutoValue abstract static class DescendantConsumers { static DescendantConsumers of( - Set unfusible, NavigableSet fusible) { + Set unfusible, NavigableSet fusible) { return new AutoValue_GreedyPipelineFuser_DescendantConsumers(unfusible, fusible); } - abstract Set getUnfusedNodes(); + abstract Set getUnfusedNodes(); abstract NavigableSet getFusibleConsumers(); } @@ -277,12 +280,13 @@ static DescendantConsumers of( /** * The minimum requirement to fuse two {@link CollectionConsumer consumers} as siblings. * - *

    This is the minimum requirement for {@link PTransformNode transforms} to be siblings. - * Different {@link PTransformNode transforms} may have additional restrictions. + *

    This is the minimum requirement for {@link PipelineNode.PTransformNode transforms} to be + * siblings. Different {@link PipelineNode.PTransformNode transforms} may have additional + * restrictions. */ @AutoValue abstract static class SiblingKey { - abstract PCollectionNode getInputCollection(); + abstract PipelineNode.PCollectionNode getInputCollection(); abstract Environment getEnv(); } @@ -293,7 +297,8 @@ abstract static class SiblingKey { * *

    For each set in the returned collection, each of {@link CollectionConsumer consumers} * present consumes from the same {@link PCollection} and is compatible, as determined by {@link - * GreedyPCollectionFusers#isCompatible(PTransformNode, PTransformNode, QueryablePipeline)}. + * GreedyPCollectionFusers#isCompatible(PipelineNode.PTransformNode, PipelineNode.PTransformNode, + * QueryablePipeline)}. * *

    Each input {@link CollectionConsumer} must have an associated {@link Environment}. */ @@ -341,7 +346,8 @@ private NavigableSet> groupSiblings( } private ExecutableStage fuseSiblings(Set mutuallyCompatible) { - PCollectionNode rootCollection = mutuallyCompatible.iterator().next().consumedCollection(); + PipelineNode.PCollectionNode rootCollection = + mutuallyCompatible.iterator().next().consumedCollection(); return GreedyStageFuser.forGrpcPortRead( pipeline, rootCollection, @@ -364,7 +370,7 @@ private static ExecutableStage sanitizeDanglingPTransformInputs(ExecutableStage possibleInputs.add(stage.getInputPCollection().getId()); possibleInputs.addAll( stage.getOutputPCollections().stream() - .map(PCollectionNode::getId) + .map(PipelineNode.PCollectionNode::getId) .collect(Collectors.toSet())); possibleInputs.addAll( stage.getSideInputs().stream() @@ -380,8 +386,9 @@ private static ExecutableStage sanitizeDanglingPTransformInputs(ExecutableStage .filter(in -> !possibleInputs.contains(in)) .collect(Collectors.toSet()); - ImmutableList.Builder pTransformNodesBuilder = ImmutableList.builder(); - for (PTransformNode transformNode : stage.getTransforms()) { + ImmutableList.Builder pTransformNodesBuilder = + ImmutableList.builder(); + for (PipelineNode.PTransformNode transformNode : stage.getTransforms()) { PTransform transform = transformNode.getTransform(); Map validInputs = transform.getInputsMap().entrySet().stream() @@ -398,14 +405,17 @@ private static ExecutableStage sanitizeDanglingPTransformInputs(ExecutableStage pTransformNodesBuilder.add(transformNode); } - ImmutableList pTransformNodes = pTransformNodesBuilder.build(); + ImmutableList pTransformNodes = pTransformNodesBuilder.build(); Components.Builder componentBuilder = stage.getComponents().toBuilder(); // Update the pTransforms in components. componentBuilder .clearTransforms() .putAllTransforms( pTransformNodes.stream() - .collect(Collectors.toMap(PTransformNode::getId, PTransformNode::getTransform))); + .collect( + Collectors.toMap( + PipelineNode.PTransformNode::getId, + PipelineNode.PTransformNode::getTransform))); Map validPCollectionMap = stage.getComponents().getPcollectionsMap().entrySet().stream() .filter(e -> !danglingInputs.contains(e.getKey())) @@ -427,8 +437,9 @@ private static ExecutableStage sanitizeDanglingPTransformInputs(ExecutableStage } /** - * A ({@link PCollectionNode}, {@link PTransformNode}) pair representing a single {@link - * PTransformNode} consuming a single materialized {@link PCollectionNode}. + * A ({@link PipelineNode.PCollectionNode}, {@link PipelineNode.PTransformNode}) pair representing + * a single {@link PipelineNode.PTransformNode} consuming a single materialized {@link + * PipelineNode.PCollectionNode}. * *

    For convenience, {@link CollectionConsumer} implements {@link Comparable}. The natural * ordering of {@link CollectionConsumer} is first by the IDs of the {@link @@ -436,13 +447,14 @@ private static ExecutableStage sanitizeDanglingPTransformInputs(ExecutableStage */ @AutoValue abstract static class CollectionConsumer implements Comparable { - static CollectionConsumer of(PCollectionNode collection, PTransformNode consumer) { + static CollectionConsumer of( + PipelineNode.PCollectionNode collection, PipelineNode.PTransformNode consumer) { return new AutoValue_GreedyPipelineFuser_CollectionConsumer(collection, consumer); } - abstract PCollectionNode consumedCollection(); + abstract PipelineNode.PCollectionNode consumedCollection(); - abstract PTransformNode consumingTransform(); + abstract PipelineNode.PTransformNode consumingTransform(); /** * {@inheritDoc}. diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyStageFuser.java similarity index 76% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyStageFuser.java index 6245577f14d1..21e11a2d35af 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/GreedyStageFuser.java @@ -15,9 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; -import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.util.ArrayDeque; @@ -27,24 +26,22 @@ import java.util.function.Supplier; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * A factory class which produces an {@link ExecutableStage} by attempting to fuse all available - * {@link PCollectionNode PCollections} when it is constructed. + * {@link PipelineNode.PCollectionNode PCollections} when it is constructed. * - *

    A {@link PCollectionNode} is fused into a stage if all of its consumers can be fused into the - * stage. A consumer can be fused into a stage if it is executed within the environment of that - * {@link ExecutableStage}, and receives only per-element inputs. To simplify integration for - * runners, this fuser specifically does not fuse PTransforms which consume side inputs or have user - * state, always making them the root of {@link ExecutableStage}. + *

    A {@link PipelineNode.PCollectionNode} is fused into a stage if all of its consumers can be + * fused into the stage. A consumer can be fused into a stage if it is executed within the + * environment of that {@link ExecutableStage}, and receives only per-element inputs. To simplify + * integration for runners, this fuser specifically does not fuse PTransforms which consume side + * inputs or have user state, always making them the root of {@link ExecutableStage}. * - *

    A {@link PCollectionNode} with consumers that execute in an environment other than a stage is - * materialized, and its consumers execute in independent stages. + *

    A {@link PipelineNode.PCollectionNode} with consumers that execute in an environment other + * than a stage is materialized, and its consumers execute in independent stages. */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) @@ -59,9 +56,9 @@ private GreedyStageFuser() { } /** - * Returns an {@link ExecutableStage} where the initial {@link PTransformNode PTransform} is a - * Remote gRPC Port Read, reading elements from the materialized {@link PCollectionNode - * PCollection}. + * Returns an {@link ExecutableStage} where the initial {@link PipelineNode.PTransformNode + * PTransform} is a Remote gRPC Port Read, reading elements from the materialized {@link + * PipelineNode.PCollectionNode PCollection}. * * @param initialNodes the initial set of sibling transforms to fuse into this node. All of the * transforms must consume the {@code inputPCollection} on a per-element basis, and must all @@ -69,36 +66,36 @@ private GreedyStageFuser() { */ public static ExecutableStage forGrpcPortRead( QueryablePipeline pipeline, - PCollectionNode inputPCollection, - Set initialNodes) { + PipelineNode.PCollectionNode inputPCollection, + Set initialNodes) { checkArgument( !initialNodes.isEmpty(), "%s must contain at least one %s.", GreedyStageFuser.class.getSimpleName(), - PTransformNode.class.getSimpleName()); + PipelineNode.PTransformNode.class.getSimpleName()); // Choose the environment from an arbitrary node. The initial nodes may not be empty for this // subgraph to make any sense, there has to be at least one processor node // (otherwise the stage is gRPC Read -> gRPC Write, which doesn't do anything). Environment environment = getStageEnvironment(pipeline, initialNodes); - ImmutableSet.Builder fusedTransforms = ImmutableSet.builder(); + ImmutableSet.Builder fusedTransforms = ImmutableSet.builder(); fusedTransforms.addAll(initialNodes); Set sideInputs = new LinkedHashSet<>(); Set userStates = new LinkedHashSet<>(); Set timers = new LinkedHashSet<>(); - Set fusedCollections = new LinkedHashSet<>(); - Set materializedPCollections = new LinkedHashSet<>(); + Set fusedCollections = new LinkedHashSet<>(); + Set materializedPCollections = new LinkedHashSet<>(); - Queue fusionCandidates = new ArrayDeque<>(); - for (PTransformNode initialConsumer : initialNodes) { + Queue fusionCandidates = new ArrayDeque<>(); + for (PipelineNode.PTransformNode initialConsumer : initialNodes) { fusionCandidates.addAll(pipeline.getOutputPCollections(initialConsumer)); sideInputs.addAll(pipeline.getSideInputs(initialConsumer)); userStates.addAll(pipeline.getUserStates(initialConsumer)); timers.addAll(pipeline.getTimers(initialConsumer)); } while (!fusionCandidates.isEmpty()) { - PCollectionNode candidate = fusionCandidates.poll(); + PipelineNode.PCollectionNode candidate = fusionCandidates.poll(); if (fusedCollections.contains(candidate) || materializedPCollections.contains(candidate)) { // This should generally mean we get to a Flatten via multiple paths through the graph and // we've already determined what to do with the output. @@ -119,7 +116,7 @@ public static ExecutableStage forGrpcPortRead( // All of the consumers of the candidate PCollection can be fused into this stage. Do so. fusedCollections.add(candidate); fusedTransforms.addAll(pipeline.getPerElementConsumers(candidate)); - for (PTransformNode consumer : pipeline.getPerElementConsumers(candidate)) { + for (PipelineNode.PTransformNode consumer : pipeline.getPerElementConsumers(candidate)) { // The outputs of every transform fused into this stage must be either materialized or // themselves fused away, so add them to the set of candidates. fusionCandidates.addAll(pipeline.getOutputPCollections(consumer)); @@ -143,18 +140,18 @@ public static ExecutableStage forGrpcPortRead( timers, fusedTransforms.build(), materializedPCollections, - DEFAULT_WIRE_CODER_SETTINGS); + ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS); } private static Environment getStageEnvironment( - QueryablePipeline pipeline, Set initialNodes) { + QueryablePipeline pipeline, Set initialNodes) { Supplier missingEnv = () -> new IllegalArgumentException( String.format( "%s must be populated on all %s in a %s", Environment.class.getSimpleName(), - PTransformNode.class.getSimpleName(), + PipelineNode.PTransformNode.class.getSimpleName(), GreedyStageFuser.class.getSimpleName())); Environment env = pipeline.getEnvironment(initialNodes.iterator().next()).orElseThrow(missingEnv); @@ -172,10 +169,10 @@ private static Environment getStageEnvironment( private static PCollectionFusibility canFuse( QueryablePipeline pipeline, - PCollectionNode candidate, + PipelineNode.PCollectionNode candidate, Environment environment, - Set fusedPCollections) { - for (PTransformNode consumer : pipeline.getPerElementConsumers(candidate)) { + Set fusedPCollections) { + for (PipelineNode.PTransformNode consumer : pipeline.getPerElementConsumers(candidate)) { if (anyInputsSideInputs(consumer, pipeline) || !GreedyPCollectionFusers.canFuse( consumer, environment, candidate, fusedPCollections, pipeline)) { @@ -199,11 +196,13 @@ private enum PCollectionFusibility { FUSE, } - private static boolean anyInputsSideInputs(PTransformNode consumer, QueryablePipeline pipeline) { + private static boolean anyInputsSideInputs( + PipelineNode.PTransformNode consumer, QueryablePipeline pipeline) { for (String inputPCollectionId : consumer.getTransform().getInputsMap().values()) { RunnerApi.PCollection pCollection = pipeline.getComponents().getPcollectionsMap().get(inputPCollectionId); - PCollectionNode pCollectionNode = PipelineNode.pCollection(inputPCollectionId, pCollection); + PipelineNode.PCollectionNode pCollectionNode = + PipelineNode.pCollection(inputPCollectionId, pCollection); if (!pipeline.getSingletonConsumers(pCollectionNode).isEmpty()) { return true; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ImmutableExecutableStage.java similarity index 80% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ImmutableExecutableStage.java index 56bd2ef6f4d3..4dc649e02ad5 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ImmutableExecutableStage.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import com.google.auto.value.AutoValue; import java.util.Collection; @@ -23,8 +23,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; /** An {@link ExecutableStage} which is constructed with all of its initial state. */ @@ -33,12 +31,12 @@ public abstract class ImmutableExecutableStage implements ExecutableStage { public static ImmutableExecutableStage ofFullComponents( Components components, Environment environment, - PCollectionNode input, + PipelineNode.PCollectionNode input, Collection sideInputs, Collection userStates, Collection timers, - Collection transforms, - Collection outputs, + Collection transforms, + Collection outputs, Collection wireCoderSettings) { Components prunedComponents = components @@ -46,7 +44,10 @@ public static ImmutableExecutableStage ofFullComponents( .clearTransforms() .putAllTransforms( transforms.stream() - .collect(Collectors.toMap(PTransformNode::getId, PTransformNode::getTransform))) + .collect( + Collectors.toMap( + PipelineNode.PTransformNode::getId, + PipelineNode.PTransformNode::getTransform))) .build(); return of( prunedComponents, @@ -63,12 +64,12 @@ public static ImmutableExecutableStage ofFullComponents( public static ImmutableExecutableStage of( Components components, Environment environment, - PCollectionNode input, + PipelineNode.PCollectionNode input, Collection sideInputs, Collection userStates, Collection timers, - Collection transforms, - Collection outputs, + Collection transforms, + Collection outputs, Collection wireCoderSettings) { return new AutoValue_ImmutableExecutableStage( components, @@ -90,7 +91,7 @@ public static ImmutableExecutableStage of( public abstract Environment getEnvironment(); @Override - public abstract PCollectionNode getInputPCollection(); + public abstract PipelineNode.PCollectionNode getInputPCollection(); @Override public abstract Collection getSideInputs(); @@ -102,10 +103,10 @@ public static ImmutableExecutableStage of( public abstract Collection getTimers(); @Override - public abstract Collection getTransforms(); + public abstract Collection getTransforms(); @Override - public abstract Collection getOutputPCollections(); + public abstract Collection getOutputPCollections(); @Override public abstract Collection getWireCoderSettings(); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/Networks.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/Networks.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/Networks.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/Networks.java index d9fdd8140d62..772a8a682edd 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/Networks.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/Networks.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/OutputDeduplicator.java similarity index 67% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/OutputDeduplicator.java index d23bb2719020..14df79941bec 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/OutputDeduplicator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -34,17 +34,15 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SyntheticComponents; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SyntheticComponents; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; import org.checkerframework.checker.nullness.qual.Nullable; /** - * Utilities to insert synthetic {@link PCollectionNode PCollections} for {@link PCollection - * PCollections} which are produced by multiple independently executable stages. + * Utilities to insert synthetic {@link PipelineNode.PCollectionNode PCollections} for {@link + * PCollection PCollections} which are produced by multiple independently executable stages. */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) @@ -62,16 +60,17 @@ class OutputDeduplicator { static DeduplicationResult ensureSingleProducer( QueryablePipeline pipeline, Collection stages, - Collection unfusedTransforms) { + Collection unfusedTransforms) { RunnerApi.Components.Builder unzippedComponents = pipeline.getComponents().toBuilder(); - Multimap pcollectionProducers = + Multimap pcollectionProducers = getProducers(pipeline, stages, unfusedTransforms); - Multimap requiresNewOutput = HashMultimap.create(); + Multimap requiresNewOutput = + HashMultimap.create(); // Create a synthetic PCollection for each of these nodes. The transforms in the runner // portion of the graph that creates them should be replaced in the result components. The // ExecutableStage must also be rewritten to have updated outputs and transforms. - for (Map.Entry> collectionProducer : + for (Map.Entry> collectionProducer : pcollectionProducers.asMap().entrySet()) { if (collectionProducer.getValue().size() > 1) { for (StageOrTransform producer : collectionProducer.getValue()) { @@ -81,17 +80,17 @@ static DeduplicationResult ensureSingleProducer( } Map updatedStages = new LinkedHashMap<>(); - Map updatedTransforms = new LinkedHashMap<>(); - Multimap originalToPartial = HashMultimap.create(); - for (Map.Entry> deduplicationTargets : - requiresNewOutput.asMap().entrySet()) { + Map updatedTransforms = new LinkedHashMap<>(); + Multimap originalToPartial = HashMultimap.create(); + for (Map.Entry> + deduplicationTargets : requiresNewOutput.asMap().entrySet()) { if (deduplicationTargets.getKey().getStage() != null) { StageDeduplication deduplication = deduplicatePCollections( deduplicationTargets.getKey().getStage(), deduplicationTargets.getValue(), unzippedComponents::containsPcollections); - for (Entry originalToPartialReplacement : + for (Entry originalToPartialReplacement : deduplication.getOriginalToPartialPCollections().entrySet()) { originalToPartial.put( originalToPartialReplacement.getKey(), originalToPartialReplacement.getValue()); @@ -107,7 +106,7 @@ static DeduplicationResult ensureSingleProducer( deduplicationTargets.getKey().getTransform(), deduplicationTargets.getValue(), unzippedComponents::containsPcollections); - for (Entry originalToPartialReplacement : + for (Entry originalToPartialReplacement : deduplication.getOriginalToPartialPCollections().entrySet()) { originalToPartial.put( originalToPartialReplacement.getKey(), originalToPartialReplacement.getValue()); @@ -124,12 +123,12 @@ static DeduplicationResult ensureSingleProducer( "%s with no %s or %s", StageOrTransform.class.getSimpleName(), ExecutableStage.class.getSimpleName(), - PTransformNode.class.getSimpleName())); + PipelineNode.PTransformNode.class.getSimpleName())); } } - Set introducedFlattens = new LinkedHashSet<>(); - for (Map.Entry> partialFlattenTargets : + Set introducedFlattens = new LinkedHashSet<>(); + for (Map.Entry> partialFlattenTargets : originalToPartial.asMap().entrySet()) { String flattenId = SyntheticComponents.uniqueId("unzipped_flatten", unzippedComponents::containsTransforms); @@ -148,27 +147,29 @@ static DeduplicationResult ensureSingleProducer( abstract static class DeduplicationResult { private static DeduplicationResult of( RunnerApi.Components components, - Set introducedTransforms, + Set introducedTransforms, Map stages, - Map unfused) { + Map unfused) { return new AutoValue_OutputDeduplicator_DeduplicationResult( components, introducedTransforms, stages, unfused); } abstract RunnerApi.Components getDeduplicatedComponents(); - abstract Set getIntroducedTransforms(); + abstract Set getIntroducedTransforms(); abstract Map getDeduplicatedStages(); - abstract Map getDeduplicatedTransforms(); + abstract Map getDeduplicatedTransforms(); } private static PTransform createFlattenOfPartials( - String transformId, String outputId, Collection generatedInputs) { + String transformId, + String outputId, + Collection generatedInputs) { PTransform.Builder newFlattenBuilder = PTransform.newBuilder(); int i = 0; - for (PCollectionNode generatedInput : generatedInputs) { + for (PipelineNode.PCollectionNode generatedInput : generatedInputs) { String localInputId = String.format("input_%s", i); i++; newFlattenBuilder.putInputs(localInputId, generatedInput.getId()); @@ -183,22 +184,23 @@ private static PTransform createFlattenOfPartials( } /** - * Returns the map from each {@link PCollectionNode} produced by any of the {@link ExecutableStage - * stages} or {@link PTransformNode transforms} to all of the {@link ExecutableStage stages} or - * {@link PTransformNode transforms} that produce it. + * Returns the map from each {@link PipelineNode.PCollectionNode} produced by any of the {@link + * ExecutableStage stages} or {@link PipelineNode.PTransformNode transforms} to all of the {@link + * ExecutableStage stages} or {@link PipelineNode.PTransformNode transforms} that produce it. */ - private static Multimap getProducers( + private static Multimap getProducers( QueryablePipeline pipeline, Iterable stages, - Iterable unfusedTransforms) { - Multimap pcollectionProducers = HashMultimap.create(); + Iterable unfusedTransforms) { + Multimap pcollectionProducers = + HashMultimap.create(); for (ExecutableStage stage : stages) { - for (PCollectionNode output : stage.getOutputPCollections()) { + for (PipelineNode.PCollectionNode output : stage.getOutputPCollections()) { pcollectionProducers.put(output, StageOrTransform.stage(stage)); } } - for (PTransformNode unfused : unfusedTransforms) { - for (PCollectionNode output : pipeline.getOutputPCollections(unfused)) { + for (PipelineNode.PTransformNode unfused : unfusedTransforms) { + for (PipelineNode.PCollectionNode output : pipeline.getOutputPCollections(unfused)) { pcollectionProducers.put(output, StageOrTransform.transform(unfused)); } } @@ -206,10 +208,10 @@ private static Multimap getProducers( } private static PTransformDeduplication deduplicatePCollections( - PTransformNode transform, - Collection duplicates, + PipelineNode.PTransformNode transform, + Collection duplicates, Predicate existingPCollectionIds) { - Map unzippedOutputs = + Map unzippedOutputs = createPartialPCollections(duplicates, existingPCollectionIds); PTransform pTransform = updateOutputs(transform.getTransform(), unzippedOutputs); return PTransformDeduplication.of( @@ -219,21 +221,22 @@ private static PTransformDeduplication deduplicatePCollections( @AutoValue abstract static class PTransformDeduplication { public static PTransformDeduplication of( - PTransformNode updatedTransform, Map originalToPartial) { + PipelineNode.PTransformNode updatedTransform, + Map originalToPartial) { return new AutoValue_OutputDeduplicator_PTransformDeduplication( updatedTransform, originalToPartial); } - abstract PTransformNode getUpdatedTransform(); + abstract PipelineNode.PTransformNode getUpdatedTransform(); - abstract Map getOriginalToPartialPCollections(); + abstract Map getOriginalToPartialPCollections(); } private static StageDeduplication deduplicatePCollections( ExecutableStage stage, - Collection duplicates, + Collection duplicates, Predicate existingPCollectionIds) { - Map unzippedOutputs = + Map unzippedOutputs = createPartialPCollections(duplicates, existingPCollectionIds); ExecutableStage updatedStage = deduplicateStageOutput(stage, unzippedOutputs); return StageDeduplication.of(updatedStage, unzippedOutputs); @@ -242,32 +245,36 @@ private static StageDeduplication deduplicatePCollections( @AutoValue abstract static class StageDeduplication { public static StageDeduplication of( - ExecutableStage updatedStage, Map originalToPartial) { + ExecutableStage updatedStage, Map originalToPartial) { return new AutoValue_OutputDeduplicator_StageDeduplication(updatedStage, originalToPartial); } abstract ExecutableStage getUpdatedStage(); - abstract Map getOriginalToPartialPCollections(); + abstract Map getOriginalToPartialPCollections(); } /** - * Returns a {@link Map} from the ID of a {@link PCollectionNode PCollection} to a {@link - * PCollectionNode} that contains part of that {@link PCollectionNode PCollection}. + * Returns a {@link Map} from the ID of a {@link PipelineNode.PCollectionNode PCollection} to a + * {@link PipelineNode.PCollectionNode} that contains part of that {@link + * PipelineNode.PCollectionNode PCollection}. */ - private static Map createPartialPCollections( - Collection duplicates, Predicate existingPCollectionIds) { - Map unzippedOutputs = new LinkedHashMap<>(); + private static Map createPartialPCollections( + Collection duplicates, + Predicate existingPCollectionIds) { + Map unzippedOutputs = new LinkedHashMap<>(); Predicate existingOrNewIds = existingPCollectionIds.or( id -> - unzippedOutputs.values().stream().map(PCollectionNode::getId).anyMatch(id::equals)); - for (PCollectionNode duplicateOutput : duplicates) { + unzippedOutputs.values().stream() + .map(PipelineNode.PCollectionNode::getId) + .anyMatch(id::equals)); + for (PipelineNode.PCollectionNode duplicateOutput : duplicates) { String id = SyntheticComponents.uniqueId(duplicateOutput.getId(), existingOrNewIds); PCollection partial = duplicateOutput.getPCollection().toBuilder().setUniqueName(id).build(); // Check to make sure there is only one duplicated output with the same id - which ensures we // only introduce one 'partial output' per producer of that output. - PCollectionNode alreadyDeduplicated = + PipelineNode.PCollectionNode alreadyDeduplicated = unzippedOutputs.put(duplicateOutput.getId(), PipelineNode.pCollection(id, partial)); checkArgument(alreadyDeduplicated == null, "a duplicate should only appear once per stage"); } @@ -275,19 +282,19 @@ private static Map createPartialPCollections( } /** - * Returns an {@link ExecutableStage} where all of the {@link PCollectionNode PCollections} - * matching the original are replaced with the introduced partial {@link PCollection} in all - * references made within the {@link ExecutableStage}. + * Returns an {@link ExecutableStage} where all of the {@link PipelineNode.PCollectionNode + * PCollections} matching the original are replaced with the introduced partial {@link + * PCollection} in all references made within the {@link ExecutableStage}. */ private static ExecutableStage deduplicateStageOutput( - ExecutableStage stage, Map originalToPartial) { - Collection updatedTransforms = new ArrayList<>(); - for (PTransformNode transform : stage.getTransforms()) { + ExecutableStage stage, Map originalToPartial) { + Collection updatedTransforms = new ArrayList<>(); + for (PipelineNode.PTransformNode transform : stage.getTransforms()) { PTransform updatedTransform = updateOutputs(transform.getTransform(), originalToPartial); updatedTransforms.add(PipelineNode.pTransform(transform.getId(), updatedTransform)); } - Collection updatedOutputs = new ArrayList<>(); - for (PCollectionNode output : stage.getOutputPCollections()) { + Collection updatedOutputs = new ArrayList<>(); + for (PipelineNode.PCollectionNode output : stage.getOutputPCollections()) { updatedOutputs.add(originalToPartial.getOrDefault(output.getId(), output)); } RunnerApi.Components updatedStageComponents = @@ -297,11 +304,16 @@ private static ExecutableStage deduplicateStageOutput( .clearTransforms() .putAllTransforms( updatedTransforms.stream() - .collect(Collectors.toMap(PTransformNode::getId, PTransformNode::getTransform))) + .collect( + Collectors.toMap( + PipelineNode.PTransformNode::getId, + PipelineNode.PTransformNode::getTransform))) .putAllPcollections( originalToPartial.values().stream() .collect( - Collectors.toMap(PCollectionNode::getId, PCollectionNode::getPCollection))) + Collectors.toMap( + PipelineNode.PCollectionNode::getId, + PipelineNode.PCollectionNode::getPCollection))) .build(); return ImmutableExecutableStage.of( updatedStageComponents, @@ -321,7 +333,7 @@ private static ExecutableStage deduplicateStageOutput( * newPCollection}. */ private static PTransform updateOutputs( - PTransform transform, Map originalToPartial) { + PTransform transform, Map originalToPartial) { PTransform.Builder updatedTransformBuilder = transform.toBuilder(); for (Map.Entry output : transform.getOutputsMap().entrySet()) { if (originalToPartial.containsKey(output.getValue())) { @@ -339,12 +351,12 @@ public static StageOrTransform stage(ExecutableStage stage) { return new AutoValue_OutputDeduplicator_StageOrTransform(stage, null); } - public static StageOrTransform transform(PTransformNode transform) { + public static StageOrTransform transform(PipelineNode.PTransformNode transform) { return new AutoValue_OutputDeduplicator_StageOrTransform(null, transform); } abstract @Nullable ExecutableStage getStage(); - abstract @Nullable PTransformNode getTransform(); + abstract PipelineNode.@Nullable PTransformNode getTransform(); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PCollectionOutputTagVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PCollectionOutputTagVisitor.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PCollectionOutputTagVisitor.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PCollectionOutputTagVisitor.java index 0d40154b09f2..5d81fb0e7ea8 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PCollectionOutputTagVisitor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PCollectionOutputTagVisitor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import java.util.AbstractMap.SimpleEntry; import java.util.Map; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineNode.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PipelineNode.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineNode.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PipelineNode.java index 12817dc20aa9..dd6bfa60f65b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineNode.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PipelineNode.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import com.google.auto.value.AutoValue; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PipelineValidator.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineValidator.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PipelineValidator.java index 022548aaff04..9c7800ee9a67 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineValidator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/PipelineValidator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -31,8 +31,8 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.TestStreamPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProjectionProducerVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProjectionProducerVisitor.java similarity index 96% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProjectionProducerVisitor.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProjectionProducerVisitor.java index 8a8bf125544f..998c6be0b3da 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProjectionProducerVisitor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProjectionProducerVisitor.java @@ -15,11 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import java.util.Map; import org.apache.beam.sdk.Pipeline.PipelineVisitor; -import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.schemas.FieldAccessDescriptor; import org.apache.beam.sdk.schemas.ProjectionProducer; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProjectionPushdownOptimizer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProjectionPushdownOptimizer.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProjectionPushdownOptimizer.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProjectionPushdownOptimizer.java index 3713a16b309e..2ad4e7daf7e0 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProjectionPushdownOptimizer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProjectionPushdownOptimizer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import java.util.AbstractMap.SimpleEntry; import java.util.Map; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProtoOverrides.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProtoOverrides.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProtoOverrides.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProtoOverrides.java index 89b1e3365792..3052a1f6d60c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ProtoOverrides.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/ProtoOverrides.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/QueryablePipeline.java similarity index 68% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/QueryablePipeline.java index 07d78240446c..f416cc261db4 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/QueryablePipeline.java @@ -15,25 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; - -import static org.apache.beam.runners.core.construction.PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.COMBINE_PER_KEY_EXTRACT_OUTPUTS_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.COMBINE_PER_KEY_MERGE_ACCUMULATORS_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.COMBINE_PER_KEY_PRECOMBINE_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.CREATE_VIEW_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.MAP_WINDOWS_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.READ_TRANSFORM_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN; -import static org.apache.beam.runners.core.construction.PTransformTranslation.TEST_STREAM_TRANSFORM_URN; +package org.apache.beam.sdk.util.construction.graph; + import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.util.ArrayDeque; @@ -56,11 +39,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.NativeTransforms; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.NativeTransforms; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @@ -112,13 +93,13 @@ public static QueryablePipeline forTransforms( /** * The {@link Pipeline} represented by a {@link Network}. * - *

    This is a directed bipartite graph consisting of {@link PTransformNode PTransformNodes} and - * {@link PCollectionNode PCollectionNodes}. Each {@link PCollectionNode} has exactly one in edge, - * and an arbitrary number of out edges. Each {@link PTransformNode} has an arbitrary number of in - * and out edges. + *

    This is a directed bipartite graph consisting of {@link PipelineNode.PTransformNode + * PTransformNodes} and {@link PipelineNode.PCollectionNode PCollectionNodes}. Each {@link + * PipelineNode.PCollectionNode} has exactly one in edge, and an arbitrary number of out edges. + * Each {@link PipelineNode.PTransformNode} has an arbitrary number of in and out edges. * - *

    Parallel edges are permitted, as a {@link PCollectionNode} can be consumed by a single - * {@link PTransformNode} any number of times with different local names. + *

    Parallel edges are permitted, as a {@link PipelineNode.PCollectionNode} can be consumed by a + * single {@link PipelineNode.PTransformNode} any number of times with different local names. */ private final Network pipelineNetwork; @@ -164,23 +145,23 @@ static Collection getPrimitiveTransformIds(RunnerApi.Components componen private static final Set PRIMITIVE_URNS = ImmutableSet.of( - PAR_DO_TRANSFORM_URN, - FLATTEN_TRANSFORM_URN, - GROUP_BY_KEY_TRANSFORM_URN, - IMPULSE_TRANSFORM_URN, - ASSIGN_WINDOWS_TRANSFORM_URN, - TEST_STREAM_TRANSFORM_URN, - MAP_WINDOWS_TRANSFORM_URN, - READ_TRANSFORM_URN, - CREATE_VIEW_TRANSFORM_URN, - COMBINE_PER_KEY_PRECOMBINE_TRANSFORM_URN, - COMBINE_PER_KEY_MERGE_ACCUMULATORS_TRANSFORM_URN, - COMBINE_PER_KEY_EXTRACT_OUTPUTS_TRANSFORM_URN, - SPLITTABLE_PAIR_WITH_RESTRICTION_URN, - SPLITTABLE_PROCESS_KEYED_URN, - SPLITTABLE_PROCESS_ELEMENTS_URN, - SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN, - SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN); + PTransformTranslation.PAR_DO_TRANSFORM_URN, + PTransformTranslation.FLATTEN_TRANSFORM_URN, + PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, + PTransformTranslation.IMPULSE_TRANSFORM_URN, + PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN, + PTransformTranslation.TEST_STREAM_TRANSFORM_URN, + PTransformTranslation.MAP_WINDOWS_TRANSFORM_URN, + PTransformTranslation.READ_TRANSFORM_URN, + PTransformTranslation.CREATE_VIEW_TRANSFORM_URN, + PTransformTranslation.COMBINE_PER_KEY_PRECOMBINE_TRANSFORM_URN, + PTransformTranslation.COMBINE_PER_KEY_MERGE_ACCUMULATORS_TRANSFORM_URN, + PTransformTranslation.COMBINE_PER_KEY_EXTRACT_OUTPUTS_TRANSFORM_URN, + PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN, + PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN, + PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN, + PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN, + PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN); /** Returns true if the provided transform is a primitive. */ private static boolean isPrimitiveTransform(PTransform transform) { @@ -192,22 +173,22 @@ private MutableNetwork buildNetwork( Collection transformIds, Components components) { MutableNetwork network = NetworkBuilder.directed().allowsParallelEdges(true).allowsSelfLoops(false).build(); - Set unproducedCollections = new HashSet<>(); + Set unproducedCollections = new HashSet<>(); for (String transformId : transformIds) { PTransform transform = components.getTransformsOrThrow(transformId); - PTransformNode transformNode = + PipelineNode.PTransformNode transformNode = PipelineNode.pTransform(transformId, this.components.getTransformsOrThrow(transformId)); network.addNode(transformNode); for (String produced : transform.getOutputsMap().values()) { - PCollectionNode producedNode = + PipelineNode.PCollectionNode producedNode = PipelineNode.pCollection(produced, components.getPcollectionsOrThrow(produced)); network.addNode(producedNode); network.addEdge(transformNode, producedNode, new PerElementEdge()); checkArgument( network.inDegree(producedNode) == 1, "A %s should have exactly one producing %s, but found %s:\nPCollection:\n%s\nProducers:\n%s", - PCollectionNode.class.getSimpleName(), - PTransformNode.class.getSimpleName(), + PipelineNode.PCollectionNode.class.getSimpleName(), + PipelineNode.PTransformNode.class.getSimpleName(), network.predecessors(producedNode).size(), producedNode, network.predecessors(producedNode)); @@ -217,7 +198,7 @@ private MutableNetwork buildNetwork( // This loop may add an edge between the consumed PCollection and the current PTransform. // The local name of the transform must be used to determine the type of edge. String pcollectionId = consumed.getValue(); - PCollectionNode consumedNode = + PipelineNode.PCollectionNode consumedNode = PipelineNode.pCollection( pcollectionId, this.components.getPcollectionsOrThrow(pcollectionId)); if (network.addNode(consumedNode)) { @@ -234,25 +215,25 @@ private MutableNetwork buildNetwork( checkArgument( unproducedCollections.isEmpty(), "%ss %s were consumed but never produced", - PCollectionNode.class.getSimpleName(), + PipelineNode.PCollectionNode.class.getSimpleName(), unproducedCollections); return network; } - public Collection getTransforms() { + public Collection getTransforms() { return pipelineNetwork.nodes().stream() - .filter(PTransformNode.class::isInstance) - .map(PTransformNode.class::cast) + .filter(PipelineNode.PTransformNode.class::isInstance) + .map(PipelineNode.PTransformNode.class::cast) .collect(Collectors.toList()); } - public Iterable getTopologicallyOrderedTransforms() { + public Iterable getTopologicallyOrderedTransforms() { return StreamSupport.stream( Networks.topologicalOrder(pipelineNetwork, Comparator.comparing(PipelineNode::getId)) .spliterator(), false) - .filter(PTransformNode.class::isInstance) - .map(PTransformNode.class::cast) + .filter(PipelineNode.PTransformNode.class::isInstance) + .map(PipelineNode.PTransformNode.class::cast) .collect(Collectors.toList()); } @@ -260,66 +241,71 @@ public Iterable getTopologicallyOrderedTransforms() { * Get the transforms that are roots of this {@link QueryablePipeline}. These are all nodes which * have no input {@link PCollection}. */ - public Set getRootTransforms() { + public Set getRootTransforms() { return pipelineNetwork.nodes().stream() .filter(pipelineNode -> pipelineNetwork.inEdges(pipelineNode).isEmpty()) - .map(pipelineNode -> (PTransformNode) pipelineNode) + .map(pipelineNode -> (PipelineNode.PTransformNode) pipelineNode) .collect(Collectors.toSet()); } - public PTransformNode getProducer(PCollectionNode pcollection) { - return (PTransformNode) Iterables.getOnlyElement(pipelineNetwork.predecessors(pcollection)); + public PipelineNode.PTransformNode getProducer(PipelineNode.PCollectionNode pcollection) { + return (PipelineNode.PTransformNode) + Iterables.getOnlyElement(pipelineNetwork.predecessors(pcollection)); } /** - * Get all of the {@link PTransformNode PTransforms} which consume the provided {@link - * PCollectionNode} on a per-element basis. + * Get all of the {@link PipelineNode.PTransformNode PTransforms} which consume the provided + * {@link PipelineNode.PCollectionNode} on a per-element basis. * - *

    If a {@link PTransformNode} consumes a {@link PCollectionNode} on a per-element basis one or - * more times, it will appear a single time in the result. + *

    If a {@link PipelineNode.PTransformNode} consumes a {@link PipelineNode.PCollectionNode} on + * a per-element basis one or more times, it will appear a single time in the result. * - *

    In theory, a transform may consume a single {@link PCollectionNode} in both a per-element - * and singleton manner. If this is the case, the transform node is included in the result, as it - * does consume the {@link PCollectionNode} on a per-element basis. + *

    In theory, a transform may consume a single {@link PipelineNode.PCollectionNode} in both a + * per-element and singleton manner. If this is the case, the transform node is included in the + * result, as it does consume the {@link PipelineNode.PCollectionNode} on a per-element basis. */ - public Set getPerElementConsumers(PCollectionNode pCollection) { + public Set getPerElementConsumers( + PipelineNode.PCollectionNode pCollection) { return pipelineNetwork.successors(pCollection).stream() .filter( consumer -> pipelineNetwork.edgesConnecting(pCollection, consumer).stream() .anyMatch(PipelineEdge::isPerElement)) - .map(pipelineNode -> (PTransformNode) pipelineNode) + .map(pipelineNode -> (PipelineNode.PTransformNode) pipelineNode) .collect(Collectors.toSet()); } /** - * Same as {@link #getPerElementConsumers(PCollectionNode)}, but returns transforms that consume - * the collection as a singleton. + * Same as {@link #getPerElementConsumers(PipelineNode.PCollectionNode)}, but returns transforms + * that consume the collection as a singleton. */ - public Set getSingletonConsumers(PCollectionNode pCollection) { + public Set getSingletonConsumers( + PipelineNode.PCollectionNode pCollection) { return pipelineNetwork.successors(pCollection).stream() .filter( consumer -> pipelineNetwork.edgesConnecting(pCollection, consumer).stream() .anyMatch(edge -> !edge.isPerElement())) - .map(pipelineNode -> (PTransformNode) pipelineNode) + .map(pipelineNode -> (PipelineNode.PTransformNode) pipelineNode) .collect(Collectors.toSet()); } /** - * Gets each {@link PCollectionNode} that the provided {@link PTransformNode} consumes on a - * per-element basis. + * Gets each {@link PipelineNode.PCollectionNode} that the provided {@link + * PipelineNode.PTransformNode} consumes on a per-element basis. */ - public Set getPerElementInputPCollections(PTransformNode ptransform) { + public Set getPerElementInputPCollections( + PipelineNode.PTransformNode ptransform) { return pipelineNetwork.inEdges(ptransform).stream() .filter(PipelineEdge::isPerElement) - .map(edge -> (PCollectionNode) pipelineNetwork.incidentNodes(edge).source()) + .map(edge -> (PipelineNode.PCollectionNode) pipelineNetwork.incidentNodes(edge).source()) .collect(Collectors.toSet()); } - public Set getOutputPCollections(PTransformNode ptransform) { + public Set getOutputPCollections( + PipelineNode.PTransformNode ptransform) { return pipelineNetwork.successors(ptransform).stream() - .map(pipelineNode -> (PCollectionNode) pipelineNode) + .map(pipelineNode -> (PipelineNode.PCollectionNode) pipelineNode) .collect(Collectors.toSet()); } @@ -331,7 +317,7 @@ public Components getComponents() { * Returns the {@link SideInputReference SideInputReferences} that the provided transform consumes * as side inputs. */ - public Collection getSideInputs(PTransformNode transform) { + public Collection getSideInputs(PipelineNode.PTransformNode transform) { return getLocalSideInputNames(transform.getTransform()).stream() .map( localName -> { @@ -347,7 +333,7 @@ public Collection getSideInputs(PTransformNode transform) { .collect(Collectors.toSet()); } - public Collection getUserStates(PTransformNode transform) { + public Collection getUserStates(PipelineNode.PTransformNode transform) { return getLocalUserStateNames(transform.getTransform()).stream() .map( localName -> { @@ -374,7 +360,7 @@ public Collection getUserStates(PTransformNode transform) { .collect(Collectors.toSet()); } - public Collection getTimers(PTransformNode transform) { + public Collection getTimers(PipelineNode.PTransformNode transform) { return getLocalTimerNames(transform.getTransform()).stream() .map( localName -> { @@ -387,10 +373,12 @@ public Collection getTimers(PTransformNode transform) { } private Set getLocalSideInputNames(PTransform transform) { - if (PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn()) - || SPLITTABLE_PAIR_WITH_RESTRICTION_URN.equals(transform.getSpec().getUrn()) - || SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN.equals(transform.getSpec().getUrn()) - || SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.equals( + if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN.equals( + transform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN.equals( + transform.getSpec().getUrn()) + || PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.equals( transform.getSpec().getUrn())) { try { return ParDoPayload.parseFrom(transform.getSpec().getPayload()).getSideInputsMap().keySet(); @@ -403,7 +391,7 @@ private Set getLocalSideInputNames(PTransform transform) { } private Set getLocalUserStateNames(PTransform transform) { - if (PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) { + if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) { try { return ParDoPayload.parseFrom(transform.getSpec().getPayload()).getStateSpecsMap().keySet(); } catch (InvalidProtocolBufferException e) { @@ -415,7 +403,7 @@ private Set getLocalUserStateNames(PTransform transform) { } private Set getLocalTimerNames(PTransform transform) { - if (PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) { + if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) { try { return ParDoPayload.parseFrom(transform.getSpec().getPayload()) .getTimerFamilySpecsMap() @@ -428,7 +416,7 @@ private Set getLocalTimerNames(PTransform transform) { } } - public Optional getEnvironment(PTransformNode parDo) { + public Optional getEnvironment(PipelineNode.PTransformNode parDo) { return Environments.getEnvironment(parDo.getId(), components); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SideInputReference.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/SideInputReference.java similarity index 92% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SideInputReference.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/SideInputReference.java index 80584bf56d46..8f16e0ac6d6c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SideInputReference.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/SideInputReference.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import com.google.auto.value.AutoValue; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; /** diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/SplittableParDoExpander.java similarity index 96% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/SplittableParDoExpander.java index 0edcc1ab13e6..66ab705e28e9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/SplittableParDoExpander.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import com.google.auto.value.AutoValue; import java.io.IOException; @@ -29,10 +29,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.graph.ProtoOverrides.TransformReplacement; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; /** @@ -69,7 +68,7 @@ public class SplittableParDoExpander { *

    Specifically this transform ensures that initial splitting is performed and that the sizing * information is available to the runner if it chooses to inspect it. */ - public static TransformReplacement createSizedReplacement() { + public static ProtoOverrides.TransformReplacement createSizedReplacement() { return SizedReplacement.builder().setDrain(false).build(); } @@ -94,13 +93,13 @@ public static TransformReplacement createSizedReplacement() { * * . */ - public static TransformReplacement createTruncateReplacement() { + public static ProtoOverrides.TransformReplacement createTruncateReplacement() { return SizedReplacement.builder().setDrain(true).build(); } /** See {@link #createSizedReplacement()} for details. */ @AutoValue - abstract static class SizedReplacement implements TransformReplacement { + abstract static class SizedReplacement implements ProtoOverrides.TransformReplacement { static Builder builder() { return new AutoValue_SplittableParDoExpander_SizedReplacement.Builder(); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TimerReference.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/TimerReference.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TimerReference.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/TimerReference.java index ac864c8644b4..6436d30be309 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TimerReference.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/TimerReference.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import com.google.auto.value.AutoValue; import org.apache.beam.model.pipeline.v1.RunnerApi; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TrivialNativeTransformExpander.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/TrivialNativeTransformExpander.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TrivialNativeTransformExpander.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/TrivialNativeTransformExpander.java index 6d7cd25349b1..d5747fc7d5e2 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TrivialNativeTransformExpander.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/TrivialNativeTransformExpander.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import java.util.Set; import org.apache.beam.model.pipeline.v1.RunnerApi; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/UserStateReference.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/UserStateReference.java similarity index 90% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/UserStateReference.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/UserStateReference.java index 052334b3ad0e..cf0e22d4fef9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/UserStateReference.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/UserStateReference.java @@ -15,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import com.google.auto.value.AutoValue; import java.io.IOException; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.UserStateId; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; /** * A reference to user state. This includes the PTransform that references the user state as well as diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/package-info.java similarity index 94% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/package-info.java index 002438e6052a..9d5af85dc5a3 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/graph/package-info.java @@ -18,7 +18,7 @@ /** Graph-manipulation utilities. */ @DefaultAnnotation(NonNull.class) -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import edu.umd.cs.findbugs.annotations.DefaultAnnotation; import org.checkerframework.checker.nullness.qual.NonNull; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/package-info.java similarity index 95% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/package-info.java index edd700fb7839..82d4d26c99d1 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/package-info.java @@ -18,7 +18,7 @@ /** Provides utilities for Beam runner authors, prior to execution. */ @DefaultAnnotation(NonNull.class) -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import edu.umd.cs.findbugs.annotations.DefaultAnnotation; import org.checkerframework.checker.nullness.qual.NonNull; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRenderer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRenderer.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRenderer.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRenderer.java index cf1af51a706e..36b7501e3d4e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRenderer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRenderer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.renderer; +package org.apache.beam.sdk.util.construction.renderer; import com.google.errorprone.annotations.FormatMethod; import com.google.errorprone.annotations.FormatString; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRenderer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRenderer.java similarity index 94% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRenderer.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRenderer.java index 070e975ddd5d..649bc73c6f60 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRenderer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRenderer.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.renderer; +package org.apache.beam.sdk.util.construction.renderer; import com.google.errorprone.annotations.FormatMethod; import com.google.errorprone.annotations.FormatString; import java.util.HashMap; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.graph.PipelineNode; -import org.apache.beam.runners.core.construction.graph.QueryablePipeline; +import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; /** * A DOT renderer for BEAM portable {@link org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline}. diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/package-info.java similarity index 93% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/package-info.java index a557304ae28a..53a670601515 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/renderer/package-info.java @@ -17,4 +17,4 @@ */ /** Classes used to render Pipelines. */ -package org.apache.beam.runners.core.construction.renderer; +package org.apache.beam.sdk.util.construction.renderer; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/ClasspathScanningResourcesDetector.java similarity index 97% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetector.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/ClasspathScanningResourcesDetector.java index 57c2296f362f..4c289603ae0b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/ClasspathScanningResourcesDetector.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.resources; +package org.apache.beam.sdk.util.construction.resources; import io.github.classgraph.ClassGraph; import java.io.File; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResources.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResources.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResources.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResources.java index 433760a6562b..545817f0d913 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResources.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResources.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.resources; +package org.apache.beam.sdk.util.construction.resources; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesDetector.java similarity index 95% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetector.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesDetector.java index 103c694d2d31..e6146db4d72e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesDetector.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.resources; +package org.apache.beam.sdk.util.construction.resources; import java.io.Serializable; import java.util.List; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesOptions.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesOptions.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesOptions.java index 884664afb9dc..614465b45e3d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesOptions.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.resources; +package org.apache.beam.sdk.util.construction.resources; import com.fasterxml.jackson.annotation.JsonIgnore; import io.github.classgraph.ClassGraph; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/package-info.java similarity index 93% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/package-info.java index 6e64dad43b20..7bcc4cb07f0f 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/resources/package-info.java @@ -17,4 +17,4 @@ */ /** Classes used to render Pipelines. */ -package org.apache.beam.runners.core.construction.resources; +package org.apache.beam.sdk.util.construction.resources; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java index a1a8823a0a35..3a7a0d5a8935 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.fn.data; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; import static org.hamcrest.core.StringContains.containsString; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertSame; @@ -38,6 +37,8 @@ import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -89,7 +90,7 @@ public void testOutboundObserver() { OutboundObserverFactory.clientDirect(), inboundObserver -> TestStreams.withOnNext(values::add).build()); multiplexer.getOutboundObserver().onNext(ELEMENTS); - assertThat(values, contains(ELEMENTS)); + MatcherAssert.assertThat(values, Matchers.contains(ELEMENTS)); } @Test @@ -159,14 +160,14 @@ public void accept(BeamFnApi.Elements input) throws Exception { assertFalse(multiplexer.hasConsumer(TIMER_INSTRUCTION_ID)); // Assert that normal and terminal Elements are passed to the consumer - assertThat( + MatcherAssert.assertThat( dataInboundValues, - contains( + Matchers.contains( ELEMENTS.toBuilder().clearTimers().build(), TERMINAL_ELEMENTS.toBuilder().clearTimers().build())); - assertThat( + MatcherAssert.assertThat( timerInboundValues, - contains( + Matchers.contains( ELEMENTS.toBuilder().clearData().build(), TERMINAL_ELEMENTS.toBuilder().clearData().build())); } @@ -222,14 +223,14 @@ public void accept(BeamFnApi.Elements input) throws Exception { multiplexer.getInboundObserver().onNext(TERMINAL_ELEMENTS); // Assert that elements are partitioned based upon the instruction id. - assertThat( + MatcherAssert.assertThat( dataInboundValues, - contains( + Matchers.contains( ELEMENTS.toBuilder().clearTimers().build(), TERMINAL_ELEMENTS.toBuilder().clearTimers().build())); - assertThat( + MatcherAssert.assertThat( timerInboundValues, - contains( + Matchers.contains( ELEMENTS.toBuilder().clearData().build(), TERMINAL_ELEMENTS.toBuilder().clearData().build())); } @@ -315,9 +316,9 @@ public void accept(BeamFnApi.Elements input) throws Exception { .onNext(BeamFnApi.Elements.newBuilder().addData(data.setTransformId("C").build()).build()); // Assert that we ignored the other two elements - assertThat( + MatcherAssert.assertThat( dataInboundValues, - contains( + Matchers.contains( BeamFnApi.Elements.newBuilder().addData(data.setTransformId("A").build()).build())); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java index 6f1dd8271fa5..529add84152d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.fn.data; -import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.empty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -42,7 +41,9 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -85,11 +86,11 @@ public void testWithDefaultBuffer() throws Exception { FnDataReceiver dataReceiver = registerOutputLocation(aggregator, endpoint, CODER); aggregator.start(); dataReceiver.accept(new byte[BeamFnDataOutboundAggregator.DEFAULT_BUFFER_LIMIT_BYTES - 50]); - assertThat(values, empty()); + MatcherAssert.assertThat(values, empty()); // Test that when we cross the buffer, we emit. dataReceiver.accept(new byte[50]); - assertEquals( + Assert.assertEquals( messageWithData( new byte[BeamFnDataOutboundAggregator.DEFAULT_BUFFER_LIMIT_BYTES - 50], new byte[50]), values.get(0)); @@ -100,18 +101,18 @@ public void testWithDefaultBuffer() throws Exception { // Test that when we cross the buffer, we emit. dataReceiver.accept(new byte[50]); - assertEquals( + Assert.assertEquals( messageWithData( new byte[BeamFnDataOutboundAggregator.DEFAULT_BUFFER_LIMIT_BYTES - 50], new byte[50]), values.get(1)); // Test that when we close with an empty buffer we only have one end of stream aggregator.sendOrCollectBufferedDataAndFinishOutboundStreams(); - assertEquals(endMessage(), values.get(2)); + Assert.assertEquals(endMessage(), values.get(2)); // Test that we can close twice. aggregator.sendOrCollectBufferedDataAndFinishOutboundStreams(); - assertEquals(endMessage(), values.get(2)); + Assert.assertEquals(endMessage(), values.get(2)); } @Test @@ -134,11 +135,11 @@ public void testConfiguredBufferLimit() throws Exception { FnDataReceiver dataReceiver = registerOutputLocation(aggregator, endpoint, CODER); aggregator.start(); dataReceiver.accept(new byte[51]); - assertThat(values, empty()); + MatcherAssert.assertThat(values, empty()); // Test that when we cross the buffer, we emit. dataReceiver.accept(new byte[49]); - assertEquals(messageWithData(new byte[51], new byte[49]), values.get(0)); + Assert.assertEquals(messageWithData(new byte[51], new byte[49]), values.get(0)); Receiver receiver; if (endpoint.isTimer()) { receiver = Iterables.getOnlyElement(aggregator.outputTimersReceivers.values()); @@ -174,7 +175,7 @@ public void testConfiguredBufferLimit() throws Exception { .setTransformId(endpoint.getTransformId()) .setIsLast(true)); } - assertEquals(builder.build(), values.get(1)); + Assert.assertEquals(builder.build(), values.get(1)); } @Test @@ -203,7 +204,7 @@ public void testConfiguredTimeLimit() throws Exception { aggregator.start(); dataReceiver.accept(new byte[1]); waitForFlush.await(); // wait the flush thread to flush the buffer - assertEquals(messageWithData(new byte[1]), values.get(0)); + Assert.assertEquals(messageWithData(new byte[1]), values.get(0)); } @Test @@ -294,7 +295,7 @@ public void testConfiguredBufferLimitMultipleEndpoints() throws Exception { registerOutputLocation(aggregator, additionalEndpoint, CODER); aggregator.start(); dataReceiver.accept(new byte[51]); - assertThat(values, empty()); + MatcherAssert.assertThat(values, empty()); // Test that when we cross the buffer, we emit. additionalDataReceiver.accept(new byte[49]); @@ -333,8 +334,9 @@ public void testConfiguredBufferLimitMultipleEndpoints() throws Exception { } private void checkEqualInAnyOrder(Elements first, Elements second) { - assertThat(first.getDataList(), Matchers.containsInAnyOrder(second.getDataList().toArray())); - assertThat( + MatcherAssert.assertThat( + first.getDataList(), Matchers.containsInAnyOrder(second.getDataList().toArray())); + MatcherAssert.assertThat( first.getTimersList(), Matchers.containsInAnyOrder(second.getTimersList().toArray())); } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/BaseExternalTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/BaseExternalTest.java similarity index 97% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/BaseExternalTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/BaseExternalTest.java index 4126e72325f7..2ff5aadced3c 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/BaseExternalTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/BaseExternalTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java index 8f1b7cf4f096..7dbe544d5948 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -30,7 +30,6 @@ import java.util.Set; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -55,6 +54,7 @@ import org.apache.beam.sdk.util.ShardedKey; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.hamcrest.Matchers; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CombineTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CombineTranslationTest.java index a3a2bb5ba533..468bce71475c 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CombineTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.junit.Assert.assertEquals; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CreatePCollectionViewTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CreatePCollectionViewTranslationTest.java index 6974c01ccb12..b421e817cfce 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CreatePCollectionViewTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DeduplicatedFlattenFactoryTest.java similarity index 96% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactoryTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DeduplicatedFlattenFactoryTest.java index 7c31b992a3d1..2a7db657c791 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DeduplicatedFlattenFactoryTest.java @@ -15,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; -import org.apache.beam.runners.core.construction.DeduplicatedFlattenFactory.FlattenWithoutDuplicateInputs; import org.apache.beam.sdk.Pipeline.PipelineVisitor.Defaults; import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput; import org.apache.beam.sdk.runners.TransformHierarchy; @@ -31,6 +30,7 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.DeduplicatedFlattenFactory.FlattenWithoutDuplicateInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PValues; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DefaultArtifactResolverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DefaultArtifactResolverTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DefaultArtifactResolverTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DefaultArtifactResolverTest.java index 53c22af2ccaf..5ff6b9e1cbee 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DefaultArtifactResolverTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DefaultArtifactResolverTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DisplayDataTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DisplayDataTranslationTest.java similarity index 86% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DisplayDataTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DisplayDataTranslationTest.java index 502e2fe0d8cd..27d116641403 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DisplayDataTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/DisplayDataTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -27,7 +27,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** Tests for {@link org.apache.beam.runners.core.construction.DisplayDataTranslation}. */ +/** Tests for {@link DisplayDataTranslation}. */ @RunWith(JUnit4.class) public class DisplayDataTranslationTest { @@ -58,7 +58,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .setStringValue("value") .setKey("foo") .setNamespace( - "org.apache.beam.runners.core.construction.DisplayDataTranslationTest$1") + "org.apache.beam.sdk.util.construction.DisplayDataTranslationTest$1") .build() .toByteString()) .build(), @@ -70,7 +70,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .setStringValue("value2") .setKey("foo2") .setNamespace( - "org.apache.beam.runners.core.construction.DisplayDataTranslationTest$1") + "org.apache.beam.sdk.util.construction.DisplayDataTranslationTest$1") .build() .toByteString()) .build(), @@ -82,7 +82,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .setBoolValue(true) .setKey("foo3") .setNamespace( - "org.apache.beam.runners.core.construction.DisplayDataTranslationTest$1") + "org.apache.beam.sdk.util.construction.DisplayDataTranslationTest$1") .build() .toByteString()) .build(), @@ -94,7 +94,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .setDoubleValue(123.4) .setKey("foo4") .setNamespace( - "org.apache.beam.runners.core.construction.DisplayDataTranslationTest$1") + "org.apache.beam.sdk.util.construction.DisplayDataTranslationTest$1") .build() .toByteString()) .build(), @@ -106,7 +106,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .setDoubleValue(4.321000099182129) .setKey("foo5") .setNamespace( - "org.apache.beam.runners.core.construction.DisplayDataTranslationTest$1") + "org.apache.beam.sdk.util.construction.DisplayDataTranslationTest$1") .build() .toByteString()) .build(), @@ -118,7 +118,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .setIntValue(321) .setKey("foo6") .setNamespace( - "org.apache.beam.runners.core.construction.DisplayDataTranslationTest$1") + "org.apache.beam.sdk.util.construction.DisplayDataTranslationTest$1") .build() .toByteString()) .build(), @@ -130,7 +130,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .setIntValue(123) .setKey("foo7") .setNamespace( - "org.apache.beam.runners.core.construction.DisplayDataTranslationTest$1") + "org.apache.beam.sdk.util.construction.DisplayDataTranslationTest$1") .build() .toByteString()) .build())); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EmptyFlattenAsCreateFactoryTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EmptyFlattenAsCreateFactoryTest.java index bc504afa63eb..f06bded6b902 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EmptyFlattenAsCreateFactoryTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.emptyIterable; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EnvironmentsTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EnvironmentsTest.java index 453f6ab6db88..93ec4f7e8f7f 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EnvironmentsTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; -import static org.apache.beam.runners.core.construction.Environments.JAVA_SDK_HARNESS_CONTAINER_URL; +import static org.apache.beam.sdk.util.construction.Environments.JAVA_SDK_HARNESS_CONTAINER_URL; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; @@ -43,7 +43,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.ProcessPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardEnvironments; -import org.apache.beam.runners.core.construction.Environments.JavaVersion; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -52,6 +51,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.construction.Environments.JavaVersion; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExecutableStageTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ExecutableStageTranslationTest.java similarity index 96% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExecutableStageTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ExecutableStageTranslationTest.java index 38d3a16d50ea..ff69cb6a40bf 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExecutableStageTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ExecutableStageTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.Is.is; @@ -25,12 +25,12 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; import org.junit.Test; /** Tests for {@link ExecutableStageTranslation}. */ diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ExternalTranslationTest.java similarity index 97% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ExternalTranslationTest.java index 8020cfa5ab54..8ee101235c65 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ExternalTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -33,7 +33,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** Tests for {@link org.apache.beam.runners.core.construction.ExternalTranslation}. */ +/** Tests for {@link ExternalTranslation}. */ @RunWith(JUnit4.class) public class ExternalTranslationTest { @Test diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ForwardingPTransformTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ForwardingPTransformTest.java index 6c94d91f630a..b985373172c8 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ForwardingPTransformTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/GroupByKeyTranslationTest.java similarity index 90% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/GroupByKeyTranslationTest.java index aae4c850f6d4..4c4fd75df690 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/GroupByKeyTranslationTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; -import static org.apache.beam.runners.core.construction.PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslationTest.java index 6760fa1f714d..cb2054e09144 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupIntoBatchesTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ModelCodersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java similarity index 95% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ModelCodersTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java index da60661af8e8..f0a277760ac4 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ModelCodersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -24,8 +24,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Coder; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.MessageWithComponents; -import org.apache.beam.runners.core.construction.ModelCoders.KvCoderComponents; -import org.apache.beam.runners.core.construction.ModelCoders.WindowedValueCoderComponents; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -35,6 +33,8 @@ import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.construction.ModelCoders.KvCoderComponents; +import org.apache.beam.sdk.util.construction.ModelCoders.WindowedValueCoderComponents; import org.apache.beam.sdk.values.KV; import org.junit.Rule; import org.junit.Test; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/MorePipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/MorePipelineTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/MorePipelineTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/MorePipelineTest.java index 32fb164d85c7..50b1aacdcd00 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/MorePipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/MorePipelineTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/NativeTransformsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/NativeTransformsTest.java similarity index 93% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/NativeTransformsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/NativeTransformsTest.java index ec7bde6d73cd..55afa5aef1b0 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/NativeTransformsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/NativeTransformsTest.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import com.google.auto.service.AutoService; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.NativeTransforms.IsNativeTransform; +import org.apache.beam.sdk.util.construction.NativeTransforms.IsNativeTransform; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionTranslationTest.java index 2c5f53fad1f7..1b7451f53ace 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionViewTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionViewTranslationTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionViewTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionViewTranslationTest.java index 6f5df9c29646..ff5f841c148e 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionViewTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionViewTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.junit.Assert.assertEquals; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformMatchersTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformMatchersTest.java index c7a2ae4a4a67..602df0ae545c 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformMatchersTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.instanceOf; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformReplacementsTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformReplacementsTest.java index d841de4581ba..019f68347935 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformReplacementsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformTranslationTest.java index 4c3c3e072ce9..e0ca074f822c 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PTransformTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ParDoTranslationTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ParDoTranslationTest.java index d6d335b56291..fa308163ed3a 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ParDoTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -31,7 +31,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -60,6 +59,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -215,7 +215,7 @@ public void toTransformProto() throws Exception { rehydratedComponents, TranslationContext.DEFAULT); assertEquals( - org.apache.beam.runners.core.construction.Timer.Coder.of( + org.apache.beam.sdk.util.construction.Timer.Coder.of( VarLongCoder.of(), GlobalWindow.Coder.INSTANCE), timerCoder); } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PipelineOptionsTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PipelineOptionsTranslationTest.java index 4f924e28dcb3..0ce313882f20 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PipelineOptionsTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PipelineTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PipelineTranslationTest.java index 60769a257522..629f4e4c5ac7 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PipelineTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -29,7 +29,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -57,6 +56,7 @@ import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReadTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReadTranslationTest.java index 7ebe55d6917c..90e1914cb935 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReadTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/RehydratedComponentsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/RehydratedComponentsTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/RehydratedComponentsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/RehydratedComponentsTest.java index 5dfd521c8f02..baca7cfe36af 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/RehydratedComponentsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/RehydratedComponentsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReplacementOutputsTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReplacementOutputsTest.java index 8a6942535db3..494857b18f24 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReplacementOutputsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReshuffleTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReshuffleTranslationTest.java similarity index 91% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReshuffleTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReshuffleTranslationTest.java index 0f8b97bff2fe..02d95dabda07 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReshuffleTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ReshuffleTranslationTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; -import static org.apache.beam.runners.core.construction.PTransformTranslation.RESHUFFLE_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.RESHUFFLE_URN; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SdkComponentsTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SdkComponentsTest.java index 6e58802beb52..b29e36ce8de2 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SdkComponentsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SerializablePipelineOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SerializablePipelineOptionsTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SerializablePipelineOptionsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SerializablePipelineOptionsTest.java index 0c2861dfffc7..9f1b5714a38f 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SerializablePipelineOptionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SerializablePipelineOptionsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.junit.Assert.assertEquals; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SingleInputOutputOverrideFactoryTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SingleInputOutputOverrideFactoryTest.java index 22982c5f2db7..35a4c2fd5142 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SingleInputOutputOverrideFactoryTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SplittableParDoTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SplittableParDoTest.java index 57cd6f218d24..470d30710242 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/SplittableParDoTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.sdk.transforms.DoFn.ProcessContinuation.stop; import static org.hamcrest.MatcherAssert.assertThat; @@ -29,7 +29,6 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor.Defaults; import org.apache.beam.sdk.coders.Coder; @@ -50,6 +49,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PValues; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TestStreamTranslationTest.java similarity index 96% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TestStreamTranslationTest.java index 459b48ea95a7..068a9781a960 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TestStreamTranslationTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; -import static org.apache.beam.runners.core.construction.PTransformTranslation.TEST_STREAM_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.TEST_STREAM_TRANSFORM_URN; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TimerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TimerTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java index f3d12e87e283..f3e7c392664d 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TimerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TransformInputsTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TransformInputsTest.java index 995611e52e91..0fe00c8b9c97 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TransformInputsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TransformUpgraderTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TransformUpgraderTest.java index 4735df5397d1..7c1a57d571a7 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformUpgraderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TransformUpgraderTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TriggerTranslationTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TriggerTranslationTest.java index 176f67531ce5..1ad33e18e2f0 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TriggerTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java similarity index 97% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java index 31f6842a42bc..af5a442d1523 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -37,9 +37,6 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Random; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint; -import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.CheckpointCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -62,6 +59,9 @@ import org.apache.beam.sdk.transforms.Min; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint; +import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.CheckpointCoder; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnconsumedReadsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnconsumedReadsTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnconsumedReadsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnconsumedReadsTest.java index e8639cbb3b52..a3d187319c7e 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnconsumedReadsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnconsumedReadsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnsupportedOverrideFactoryTest.java similarity index 97% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnsupportedOverrideFactoryTest.java index 6e8d64a64f86..89450ae8d38f 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnsupportedOverrideFactoryTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.util.Collections; import org.apache.beam.sdk.testing.TestPipeline; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ValidateRunnerXlangTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ValidateRunnerXlangTest.java index efefa77f508a..c41b2151d4cc 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ValidateRunnerXlangTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import java.io.IOException; import java.io.Serializable; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WindowIntoTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WindowIntoTranslationTest.java index 2439657a93bb..b55ac151357b 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WindowIntoTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.junit.Assert.assertEquals; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WindowingStrategyTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WindowingStrategyTranslationTest.java index 6981e74db782..95d3171c524e 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WindowingStrategyTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WriteFilesTranslationTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WriteFilesTranslationTest.java index 6cea7beb4f46..3c3a6b7417ad 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/WriteFilesTranslationTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction; +package org.apache.beam.sdk.util.construction; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageMatcher.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ExecutableStageMatcher.java similarity index 96% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageMatcher.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ExecutableStageMatcher.java index 83bde34a0a9f..e31b9f3d6edf 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageMatcher.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ExecutableStageMatcher.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -24,8 +24,8 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Description; import org.hamcrest.Matcher; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ExecutableStageTest.java similarity index 94% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ExecutableStageTest.java index b46769cd7267..6e8d78e66584 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ExecutableStageTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; -import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; +import static org.apache.beam.sdk.util.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.contains; @@ -38,9 +38,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.TimerFamilySpec; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.junit.Test; import org.junit.runner.RunWith; @@ -51,8 +51,7 @@ public class ExecutableStageTest { @Test public void testRoundTripToFromTransform() throws Exception { - Environment env = - org.apache.beam.runners.core.construction.Environments.createDockerEnvironment("foo"); + Environment env = Environments.createDockerEnvironment("foo"); PTransform pt = PTransform.newBuilder() .putInputs("input", "input.out") diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/FieldAccessVisitorTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitorTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/FieldAccessVisitorTest.java index b6ef73d3c5e7..d43d59a88396 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/FieldAccessVisitorTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FusedPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/FusedPipelineTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FusedPipelineTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/FusedPipelineTest.java index db6752dcc503..f2637d1e8f53 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FusedPipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/FusedPipelineTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.hamcrest.MatcherAssert.assertThat; @@ -33,13 +33,13 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/GreedyPipelineFuserTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/GreedyPipelineFuserTest.java index dffd15077d42..9b403b869437 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/GreedyPipelineFuserTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.getOnlyElement; import static org.hamcrest.MatcherAssert.assertThat; @@ -46,11 +46,11 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.TimerFamilySpec; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.hamcrest.Matchers; import org.hamcrest.core.AnyOf; import org.junit.Before; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/GreedyStageFuserTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/GreedyStageFuserTest.java index af4ea7e79f1c..242e84104ea6 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/GreedyStageFuserTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.getOnlyElement; import static org.hamcrest.MatcherAssert.assertThat; @@ -38,10 +38,10 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.TimerFamilySpec; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ImmutableExecutableStageTest.java similarity index 93% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ImmutableExecutableStageTest.java index 2554b8505fe9..69fdc85c08dc 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ImmutableExecutableStageTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; -import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; +import static org.apache.beam.sdk.util.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.contains; @@ -34,9 +34,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/NetworksTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/NetworksTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/NetworksTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/NetworksTest.java index af0368c52b09..7dc9fa466266 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/NetworksTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/NetworksTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/OutputDeduplicatorTest.java similarity index 97% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/OutputDeduplicatorTest.java index 27a222f3a82b..2d1c0a464764 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/OutputDeduplicatorTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; -import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; +import static org.apache.beam.sdk.util.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.getOnlyElement; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -38,10 +38,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.graph.OutputDeduplicator.DeduplicationResult; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; import org.junit.runner.RunWith; @@ -139,7 +138,7 @@ public void unchangedWithNoDuplicates() { PTransformNode redTransform = PipelineNode.pTransform("red", red); PTransformNode blueTransform = PipelineNode.pTransform("blue", blue); QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components); - DeduplicationResult result = + OutputDeduplicator.DeduplicationResult result = OutputDeduplicator.ensureSingleProducer( pipeline, ImmutableList.of(oneStage, twoStage), @@ -260,7 +259,7 @@ public void duplicateOverStages() { PTransformNode redTransform = PipelineNode.pTransform("red", red); PTransformNode blueTransform = PipelineNode.pTransform("blue", blue); QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components); - DeduplicationResult result = + OutputDeduplicator.DeduplicationResult result = OutputDeduplicator.ensureSingleProducer( pipeline, ImmutableList.of(oneStage, twoStage), @@ -380,7 +379,7 @@ public void duplicateOverStagesAndTransforms() { PTransformNode redTransform = PipelineNode.pTransform("red", red); PTransformNode blueTransform = PipelineNode.pTransform("blue", blue); QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components); - DeduplicationResult result = + OutputDeduplicator.DeduplicationResult result = OutputDeduplicator.ensureSingleProducer( pipeline, ImmutableList.of(oneStage), @@ -580,7 +579,7 @@ public void multipleDuplicatesInStages() { PTransformNode redTransform = PipelineNode.pTransform("red", red); PTransformNode blueTransform = PipelineNode.pTransform("blue", blue); QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components); - DeduplicationResult result = + OutputDeduplicator.DeduplicationResult result = OutputDeduplicator.ensureSingleProducer( pipeline, ImmutableList.of(oneStage, twoStage, multiStage), diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProjectionProducerVisitorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProjectionProducerVisitorTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProjectionProducerVisitorTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProjectionProducerVisitorTest.java index 89c1cf8904ae..4644e2458dc4 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProjectionProducerVisitorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProjectionProducerVisitorTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProjectionPushdownOptimizerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProjectionPushdownOptimizerTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProjectionPushdownOptimizerTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProjectionPushdownOptimizerTest.java index c9b6ba89fd1c..9707062c4f2c 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProjectionPushdownOptimizerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProjectionPushdownOptimizerTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import java.util.Map; import java.util.Objects; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProtoOverridesTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProtoOverridesTest.java index cdd00ea5c5db..3600bc05a53b 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/ProtoOverridesTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; @@ -35,7 +35,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy; -import org.apache.beam.runners.core.construction.graph.ProtoOverrides.TransformReplacement; +import org.apache.beam.sdk.util.construction.graph.ProtoOverrides.TransformReplacement; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/QueryablePipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/QueryablePipelineTest.java similarity index 97% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/QueryablePipelineTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/QueryablePipelineTest.java index 90e6af182e63..9ff664887590 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/QueryablePipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/QueryablePipelineTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.getOnlyElement; import static org.hamcrest.MatcherAssert.assertThat; @@ -37,11 +37,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode; -import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.Read; @@ -57,6 +52,11 @@ import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpanderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/SplittableParDoExpanderTest.java similarity index 97% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpanderTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/SplittableParDoExpanderTest.java index 548f5feab271..aec143022de5 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpanderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/graph/SplittableParDoExpanderTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.graph; +package org.apache.beam.sdk.util.construction.graph; import static org.apache.beam.sdk.transforms.DoFn.ProcessContinuation.resume; import static org.apache.beam.sdk.transforms.DoFn.ProcessContinuation.stop; @@ -23,8 +23,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.Create; @@ -32,6 +30,8 @@ import org.apache.beam.sdk.transforms.DoFn.UnboundedPerElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRendererTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRendererTest.java similarity index 98% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRendererTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRendererTest.java index 8d7e02d70c8d..4e4fc291630a 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRendererTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRendererTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.renderer; +package org.apache.beam.sdk.util.construction.renderer; import static org.junit.Assert.assertEquals; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRendererTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRendererTest.java similarity index 95% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRendererTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRendererTest.java index 0be7117d6057..e957ecfcd4c4 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRendererTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRendererTest.java @@ -15,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.renderer; +package org.apache.beam.sdk.util.construction.renderer; import static org.junit.Assert.assertEquals; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/resources/ClasspathScanningResourcesDetectorTest.java similarity index 92% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/resources/ClasspathScanningResourcesDetectorTest.java index 07c009ef1451..1cec4f42f942 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/resources/ClasspathScanningResourcesDetectorTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.resources; +package org.apache.beam.sdk.util.construction.resources; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.hasItem; @@ -55,7 +55,7 @@ public void shouldDetectDirectories() throws Exception { List result = detector.detect(classLoader); - assertThat(result, hasItem(containsString(folder.getAbsolutePath()))); + assertThat(result, hasItem(containsString(folder.getCanonicalPath()))); } @Test @@ -67,7 +67,7 @@ public void shouldDetectJarFiles() throws Exception { List result = detector.detect(classLoader); - assertThat(result, hasItem(containsString(jarFile.getAbsolutePath()))); + assertThat(result, hasItem(containsString(jarFile.getCanonicalPath()))); } @Test @@ -85,7 +85,7 @@ public void shouldDetectResourcesInOrderTheyAppearInURLClassLoader() throws Exce assertThat( result, containsInRelativeOrder( - containsString(file1.getAbsolutePath()), containsString(file2.getAbsolutePath()))); + containsString(file1.getCanonicalPath()), containsString(file2.getCanonicalPath()))); } private File createTestTmpJarFile(String name) throws IOException { @@ -103,12 +103,12 @@ public void shouldNotDetectOrdinaryFiles() throws Exception { List result = detector.detect(classLoader); - assertThat(result, not(hasItem(containsString(textFile.getAbsolutePath())))); + assertThat(result, not(hasItem(containsString(textFile.getCanonicalPath())))); } @Test public void shouldDetectClassPathResourceFromJavaClassPathEnvVariable() throws IOException { - String path = tmpFolder.newFolder("folder").getAbsolutePath(); + String path = tmpFolder.newFolder("folder").getCanonicalPath(); System.setProperty("java.class.path", path); ClasspathScanningResourcesDetector detector = new ClasspathScanningResourcesDetector(new ClassGraph()); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesTest.java similarity index 99% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesTest.java index bb4d7db2eb0e..971bdcf66503 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/resources/PipelineResourcesTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core.construction.resources; +package org.apache.beam.sdk.util.construction.resources; import static junit.framework.TestCase.assertTrue; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/sdks/java/expansion-service/build.gradle b/sdks/java/expansion-service/build.gradle index 6947e53354db..206b2fe1bc46 100644 --- a/sdks/java/expansion-service/build.gradle +++ b/sdks/java/expansion-service/build.gradle @@ -39,7 +39,6 @@ dependencies { implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(path: ":runners:core-construction-java") implementation project(path: ":runners:java-fn-execution") implementation project(path: ":sdks:java:harness") permitUnusedDeclared project(path: ":model:fn-execution") diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java index 9b0d8beb6fdc..8dd50794df39 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.expansion.service; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; -import static org.apache.beam.runners.core.construction.PTransformTranslation.READ_TRANSFORM_URN; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.PTransformTranslation.READ_TRANSFORM_URN; import com.google.auto.service.AutoService; import java.io.ByteArrayInputStream; @@ -45,14 +45,6 @@ import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExternalConfigurationPayload; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -76,6 +68,14 @@ import org.apache.beam.sdk.transforms.ExternalTransformBuilder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java index 0c93a9312046..fa0ea7720e75 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.expansion.service; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import java.io.IOException; import java.util.HashMap; diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java index ee2f1524a4e5..136b45fa7fa2 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.expansion.service; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/TransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/TransformProvider.java index fc58eb592911..65a9a0177cc9 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/TransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/TransformProvider.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.expansion.service; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; -import static org.apache.beam.runners.core.construction.resources.PipelineResources.detectClassPathResourcesToStage; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import java.util.Collections; @@ -28,11 +26,13 @@ import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.resources.PipelineResources; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; @@ -122,7 +122,8 @@ default Map> apply( } default String getTransformUniqueID(RunnerApi.FunctionSpec spec) { - if (getUrn(ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM).equals(spec.getUrn())) { + if (BeamUrns.getUrn(ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM) + .equals(spec.getUrn())) { ExternalTransforms.SchemaTransformPayload payload; try { payload = ExternalTransforms.SchemaTransformPayload.parseFrom(spec.getPayload()); @@ -130,7 +131,7 @@ default String getTransformUniqueID(RunnerApi.FunctionSpec spec) { } catch (InvalidProtocolBufferException e) { throw new IllegalArgumentException( "Invalid payload type for URN " - + getUrn(ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM), + + BeamUrns.getUrn(ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM), e); } } @@ -157,7 +158,7 @@ default List getDependencies(RunnerApi.FunctionSpec spec, PipelineOption throw new RuntimeException( "Cannot detect classpath: classloader is null (is it the bootstrap classloader?)"); } - filesToStage = detectClassPathResourcesToStage(classLoader, options); + filesToStage = PipelineResources.detectClassPathResourcesToStage(classLoader, options); if (filesToStage.isEmpty()) { throw new IllegalArgumentException("No classpath elements found."); } diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java index 6272b9445eb8..d060d5916e9f 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java @@ -23,13 +23,13 @@ import java.util.Collections; import java.util.List; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java index 6578287eb4c4..294cb016c5e0 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProviderTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.expansion.service; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; @@ -29,9 +29,6 @@ import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -52,6 +49,9 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java index 56a70cc1ca25..eb277917588d 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java @@ -41,7 +41,6 @@ import org.apache.beam.model.expansion.v1.ExpansionApi; import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -54,6 +53,7 @@ import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java index 8897c32320ed..eb6fbb7e903d 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExternalTest.java @@ -26,7 +26,6 @@ import java.net.ServerSocket; import java.nio.charset.StandardCharsets; import java.util.Map; -import org.apache.beam.runners.core.construction.External; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -38,6 +37,7 @@ import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.construction.External; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProviderTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProviderTest.java index e8f475ea66b6..34e22bc24122 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProviderTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProviderTest.java @@ -17,10 +17,10 @@ */ package org.apache.beam.sdk.expansion.service; -import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; import static org.apache.beam.sdk.expansion.service.JavaClassLookupTransformProvider.ALLOW_LIST_VERSION; import static org.apache.beam.sdk.expansion.service.JavaClassLookupTransformProvider.AllowList; import static org.apache.beam.sdk.expansion.service.JavaClassLookupTransformProvider.AllowedClass; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; @@ -46,8 +46,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.Schema; @@ -60,6 +58,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/extensions/avro/build.gradle b/sdks/java/extensions/avro/build.gradle index 13fd2fae6178..166bb91ac6e2 100644 --- a/sdks/java/extensions/avro/build.gradle +++ b/sdks/java/extensions/avro/build.gradle @@ -63,7 +63,6 @@ dependencies { // Exclude Avro dependencies from "core" since Avro support moved to this extension exclude group: "org.apache.avro", module: "avro" } - implementation(project(path: ":runners:core-construction-java")) implementation library.java.error_prone_annotations implementation library.java.avro implementation library.java.joda_time diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderRegistrar.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderRegistrar.java index 7eca6e6771b4..14ab48f66699 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderRegistrar.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderRegistrar.java @@ -19,10 +19,10 @@ import com.google.auto.service.AutoService; import java.util.Map; -import org.apache.beam.runners.core.construction.CoderTranslator; -import org.apache.beam.runners.core.construction.CoderTranslatorRegistrar; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; +import org.apache.beam.sdk.util.construction.CoderTranslator; +import org.apache.beam.sdk.util.construction.CoderTranslatorRegistrar; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** Coder registrar for AvroGenericCoder. */ diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java index 72e40d8509b3..67f386411d81 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java @@ -20,10 +20,10 @@ import java.util.Collections; import java.util.List; import org.apache.avro.Schema; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; -import org.apache.beam.runners.core.construction.CoderTranslator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; +import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; +import org.apache.beam.sdk.util.construction.CoderTranslator; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; /** Coder translator for AvroGenericCoder. */ diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/AvroCoderTranslationTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/AvroCoderTranslationTest.java index 1baaae7eb1b6..996d73348687 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/AvroCoderTranslationTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/AvroCoderTranslationTest.java @@ -23,13 +23,12 @@ import org.apache.avro.SchemaBuilder; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -53,7 +52,7 @@ public void toAndFromProto() throws Exception { CoderTranslation.fromProto( coderProto, RehydratedComponents.forComponents(encodedComponents), - TranslationContext.DEFAULT); + CoderTranslation.TranslationContext.DEFAULT); assertThat(decodedCoder, equalTo(coder)); } } diff --git a/sdks/java/extensions/python/build.gradle b/sdks/java/extensions/python/build.gradle index a83afac68815..7c04259b05ea 100644 --- a/sdks/java/extensions/python/build.gradle +++ b/sdks/java/extensions/python/build.gradle @@ -21,19 +21,16 @@ applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.extensions.python') description = "Apache Beam :: SDKs :: Java :: Extensions :: Python" -evaluationDependsOn(":runners:core-construction-java") - dependencies { implementation library.java.vendored_guava_32_1_2_jre implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api implementation project(path: ":model:pipeline", configuration: "shadow") - implementation project(path: ":runners:core-construction-java") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":sdks:java:transform-service:launcher") testImplementation library.java.junit testImplementation library.java.hamcrest - testImplementation project(":runners:core-construction-java").sourceSets.test.output + testImplementation project(":sdks:java:core").sourceSets.test.output testRuntimeOnly library.java.slf4j_simple } diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java index bd9ec7aa9816..5504706be8f7 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java @@ -33,7 +33,6 @@ import java.util.UUID; import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.repackaged.core.org.apache.commons.lang3.ClassUtils; -import org.apache.beam.runners.core.construction.External; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.RowCoder; @@ -51,6 +50,7 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.PythonCallableSource; import org.apache.beam.sdk.util.ReleaseInfo; +import org.apache.beam.sdk.util.construction.External; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/ConsecutivePythonTransformsTest.java b/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/ConsecutivePythonTransformsTest.java index feacf443f09b..03140c700cd0 100644 --- a/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/ConsecutivePythonTransformsTest.java +++ b/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/ConsecutivePythonTransformsTest.java @@ -18,13 +18,13 @@ package org.apache.beam.sdk.extensions.python; import java.util.Arrays; -import org.apache.beam.runners.core.construction.BaseExternalTest; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.UsesPythonExpansionService; import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.util.PythonCallableSource; +import org.apache.beam.sdk.util.construction.BaseExternalTest; import org.apache.beam.sdk.values.PCollection; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/DataframeTransformTest.java b/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/DataframeTransformTest.java index c35c8347fc8b..7ee9fe6460bb 100644 --- a/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/DataframeTransformTest.java +++ b/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/DataframeTransformTest.java @@ -17,12 +17,12 @@ */ package org.apache.beam.sdk.extensions.python.transforms; -import org.apache.beam.runners.core.construction.BaseExternalTest; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.UsesPythonExpansionService; import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.construction.BaseExternalTest; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.junit.Test; diff --git a/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/PythonMapTest.java b/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/PythonMapTest.java index 4b162a9d5435..5f18b4f60a17 100644 --- a/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/PythonMapTest.java +++ b/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/PythonMapTest.java @@ -17,12 +17,12 @@ */ package org.apache.beam.sdk.extensions.python.transforms; -import org.apache.beam.runners.core.construction.BaseExternalTest; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.UsesPythonExpansionService; import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.construction.BaseExternalTest; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; diff --git a/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/RunInferenceTransformTest.java b/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/RunInferenceTransformTest.java index 5f7b80f621e8..2d1b520d731a 100644 --- a/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/RunInferenceTransformTest.java +++ b/sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/RunInferenceTransformTest.java @@ -20,7 +20,6 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; -import org.apache.beam.runners.core.construction.BaseExternalTest; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VarLongCoder; @@ -32,6 +31,7 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.util.construction.BaseExternalTest; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index e1f3d660bcab..70f9c1f298bd 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -86,7 +86,7 @@ ext.summary = "This contains the SDK Fn Harness for Beam Java" dependencies { implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) implementation project(path: ":model:fn-execution", configuration: "shadow") - implementation project(":runners:core-construction-java") + implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":runners:core-java") permitUnusedDeclared project(path: ":sdks:java:transform-service:launcher") testImplementation library.java.junit diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java index ce7ae0a1ea45..21b0f26974a3 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java @@ -45,11 +45,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardRunnerProtocols; -import org.apache.beam.runners.core.construction.BeamUrns; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.graph.ExecutableStage; -import org.apache.beam.runners.core.construction.graph.FusedPipeline; -import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser; import org.apache.beam.runners.core.metrics.ExecutionStateTracker; import org.apache.beam.runners.fnexecution.control.BundleProgressHandler; import org.apache.beam.runners.fnexecution.control.ControlClientPool; @@ -89,6 +84,11 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.util.construction.graph.FusedPipeline; +import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java index c9a34b5ce4ed..8e4f6a2b2036 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java @@ -26,13 +26,13 @@ import org.apache.beam.fn.harness.MapFnRunners.WindowedValueMapFnFactory; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.function.ThrowingFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java index cc16a4fbbe72..26d07b3d8b2e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java @@ -39,8 +39,6 @@ import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.Urns; import org.apache.beam.runners.core.metrics.MonitoringInfoEncodings; @@ -50,6 +48,8 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.fn.data.RemoteGrpcPortRead; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.slf4j.Logger; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java index 310e9ef50834..ade5c32a8c14 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java @@ -28,11 +28,11 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.RemoteGrpcPortWrite; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java index 42810693bbe6..984aa30e7fb5 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java @@ -24,8 +24,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.fn.data.FnDataReceiver; @@ -36,6 +34,8 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java index 09074d7f6cb9..8ef3f96b00b6 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ExternalWorkerService.java @@ -26,14 +26,14 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StopWorkerResponse; import org.apache.beam.model.fnexecution.v1.BeamFnExternalWorkerPoolGrpc.BeamFnExternalWorkerPoolImplBase; import org.apache.beam.model.pipeline.v1.Endpoints; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.sdk.fn.server.FnService; import org.apache.beam.sdk.fn.server.GrpcFnServer; import org.apache.beam.sdk.fn.server.ServerFactory; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.Sleeper; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java index 1a223ff7de6c..c54f50705728 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java @@ -22,9 +22,9 @@ import com.google.auto.service.AutoService; import java.io.IOException; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** Executes flatten PTransforms. */ diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index a3ae67fc2a56..f85622ab89fe 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -57,11 +57,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.TimerFamilySpec; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.LateDataUtils; -import org.apache.beam.runners.core.construction.PCollectionViewTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.ShortIdMap; import org.apache.beam.runners.core.metrics.SimpleMonitoringInfoBuilder; @@ -110,6 +105,11 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.util.construction.PCollectionViewTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.Row; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index e22dd6d5b2ad..3301fd17e1bc 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -48,7 +48,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor; import org.apache.beam.model.fnexecution.v1.BeamFnControlGrpc; import org.apache.beam.model.pipeline.v1.Endpoints; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.runners.core.metrics.ShortIdMap; import org.apache.beam.sdk.fn.IdGenerator; @@ -63,6 +62,7 @@ import org.apache.beam.sdk.options.ExecutorOptions; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java index e7b29550afdf..d75d7eea39c2 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java @@ -30,7 +30,6 @@ import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.core.metrics.ShortIdMap; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; @@ -38,6 +37,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Timer; /** A factory able to instantiate an appropriate handler for a given PTransform. */ @SuppressWarnings({ diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ToStringFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ToStringFnRunner.java index 5642489a1884..0c6acb046769 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ToStringFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ToStringFnRunner.java @@ -21,8 +21,8 @@ import java.util.Map; import java.util.Objects; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.function.ThrowingFunction; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMappingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMappingFnRunner.java index 53f1a847364c..e4d8f7bc6f6e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMappingFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMappingFnRunner.java @@ -22,11 +22,11 @@ import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.PCollectionViewTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.function.ThrowingFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; +import org.apache.beam.sdk.util.construction.PCollectionViewTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMergingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMergingFnRunner.java index 156add3df7c6..38bd665b34ff 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMergingFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMergingFnRunner.java @@ -26,12 +26,12 @@ import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.function.ThrowingFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowFn.MergeContext; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 344c84422ecb..43e68f2c6501 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -72,9 +72,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardRunnerProtocols; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy; -import org.apache.beam.runners.core.construction.BeamUrns; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.Urns; import org.apache.beam.runners.core.metrics.ShortIdMap; import org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver; @@ -90,6 +87,9 @@ import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ReflectHelpers; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java index e27df577779c..2698bdf4b1fc 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java @@ -39,7 +39,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.Labels; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns; @@ -53,6 +52,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java index 2751fade988f..cfc247039503 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java @@ -36,7 +36,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.SideInputReader; -import org.apache.beam.runners.core.construction.BeamUrns; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -62,6 +61,7 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CombineFnUtil; +import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java index 6dbc600fa448..46c9e055a834 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiTimerBundleTracker.java @@ -26,13 +26,13 @@ import java.util.function.Function; import java.util.function.Supplier; import javax.annotation.Nullable; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ComparisonChain; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java index 358d7924767e..cc9204a79062 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness.state; -import static org.apache.beam.runners.core.construction.ModelCoders.STATE_BACKED_ITERABLE_CODER_URN; +import static org.apache.beam.sdk.util.construction.ModelCoders.STATE_BACKED_ITERABLE_CODER_URN; import com.google.auto.service.AutoService; import java.io.DataOutputStream; @@ -35,9 +35,6 @@ import org.apache.beam.fn.harness.Caches; import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; -import org.apache.beam.runners.core.construction.CoderTranslator; -import org.apache.beam.runners.core.construction.CoderTranslatorRegistrar; import org.apache.beam.sdk.coders.IterableLikeCoder; import org.apache.beam.sdk.fn.stream.PrefetchableIterable; import org.apache.beam.sdk.fn.stream.PrefetchableIterators; @@ -46,6 +43,9 @@ import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; +import org.apache.beam.sdk.util.construction.CoderTranslator; +import org.apache.beam.sdk.util.construction.CoderTranslatorRegistrar; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java index 659afff2a885..3d81564fee31 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java @@ -30,11 +30,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.function.ThrowingFunction; @@ -50,6 +45,11 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.joda.time.Duration; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java index 05f720bd24c4..55342f9362b2 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java @@ -53,7 +53,6 @@ import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.MessageWithComponents; -import org.apache.beam.runners.core.construction.CoderTranslation; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns; import org.apache.beam.runners.core.metrics.MonitoringInfoEncodings; @@ -67,6 +66,7 @@ import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java index 38c3763a535c..071ac037883b 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java @@ -42,7 +42,6 @@ import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.MessageWithComponents; -import org.apache.beam.runners.core.construction.CoderTranslation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.fn.data.BeamFnDataOutboundAggregator; @@ -53,6 +52,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.hamcrest.collection.IsMapContaining; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java index b43f480977a6..1bdc3d899718 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java @@ -30,9 +30,6 @@ import java.util.HashMap; import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -42,6 +39,9 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java index bffe10b34958..828d146f0b99 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java @@ -30,11 +30,11 @@ import java.util.Map; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 7c42104ca7cb..11f25ab0116e 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -67,15 +67,6 @@ import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.graph.ProtoOverrides; -import org.apache.beam.runners.core.construction.graph.SplittableParDoExpander; import org.apache.beam.runners.core.metrics.DistributionData; import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; @@ -136,6 +127,15 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.graph.ProtoOverrides; +import org.apache.beam.sdk.util.construction.graph.SplittableParDoExpander; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -854,7 +854,7 @@ public void testUsingMetrics() throws Exception { } private class TestBeamFnDataOutboundAggregator extends BeamFnDataOutboundAggregator { - private Map>> timers; + private Map>> timers; private Map>> dataOutput; private Supplier processBundleRequestIdSupplier; @@ -865,7 +865,7 @@ public TestBeamFnDataOutboundAggregator(Supplier bundleIdSupplier) { this.processBundleRequestIdSupplier = bundleIdSupplier; } - public Map>> + public Map>> getOutputTimers() { return timers; } @@ -893,7 +893,7 @@ public FnDataReceiver registerOutputTimersLocation( LogicalEndpoint.timer( processBundleRequestIdSupplier.get(), pTransformId, timerFamilyId), e -> new ArrayList<>()) - .add((org.apache.beam.runners.core.construction.Timer) data); + .add((org.apache.beam.sdk.util.construction.Timer) data); } } @@ -1123,25 +1123,25 @@ public void testTimers() throws Exception { fakeStateClient.getData()); } - private org.apache.beam.runners.core.construction.Timer timerInGlobalWindow( + private org.apache.beam.sdk.util.construction.Timer timerInGlobalWindow( K userKey, Instant holdTimestamp, Instant fireTimestamp) { return dynamicTimerInGlobalWindow(userKey, "", holdTimestamp, fireTimestamp); } - private org.apache.beam.runners.core.construction.Timer clearedTimerInGlobalWindow( + private org.apache.beam.sdk.util.construction.Timer clearedTimerInGlobalWindow( K userKey) { return clearedTimerInGlobalWindow(userKey, ""); } - private org.apache.beam.runners.core.construction.Timer clearedTimerInGlobalWindow( + private org.apache.beam.sdk.util.construction.Timer clearedTimerInGlobalWindow( K userKey, String dynamicTimerTag) { - return org.apache.beam.runners.core.construction.Timer.cleared( + return org.apache.beam.sdk.util.construction.Timer.cleared( userKey, dynamicTimerTag, Collections.singletonList(GlobalWindow.INSTANCE)); } - private org.apache.beam.runners.core.construction.Timer dynamicTimerInGlobalWindow( + private org.apache.beam.sdk.util.construction.Timer dynamicTimerInGlobalWindow( K userKey, String dynamicTimerTag, Instant holdTimestamp, Instant fireTimestamp) { - return org.apache.beam.runners.core.construction.Timer.of( + return org.apache.beam.sdk.util.construction.Timer.of( userKey, dynamicTimerTag, Collections.singletonList(GlobalWindow.INSTANCE), diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java index 65fb4a42c03e..e559e9faa309 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java @@ -37,12 +37,12 @@ import org.apache.beam.model.fnexecution.v1.BeamFnControlGrpc; import org.apache.beam.model.fnexecution.v1.BeamFnLoggingGrpc; import org.apache.beam.model.pipeline.v1.Endpoints; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.harness.JvmInitializer; import org.apache.beam.sdk.options.ExecutorOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ServerBuilder; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java index 4a9f05b65ab5..cf750bfd7d6a 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java @@ -31,13 +31,13 @@ import org.apache.beam.fn.harness.MapFnRunners.ValueMapFnFactory; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.CoderTranslation; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.function.ThrowingFunction; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java index 57fe54156692..9328dc86c009 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java @@ -40,7 +40,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse; import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.core.metrics.ShortIdMap; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.BeamFnDataOutboundAggregator; @@ -54,6 +53,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.Timer; import org.joda.time.Instant; /** diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/ToStringFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/ToStringFnRunnerTest.java index cc5e6465c865..262797d1ac2f 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/ToStringFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/ToStringFnRunnerTest.java @@ -20,9 +20,9 @@ import static org.junit.Assert.assertEquals; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.function.ThrowingFunction; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.KV; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMappingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMappingFnRunnerTest.java index 11a651145eac..4d2ed72756df 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMappingFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMappingFnRunnerTest.java @@ -20,14 +20,14 @@ import static org.junit.Assert.assertEquals; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.function.ThrowingFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.KV; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMergingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMergingFnRunnerTest.java index 189d83d87b9c..2deaa602c252 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMergingFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMergingFnRunnerTest.java @@ -23,15 +23,15 @@ import java.util.Collections; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.function.ThrowingFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java index 75c91783076f..2d1e323707f7 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java @@ -109,12 +109,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.Trigger; import org.apache.beam.model.pipeline.v1.RunnerApi.Trigger.Always; import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy; -import org.apache.beam.runners.core.construction.BeamUrns; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.ModelCoders; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.Timer; import org.apache.beam.runners.core.metrics.ShortIdMap; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -141,6 +135,12 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.DoFnWithExecutionInformation; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.util.construction.ModelCoders; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java index 2d1aaf0b0179..245c87f3e194 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java @@ -55,7 +55,6 @@ import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.runners.core.metrics.DistributionData; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.Labels; @@ -74,6 +73,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; diff --git a/sdks/java/io/amazon-web-services/build.gradle b/sdks/java/io/amazon-web-services/build.gradle index 35aabe1507a6..211b3b9a4a73 100644 --- a/sdks/java/io/amazon-web-services/build.gradle +++ b/sdks/java/io/amazon-web-services/build.gradle @@ -54,6 +54,7 @@ dependencies { testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation "io.findify:s3mock_2.12:0.2.6" + testImplementation library.java.commons_lang3 testImplementation library.java.hamcrest testImplementation library.java.mockito_core testImplementation library.java.junit @@ -62,7 +63,6 @@ dependencies { testImplementation 'org.elasticmq:elasticmq-rest-sqs_2.12:0.15.6' testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") - testImplementation project(path: ":runners:core-construction-java", configuration: "testRuntimeMigration") } test { diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/options/AwsModuleTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/options/AwsModuleTest.java index dfdeead59a9d..0099b08b7043 100644 --- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/options/AwsModuleTest.java +++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/options/AwsModuleTest.java @@ -40,9 +40,9 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import java.util.List; -import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.common.ReflectHelpers; +import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.hamcrest.Matchers; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index f025f9e80396..f4f9e7a2dc30 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -42,7 +42,6 @@ dependencies { permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 implementation project(":sdks:java:extensions:google-cloud-platform-core") implementation project(":sdks:java:extensions:protobuf") - implementation project(":runners:core-construction-java") implementation project(":sdks:java:extensions:arrow") implementation library.java.avro implementation library.java.bigdataoss_util diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java index 8f48890ad56d..f659cc066829 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import static org.apache.beam.runners.core.construction.TransformUpgrader.fromByteArray; -import static org.apache.beam.runners.core.construction.TransformUpgrader.toByteArray; +import static org.apache.beam.sdk.util.construction.TransformUpgrader.fromByteArray; +import static org.apache.beam.sdk.util.construction.TransformUpgrader.toByteArray; import com.google.api.services.bigquery.model.Clustering; import com.google.api.services.bigquery.model.TableRow; @@ -37,9 +37,6 @@ import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.FromBeamRowFunction; @@ -59,6 +56,9 @@ import org.apache.beam.sdk.schemas.logicaltypes.NanosDuration; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubPayloadTranslation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubPayloadTranslation.java index bc7214f44cff..b77928bd957e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubPayloadTranslation.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubPayloadTranslation.java @@ -27,10 +27,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PubSubReadPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.PubSubWritePayload; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.Read.Unbounded; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath; @@ -42,6 +38,10 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @SuppressWarnings({ diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java index 882294de1771..b9c187fe0272 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java @@ -33,8 +33,6 @@ import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.gcp.pubsublite.SubscriberOptions; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -44,6 +42,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java index b4f6ddb76f72..e95ad4678ea8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java @@ -168,8 +168,8 @@ private void storageReadWithSchema(DataFormat format) { /** * Tests a pipeline where {@link - * org.apache.beam.runners.core.construction.graph.ProjectionPushdownOptimizer} may do - * optimizations, depending on the runner. The pipeline should run successfully either way. + * org.apache.beam.sdk.util.construction.graph.ProjectionPushdownOptimizer} may do optimizations, + * depending on the runner. The pipeline should run successfully either way. */ @Test public void testBigQueryStorageReadProjectionPushdown() throws Exception { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubReadPayloadTranslationTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubReadPayloadTranslationTest.java index 4cf547dea934..e817b2887aef 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubReadPayloadTranslationTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubReadPayloadTranslationTest.java @@ -22,9 +22,6 @@ import java.util.Arrays; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PubSubReadPayload; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; @@ -34,6 +31,9 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.resourcehints.ResourceHints; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValues; import org.junit.Test; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubWritePayloadTranslationTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubWritePayloadTranslationTest.java index 02e424bc1c83..45fbab0576fb 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubWritePayloadTranslationTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubSubWritePayloadTranslationTest.java @@ -22,9 +22,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PubSubWritePayload; -import org.apache.beam.runners.core.construction.Environments; -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.io.gcp.pubsub.PubSubPayloadTranslation.PubSubWritePayloadTranslator; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSink.PubsubSink; @@ -35,6 +32,9 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.resourcehints.ResourceHints; +import org.apache.beam.sdk.util.construction.Environments; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java index a33b17b08e10..fa25f5914795 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java @@ -23,8 +23,6 @@ import org.apache.beam.model.expansion.v1.ExpansionApi; import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.expansion.service.ExpansionService; import org.apache.beam.sdk.options.ValueProvider; @@ -36,6 +34,8 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; diff --git a/sdks/java/io/json/build.gradle b/sdks/java/io/json/build.gradle index fe1f607a3696..d64342eaaceb 100644 --- a/sdks/java/io/json/build.gradle +++ b/sdks/java/io/json/build.gradle @@ -27,7 +27,6 @@ ext.summary = "IO to read and write JSON files." dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.everit_json_schema testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.junit testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index ea53a4d60dbd..16f3a39b2073 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -52,7 +52,6 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:protobuf") - implementation project(":runners:core-construction-java") implementation project(":sdks:java:expansion-service") permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 implementation library.java.avro @@ -93,7 +92,6 @@ dependencies { testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") // For testing Cross-language transforms - testImplementation project(":runners:core-construction-java") testImplementation library.java.avro testImplementation library.java.junit testImplementation library.java.powermock diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 8fd0c34cfa90..392069f1926e 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -39,8 +39,6 @@ import java.util.Set; import java.util.regex.Pattern; import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -91,6 +89,8 @@ import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.MonotonicallyIncreasing; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.WallTime; import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.util.construction.PTransformMatchers; +import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java index 71ceb43dbeee..dd859af50864 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java @@ -29,8 +29,6 @@ import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExternalConfigurationPayload; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.expansion.service.ExpansionService; import org.apache.beam.sdk.io.kafka.KafkaIO.ByteArrayKafkaRecord; @@ -44,6 +42,8 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/io/kafka/upgrade/build.gradle b/sdks/java/io/kafka/upgrade/build.gradle index 5a62fe230a33..d22900d0f7ea 100644 --- a/sdks/java/io/kafka/upgrade/build.gradle +++ b/sdks/java/io/kafka/upgrade/build.gradle @@ -31,7 +31,6 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.vendored_guava_32_1_2_jre - implementation project(":runners:core-construction-java") implementation project(":sdks:java:expansion-service") permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 permitUnusedDeclared project(":model:pipeline") diff --git a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java index 0e1dfcd21a72..ce5ccb559870 100644 --- a/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java +++ b/sdks/java/io/kafka/upgrade/src/main/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslation.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.io.kafka.upgrade; -import static org.apache.beam.runners.core.construction.TransformUpgrader.fromByteArray; -import static org.apache.beam.runners.core.construction.TransformUpgrader.toByteArray; +import static org.apache.beam.sdk.util.construction.TransformUpgrader.fromByteArray; +import static org.apache.beam.sdk.util.construction.TransformUpgrader.toByteArray; import com.google.auto.service.AutoService; import java.io.IOException; @@ -32,9 +32,6 @@ import java.util.stream.Collectors; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; -import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.runners.core.construction.SdkComponents; -import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.io.kafka.DeserializerProvider; import org.apache.beam.sdk.io.kafka.KafkaIO; import org.apache.beam.sdk.io.kafka.KafkaIO.Read; @@ -51,6 +48,9 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; +import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java b/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java index 9e8ead4f5865..18708c560018 100644 --- a/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java +++ b/sdks/java/io/kafka/upgrade/src/test/java/org/apache/beam/sdk/io/kafka/upgrade/KafkaIOTranslationTest.java @@ -27,7 +27,6 @@ import java.util.Map; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.beam.runners.core.construction.TransformUpgrader; import org.apache.beam.sdk.io.kafka.KafkaIO; import org.apache.beam.sdk.io.kafka.KafkaIO.Read; import org.apache.beam.sdk.io.kafka.KafkaIO.Write; @@ -35,6 +34,7 @@ import org.apache.beam.sdk.io.kafka.upgrade.KafkaIOTranslation.KafkaIOReadWithMetadataTranslator; import org.apache.beam.sdk.io.kafka.upgrade.KafkaIOTranslation.KafkaIOWriteTranslator; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.construction.TransformUpgrader; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java b/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java index 0da124f3e80a..94b9912f508d 100644 --- a/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java +++ b/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java @@ -61,9 +61,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -/** - * An {@link org.apache.beam.runners.core.construction.expansion.ExpansionService} useful for tests. - */ +/** An {@link org.apache.beam.sdk.util.construction.expansion.ExpansionService} useful for tests. */ @SuppressWarnings({ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) }) diff --git a/sdks/java/testing/jpms-tests/build.gradle b/sdks/java/testing/jpms-tests/build.gradle index 9b881d7a2a9e..ac2050df7d92 100644 --- a/sdks/java/testing/jpms-tests/build.gradle +++ b/sdks/java/testing/jpms-tests/build.gradle @@ -74,6 +74,7 @@ configurations { dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":sdks:java:extensions:google-cloud-platform-core") + permitUnusedDeclared project(":sdks:java:extensions:google-cloud-platform-core") // for autoservice filesystems testImplementation library.java.junit testImplementation library.java.hamcrest diff --git a/sdks/java/transform-service/build.gradle b/sdks/java/transform-service/build.gradle index 2fee0b4f1708..231f955cbe72 100644 --- a/sdks/java/transform-service/build.gradle +++ b/sdks/java/transform-service/build.gradle @@ -41,7 +41,6 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(path: ":runners:core-construction-java") implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.jackson_annotations diff --git a/sdks/java/transform-service/launcher/build.gradle b/sdks/java/transform-service/launcher/build.gradle index 5ae514af1dc9..c541ed54061c 100644 --- a/sdks/java/transform-service/launcher/build.gradle +++ b/sdks/java/transform-service/launcher/build.gradle @@ -22,8 +22,7 @@ applyJavaNature( ) description = "Apache Beam :: SDKs :: Java :: Transform Service :: Launcher" -ext.summary = """Contains code that can be used to run an transform service.""" - +ext.summary = """Builds uber jar for running transform service launcher""" // Exclude tests that need a runner test { diff --git a/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java index 4ef84b02061b..a9ae5360a859 100644 --- a/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java +++ b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java @@ -18,8 +18,6 @@ package org.apache.beam.sdk.transformservice.launcher; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import java.io.BufferedReader; import java.io.File; @@ -33,6 +31,7 @@ import java.util.ArrayList; import java.util.UUID; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -48,7 +47,7 @@ public void testLauncherCreatesCredentialsDir() throws IOException { file.deleteOnExit(); TransformServiceLauncher.forProject(projectName, 12345, null); Path expectedCredentialsDir = Paths.get(expectedTempDir.toString(), "credentials_dir"); - assertTrue(expectedCredentialsDir.toFile().exists()); + Assert.assertTrue(expectedCredentialsDir.toFile().exists()); } @Test @@ -59,7 +58,7 @@ public void testLauncherCreatesDependenciesDir() throws IOException { file.deleteOnExit(); TransformServiceLauncher.forProject(projectName, 12345, null); Path expectedCredentialsDir = Paths.get(expectedTempDir.toString(), "dependencies_dir"); - assertTrue(expectedCredentialsDir.toFile().exists()); + Assert.assertTrue(expectedCredentialsDir.toFile().exists()); } @Test @@ -89,7 +88,7 @@ public void testLauncherInstallsDependencies() throws IOException { // specified set of packages. Path expectedUpdatedRequirementsFile = Paths.get(expectedTempDir.toString(), "dependencies_dir", "requirements.txt"); - assertTrue(expectedUpdatedRequirementsFile.toFile().exists()); + Assert.assertTrue(expectedUpdatedRequirementsFile.toFile().exists()); ArrayList expectedUpdatedRequirementsFileLines = new ArrayList<>(); try (BufferedReader bufReader = @@ -101,9 +100,9 @@ public void testLauncherInstallsDependencies() throws IOException { } } - assertEquals(2, expectedUpdatedRequirementsFileLines.size()); - assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage1")); - assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage2")); + Assert.assertEquals(2, expectedUpdatedRequirementsFileLines.size()); + Assert.assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage1")); + Assert.assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage2")); } @Test @@ -153,7 +152,7 @@ public void testLauncherInstallsLocalDependencies() throws IOException { // specified set of packages. Path expectedUpdatedRequirementsFile = Paths.get(expectedTempDir.toString(), "dependencies_dir", "requirements.txt"); - assertTrue(expectedUpdatedRequirementsFile.toFile().exists()); + Assert.assertTrue(expectedUpdatedRequirementsFile.toFile().exists()); ArrayList expectedUpdatedRequirementsFileLines = new ArrayList<>(); try (BufferedReader bufReader = @@ -168,16 +167,16 @@ public void testLauncherInstallsLocalDependencies() throws IOException { // To make local packages available to the expansion service Docker containers, the temporary // requirements file should contain names of the local packages relative to the dependencies // volume and local packages should have been copied to the dependencies volume. - assertEquals(3, expectedUpdatedRequirementsFileLines.size()); - assertTrue(expectedUpdatedRequirementsFileLines.contains(dependency1FileName)); - assertTrue(expectedUpdatedRequirementsFileLines.contains(dependency2FileName)); - assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage")); + Assert.assertEquals(3, expectedUpdatedRequirementsFileLines.size()); + Assert.assertTrue(expectedUpdatedRequirementsFileLines.contains(dependency1FileName)); + Assert.assertTrue(expectedUpdatedRequirementsFileLines.contains(dependency2FileName)); + Assert.assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage")); - assertTrue( + Assert.assertTrue( Paths.get(expectedTempDir.toString(), "dependencies_dir", dependency1FileName) .toFile() .exists()); - assertTrue( + Assert.assertTrue( Paths.get(expectedTempDir.toString(), "dependencies_dir", dependency2FileName) .toFile() .exists()); diff --git a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java index 75515ad75f7c..9c335ffb9b28 100644 --- a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java +++ b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java @@ -28,8 +28,8 @@ import org.apache.beam.model.expansion.v1.ExpansionApi.ExpansionResponse; import org.apache.beam.model.expansion.v1.ExpansionServiceGrpc; import org.apache.beam.model.pipeline.v1.Endpoints; -import org.apache.beam.runners.core.construction.DefaultExpansionServiceClientFactory; -import org.apache.beam.runners.core.construction.ExpansionServiceClientFactory; +import org.apache.beam.sdk.util.construction.DefaultExpansionServiceClientFactory; +import org.apache.beam.sdk.util.construction.ExpansionServiceClientFactory; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannelBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java index 14cf9a4e0186..857ae2017afb 100644 --- a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java +++ b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java @@ -31,8 +31,8 @@ import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; -import org.apache.beam.runners.core.construction.ExpansionServiceClient; -import org.apache.beam.runners.core.construction.ExpansionServiceClientFactory; +import org.apache.beam.sdk.util.construction.ExpansionServiceClient; +import org.apache.beam.sdk.util.construction.ExpansionServiceClientFactory; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; import org.junit.Before; import org.junit.Test; diff --git a/sdks/python/apache_beam/transforms/external_java.py b/sdks/python/apache_beam/transforms/external_java.py index 29fc2587542b..534b2622c8a0 100644 --- a/sdks/python/apache_beam/transforms/external_java.py +++ b/sdks/python/apache_beam/transforms/external_java.py @@ -122,7 +122,7 @@ def run_pipeline_with_expansion_service(pipeline_options): @staticmethod def run_pipeline(pipeline_options, expansion_service, wait_until_finish=True): # The actual definitions of these transforms is in - # org.apache.beam.runners.core.construction.TestExpansionService. + # org.apache.beam.sdk.util.construction.TestExpansionService. TEST_COUNT_URN = "beam:transforms:xlang:count" TEST_FILTER_URN = "beam:transforms:xlang:filter_less_than_eq" diff --git a/settings.gradle.kts b/settings.gradle.kts index 512820fce3f1..2906095db9ed 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -116,7 +116,6 @@ include(":it:truthmatchers") include(":it:mongodb") include(":it:splunk") include(":it:neo4j") -include(":runners:core-construction-java") include(":runners:core-java") include(":runners:direct-java") include(":runners:extensions-java:metrics") diff --git a/website/www/site/content/en/contribute/runner-guide.md b/website/www/site/content/en/contribute/runner-guide.md index c2349ccea32c..36978274748b 100644 --- a/website/www/site/content/en/contribute/runner-guide.md +++ b/website/www/site/content/en/contribute/runner-guide.md @@ -553,7 +553,7 @@ language it may have been written in to begin with. If your runner is Java-based, the tools to interact with pipelines in an SDK-agnostic manner are in the `beam-runners-core-construction-java` -artifact, in the `org.apache.beam.runners.core.construction` namespace. +artifact, in the `org.apache.beam.sdk.util.construction` namespace. The utilities are named consistently, like so: * `PTransformTranslation` - registry of known transforms and standard URNs