diff --git a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Spark.groovy b/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Spark.groovy deleted file mode 100644 index 2cf4edb88f12..000000000000 --- a/.test-infra/jenkins/job_PostCommit_CrossLanguageValidatesRunner_Spark.groovy +++ /dev/null @@ -1,51 +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. - */ - -import CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -import static PythonTestProperties.CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS - -// This job runs the suite of ValidatesRunner tests against the Flink runner. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_XVR_Spark', - 'Run XVR_Spark PostCommit', 'Spark CrossLanguageValidatesRunner Tests', this) { - description('Runs the CrossLanguageValidatesRunner suite on the Spark runner.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS.each { pythonVersion -> - shell("echo \"*** RUN CROSS-LANGUAGE SPARK USING PYTHON ${pythonVersion} ***\"") - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:2:job-server:validatesCrossLanguageRunner') - commonJobProperties.setGradleSwitches(delegate) - switches("-PpythonVersion=${pythonVersion}") - // only run non-python task (e.g. GoUsingJava) once - switches("-PskipNonPythonTask=${pythonVersion != CROSS_LANGUAGE_VALIDATES_RUNNER_PYTHON_VERSIONS[0]}") - } - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Spark.groovy index 10dff5860286..f4af7dea6c0a 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Spark.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Spark.groovy @@ -36,7 +36,6 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Spark', steps { gradle { rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:2:examplesIntegrationTest') tasks(':runners:spark:3:examplesIntegrationTest') commonJobProperties.setGradleSwitches(delegate) } diff --git a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark2_Streaming.groovy b/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark2_Streaming.groovy deleted file mode 100644 index 14e1ff3d6078..000000000000 --- a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark2_Streaming.groovy +++ /dev/null @@ -1,43 +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. - */ - -import CommonJobProperties as commonJobProperties -import PostcommitJobBuilder - -// This job runs the suite of Java ValidatesRunner tests against the Spark runner in streaming mode. -PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_PVR_Spark2_Streaming', - 'Run Java Spark v2 PortableValidatesRunner Streaming', 'Java Spark v2 PortableValidatesRunner Streaming Tests', this) { - description('Runs the Java PortableValidatesRunner suite on the Spark v2 runner in streaming mode.') - - // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) - - // Publish all test results to Jenkins - publishers { - archiveJunit('**/build/test-results/**/*.xml') - } - - // Gradle goals for this job. - steps { - gradle { - rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:2:job-server:validatesPortableRunnerStreaming') - commonJobProperties.setGradleSwitches(delegate) - } - } - } diff --git a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark_Batch.groovy b/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark_Batch.groovy index ce7340f64da8..ba08cc2e4404 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark_Batch.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_PortableValidatesRunner_Spark_Batch.groovy @@ -36,9 +36,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_PVR_Spark_Batch', steps { gradle { rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:2:job-server:validatesPortableRunnerBatch') tasks(':runners:spark:3:job-server:validatesPortableRunnerBatch') - tasks(':runners:spark:2:job-server:validatesPortableRunnerDocker') tasks(':runners:spark:3:job-server:validatesPortableRunnerDocker') commonJobProperties.setGradleSwitches(delegate) } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark.groovy index 2509e6884d7d..0cb6bd2d3710 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Spark.groovy @@ -37,7 +37,6 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Spark', steps { gradle { rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:2:validatesRunner') tasks(':runners:spark:3:validatesRunner') commonJobProperties.setGradleSwitches(delegate) } diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.groovy index 7253140faa51..31d5d5cf07b4 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.groovy @@ -36,7 +36,6 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_SparkSt steps { gradle { rootBuildScriptDir(commonJobProperties.checkoutDir) - tasks(':runners:spark:2:validatesStructuredStreamingRunnerBatch') tasks(':runners:spark:3:validatesStructuredStreamingRunnerBatch') commonJobProperties.setGradleSwitches(delegate) } diff --git a/CHANGES.md b/CHANGES.md index b26b5f886fdf..cba62b5fb803 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,7 +68,7 @@ ## Breaking Changes -* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). +* The deprecated SparkRunner for Spark 2 (see [2.41.0](#2410---2022-08-23)) was finally removed ([#25263](https://github.com/apache/beam/pull/25263)). ## Deprecations diff --git a/build.gradle.kts b/build.gradle.kts index b70d8591bf83..de7bf456ec18 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -254,8 +254,6 @@ tasks.register("javaPreCommit") { dependsOn(":runners:portability:java:build") dependsOn(":runners:samza:build") dependsOn(":runners:samza:job-server:build") - dependsOn(":runners:spark:2:build") - dependsOn(":runners:spark:2:job-server:build") dependsOn(":runners:spark:3:build") dependsOn(":runners:spark:3:job-server:build") dependsOn(":runners:twister2:build") @@ -340,7 +338,6 @@ tasks.register("javaPostCommitSickbay") { dependsOn(":runners:flink:1.13:validatesRunnerSickbay") dependsOn(":runners:flink:1.14:validatesRunnerSickbay") dependsOn(":runners:flink:1.15:validatesRunnerSickbay") - dependsOn(":runners:spark:2:job-server:validatesRunnerSickbay") dependsOn(":runners:spark:3:job-server:validatesRunnerSickbay") dependsOn(":runners:direct-java:validatesRunnerSickbay") dependsOn(":runners:portability:java:validatesRunnerSickbay") @@ -353,7 +350,6 @@ tasks.register("javaHadoopVersionsTest") { dependsOn(":sdks:java:io:hcatalog:hadoopVersionsTest") dependsOn(":sdks:java:io:parquet:hadoopVersionsTest") dependsOn(":sdks:java:extensions:sorter:hadoopVersionsTest") - dependsOn(":runners:spark:2:hadoopVersionsTest") dependsOn(":runners:spark:3:hadoopVersionsTest") } @@ -548,7 +544,6 @@ tasks.register("typescriptPreCommit") { } tasks.register("pushAllDockerImages") { - dependsOn(":runners:spark:2:job-server:container:dockerPush") dependsOn(":runners:spark:3:job-server:container:dockerPush") dependsOn(":sdks:java:container:pushAll") dependsOn(":sdks:python:container:pushAll") 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 4d9c2109b751..d9ea6f0f7590 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -505,7 +505,6 @@ class BeamModulePlugin implements Plugin { def jmh_version = "1.34" // Export Spark versions, so they are defined in a single place only - project.ext.spark2_version = spark2_version project.ext.spark3_version = spark3_version // A map of maps containing common libraries used per language. To use: @@ -730,8 +729,6 @@ class BeamModulePlugin implements Plugin { slf4j_jcl : "org.slf4j:slf4j-jcl:$slf4j_version", snappy_java : "org.xerial.snappy:snappy-java:1.1.8.4", spark_core : "org.apache.spark:spark-core_2.11:$spark2_version", - spark_network_common : "org.apache.spark:spark-network-common_2.11:$spark2_version", - spark_sql : "org.apache.spark:spark-sql_2.11:$spark2_version", spark_streaming : "org.apache.spark:spark-streaming_2.11:$spark2_version", spark3_core : "org.apache.spark:spark-core_2.12:$spark3_version", spark3_network_common : "org.apache.spark:spark-network-common_2.12:$spark3_version", diff --git a/runners/spark/2/build.gradle b/runners/spark/2/build.gradle deleted file mode 100644 index e943b01e347f..000000000000 --- a/runners/spark/2/build.gradle +++ /dev/null @@ -1,33 +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. - */ - -def basePath = '..' -/* All properties required for loading the Spark build script */ -project.ext { - // Spark 2 version as defined in BeamModulePlugin - spark_version = spark2_version - spark_scala_version = '2.11' - // Copy shared sources for Spark 2 to use Spark 3 as primary version in place - copySourceBase = true - archives_base_name = 'beam-runners-spark' - // Only export Javadocs for Spark 3 - exportJavadoc = false -} - -// Load the main build script which contains all build logic. -apply from: "$basePath/spark_runner.gradle" diff --git a/runners/spark/2/job-server/build.gradle b/runners/spark/2/job-server/build.gradle deleted file mode 100644 index adb9121d1d0b..000000000000 --- a/runners/spark/2/job-server/build.gradle +++ /dev/null @@ -1,31 +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. - */ - -def basePath = '../../job-server' - -project.ext { - // Look for the source code in the parent module - main_source_dirs = ["$basePath/src/main/java"] - test_source_dirs = ["$basePath/src/test/java"] - main_resources_dirs = ["$basePath/src/main/resources"] - test_resources_dirs = ["$basePath/src/test/resources"] - archives_base_name = 'beam-runners-spark-job-server' -} - -// Load the main build script which contains all build logic. -apply from: "$basePath/spark_job_server.gradle" diff --git a/runners/spark/2/job-server/container/build.gradle b/runners/spark/2/job-server/container/build.gradle deleted file mode 100644 index 10cacb109a2a..000000000000 --- a/runners/spark/2/job-server/container/build.gradle +++ /dev/null @@ -1,27 +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. - */ - -def basePath = '../../../job-server/container' - -project.ext { - resource_path = basePath - spark_job_server_image = 'spark_job_server' -} - -// Load the main build script which contains all build logic. -apply from: "$basePath/spark_job_server_container.gradle" diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/Constants.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/Constants.java deleted file mode 100644 index 08c187ce6c6d..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/Constants.java +++ /dev/null @@ -1,25 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming; - -public class Constants { - - public static final String BEAM_SOURCE_OPTION = "beam-source"; - public static final String DEFAULT_PARALLELISM = "default-parallelism"; - public static final String PIPELINE_OPTIONS = "pipeline-options"; -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineOptions.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineOptions.java deleted file mode 100644 index 3371a403b2c9..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineOptions.java +++ /dev/null @@ -1,42 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming; - -import org.apache.beam.runners.spark.SparkCommonPipelineOptions; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; - -/** - * Spark runner {@link PipelineOptions} handles Spark execution-related configurations, such as the - * master address, and other user-related knobs. - */ -public interface SparkStructuredStreamingPipelineOptions extends SparkCommonPipelineOptions { - - /** Set to true to run the job in test mode. */ - @Default.Boolean(false) - boolean getTestMode(); - - void setTestMode(boolean testMode); - - @Description("Enable if the runner should use the currently active Spark session.") - @Default.Boolean(false) - boolean getUseActiveSparkSession(); - - void setUseActiveSparkSession(boolean value); -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineResult.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineResult.java deleted file mode 100644 index 1392ae8f0c7f..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineResult.java +++ /dev/null @@ -1,134 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming; - -import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults; - -import java.io.IOException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsAccumulator; -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.UserCodeException; -import org.apache.spark.SparkException; -import org.joda.time.Duration; - -/** Represents a Spark pipeline execution result. */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class SparkStructuredStreamingPipelineResult implements PipelineResult { - - final Future pipelineExecution; - final Runnable onTerminalState; - - PipelineResult.State state; - - SparkStructuredStreamingPipelineResult( - final Future pipelineExecution, final Runnable onTerminalState) { - this.pipelineExecution = pipelineExecution; - this.onTerminalState = onTerminalState; - // pipelineExecution is expected to have started executing eagerly. - this.state = State.RUNNING; - } - - private static RuntimeException runtimeExceptionFrom(final Throwable e) { - return (e instanceof RuntimeException) ? (RuntimeException) e : new RuntimeException(e); - } - - private static RuntimeException beamExceptionFrom(final Throwable e) { - // Scala doesn't declare checked exceptions in the bytecode, and the Java compiler - // won't let you catch something that is not declared, so we can't catch - // SparkException directly, instead we do an instanceof check. - - if (e instanceof SparkException) { - if (e.getCause() != null && e.getCause() instanceof UserCodeException) { - UserCodeException userException = (UserCodeException) e.getCause(); - return new Pipeline.PipelineExecutionException(userException.getCause()); - } else if (e.getCause() != null) { - return new Pipeline.PipelineExecutionException(e.getCause()); - } - } - - return runtimeExceptionFrom(e); - } - - private State awaitTermination(Duration duration) - throws TimeoutException, ExecutionException, InterruptedException { - pipelineExecution.get(duration.getMillis(), TimeUnit.MILLISECONDS); - // Throws an exception if the job is not finished successfully in the given time. - return PipelineResult.State.DONE; - } - - @Override - public PipelineResult.State getState() { - return state; - } - - @Override - public PipelineResult.State waitUntilFinish() { - return waitUntilFinish(Duration.millis(Long.MAX_VALUE)); - } - - @Override - public State waitUntilFinish(final Duration duration) { - try { - State finishState = awaitTermination(duration); - offerNewState(finishState); - - } catch (final TimeoutException e) { - // ignore. - } catch (final ExecutionException e) { - offerNewState(PipelineResult.State.FAILED); - throw beamExceptionFrom(e.getCause()); - } catch (final Exception e) { - offerNewState(PipelineResult.State.FAILED); - throw beamExceptionFrom(e); - } - - return state; - } - - @Override - public MetricResults metrics() { - return asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value()); - } - - @Override - public PipelineResult.State cancel() throws IOException { - offerNewState(PipelineResult.State.CANCELLED); - return state; - } - - private void offerNewState(State newState) { - State oldState = this.state; - this.state = newState; - if (!oldState.isTerminal() && newState.isTerminal()) { - try { - onTerminalState.run(); - } catch (Exception e) { - throw beamExceptionFrom(e); - } - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java deleted file mode 100644 index 98b068a9cce7..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java +++ /dev/null @@ -1,228 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming; - -import static org.apache.beam.runners.spark.SparkCommonPipelineOptions.prepareFilesToStage; - -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.metrics.MetricsPusher; -import org.apache.beam.runners.spark.structuredstreaming.aggregators.AggregatorsAccumulator; -import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetricSource; -import org.apache.beam.runners.spark.structuredstreaming.metrics.CompositeSource; -import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsAccumulator; -import org.apache.beam.runners.spark.structuredstreaming.metrics.SparkBeamMetricSource; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.PipelineTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.batch.PipelineTranslatorBatch; -import org.apache.beam.runners.spark.structuredstreaming.translation.streaming.PipelineTranslatorStreaming; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineRunner; -import org.apache.beam.sdk.metrics.MetricsEnvironment; -import org.apache.beam.sdk.metrics.MetricsOptions; -import org.apache.beam.sdk.options.ExperimentalOptions; -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.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.spark.SparkEnv$; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.metrics.MetricsSystem; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * SparkStructuredStreamingRunner is based on spark structured streaming framework and is no more - * based on RDD/DStream API. See - * https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html It is still - * experimental, its coverage of the Beam model is partial. The SparkStructuredStreamingRunner - * translate operations defined on a pipeline to a representation executable by Spark, and then - * submitting the job to Spark to be executed. If we wanted to run a Beam pipeline with the default - * options of a single threaded spark instance in local mode, we would do the following: - * - *

{@code Pipeline p = [logic for pipeline creation] SparkStructuredStreamingPipelineResult - * result = (SparkStructuredStreamingPipelineResult) p.run(); } - * - *

To create a pipeline runner to run against a different spark cluster, with a custom master url - * we would do the following: - * - *

{@code Pipeline p = [logic for pipeline creation] SparkStructuredStreamingPipelineOptions - * options = SparkPipelineOptionsFactory.create(); options.setSparkMaster("spark://host:port"); - * SparkStructuredStreamingPipelineResult result = (SparkStructuredStreamingPipelineResult) p.run(); - * } - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public final class SparkStructuredStreamingRunner - extends PipelineRunner { - - private static final Logger LOG = LoggerFactory.getLogger(SparkStructuredStreamingRunner.class); - - /** Options used in this pipeline runner. */ - private final SparkStructuredStreamingPipelineOptions options; - - /** - * Creates and returns a new SparkStructuredStreamingRunner with default options. In particular, - * against a spark instance running in local mode. - * - * @return A pipeline runner with default options. - */ - public static SparkStructuredStreamingRunner create() { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - return new SparkStructuredStreamingRunner(options); - } - - /** - * Creates and returns a new SparkStructuredStreamingRunner with specified options. - * - * @param options The SparkStructuredStreamingPipelineOptions to use when executing the job. - * @return A pipeline runner that will execute with specified options. - */ - public static SparkStructuredStreamingRunner create( - SparkStructuredStreamingPipelineOptions options) { - return new SparkStructuredStreamingRunner(options); - } - - /** - * Creates and returns a new SparkStructuredStreamingRunner with specified options. - * - * @param options The PipelineOptions to use when executing the job. - * @return A pipeline runner that will execute with specified options. - */ - public static SparkStructuredStreamingRunner fromOptions(PipelineOptions options) { - return new SparkStructuredStreamingRunner( - PipelineOptionsValidator.validate(SparkStructuredStreamingPipelineOptions.class, options)); - } - - /** - * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single - * thread. - */ - private SparkStructuredStreamingRunner(SparkStructuredStreamingPipelineOptions options) { - this.options = options; - } - - @Override - public SparkStructuredStreamingPipelineResult run(final Pipeline pipeline) { - MetricsEnvironment.setMetricsSupported(true); - - LOG.info( - "*** SparkStructuredStreamingRunner is based on spark structured streaming framework and is no more \n" - + " based on RDD/DStream API. See\n" - + " https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html\n" - + " It is still experimental, its coverage of the Beam model is partial. ***"); - - LOG.warn( - "Support for Spark 2 is deprecated, this runner will be removed in a few releases.\n" - + "Spark 2 is reaching its EOL, consider migrating to Spark 3."); - - // clear state of Aggregators, Metrics and Watermarks if exists. - AggregatorsAccumulator.clear(); - MetricsAccumulator.clear(); - - final AbstractTranslationContext translationContext = translatePipeline(pipeline); - - final ExecutorService executorService = - Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LocalSpark-thread").build()); - final Future submissionFuture = - executorService.submit( - () -> { - // TODO initialise other services: checkpointing, metrics system, listeners, ... - translationContext.startPipeline(); - }); - executorService.shutdown(); - - Runnable onTerminalState = - options.getUseActiveSparkSession() - ? () -> {} - : () -> translationContext.getSparkSession().stop(); - SparkStructuredStreamingPipelineResult result = - new SparkStructuredStreamingPipelineResult(submissionFuture, onTerminalState); - - if (options.getEnableSparkMetricSinks()) { - registerMetricsSource(options.getAppName()); - } - - MetricsPusher metricsPusher = - new MetricsPusher( - MetricsAccumulator.getInstance().value(), options.as(MetricsOptions.class), result); - metricsPusher.start(); - - if (options.getTestMode()) { - result.waitUntilFinish(); - } - - return result; - } - - private AbstractTranslationContext translatePipeline(Pipeline pipeline) { - PipelineTranslator.detectTranslationMode(pipeline, options); - - // Default to using the primitive versions of Read.Bounded and Read.Unbounded for non-portable - // execution. - // TODO(https://github.com/apache/beam/issues/20530): Use SDF read as default when we address - // performance issue. - if (!ExperimentalOptions.hasExperiment(pipeline.getOptions(), "beam_fn_api")) { - SplittableParDo.convertReadBasedSplittableDoFnsToPrimitiveReadsIfNecessary(pipeline); - } - - PipelineTranslator.replaceTransforms(pipeline, options); - prepareFilesToStage(options); - PipelineTranslator pipelineTranslator = - options.isStreaming() - ? new PipelineTranslatorStreaming(options) - : new PipelineTranslatorBatch(options); - - final JavaSparkContext jsc = - JavaSparkContext.fromSparkContext( - pipelineTranslator.getTranslationContext().getSparkSession().sparkContext()); - initAccumulators(options, jsc); - - pipelineTranslator.translate(pipeline); - return pipelineTranslator.getTranslationContext(); - } - - private void registerMetricsSource(String appName) { - final MetricsSystem metricsSystem = SparkEnv$.MODULE$.get().metricsSystem(); - final AggregatorMetricSource aggregatorMetricSource = - new AggregatorMetricSource(null, AggregatorsAccumulator.getInstance().value()); - final SparkBeamMetricSource metricsSource = new SparkBeamMetricSource(null); - final CompositeSource compositeSource = - new CompositeSource( - appName + ".Beam", - metricsSource.metricRegistry(), - aggregatorMetricSource.metricRegistry()); - // re-register the metrics in case of context re-use - metricsSystem.removeSource(compositeSource); - metricsSystem.registerSource(compositeSource); - } - - /** Init Metrics/Aggregators accumulators. This method is idempotent. */ - public static void initAccumulators( - SparkStructuredStreamingPipelineOptions opts, JavaSparkContext jsc) { - // Init metrics accumulators - MetricsAccumulator.init(jsc); - AggregatorsAccumulator.init(jsc); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunnerRegistrar.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunnerRegistrar.java deleted file mode 100644 index fb759da960ce..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunnerRegistrar.java +++ /dev/null @@ -1,54 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming; - -import com.google.auto.service.AutoService; -import org.apache.beam.sdk.PipelineRunner; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; - -/** - * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the {@link - * SparkStructuredStreamingRunner}. - * - *

{@link AutoService} will register Spark's implementations of the {@link PipelineRunner} and - * {@link PipelineOptions} as available pipeline runner services. - */ -public final class SparkStructuredStreamingRunnerRegistrar { - private SparkStructuredStreamingRunnerRegistrar() {} - - /** Registers the {@link SparkStructuredStreamingRunner}. */ - @AutoService(PipelineRunnerRegistrar.class) - public static class Runner implements PipelineRunnerRegistrar { - @Override - public Iterable>> getPipelineRunners() { - return ImmutableList.of(SparkStructuredStreamingRunner.class); - } - } - - /** Registers the {@link SparkStructuredStreamingPipelineOptions}. */ - @AutoService(PipelineOptionsRegistrar.class) - public static class Options implements PipelineOptionsRegistrar { - @Override - public Iterable> getPipelineOptions() { - return ImmutableList.of(SparkStructuredStreamingPipelineOptions.class); - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/AggregatorsAccumulator.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/AggregatorsAccumulator.java deleted file mode 100644 index bdba5509a9ba..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/AggregatorsAccumulator.java +++ /dev/null @@ -1,73 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.aggregators; - -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.util.AccumulatorV2; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * For resilience, {@link AccumulatorV2 Accumulators} are required to be wrapped in a Singleton. - * - * @see accumulatorsV2 - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class AggregatorsAccumulator { - private static final Logger LOG = LoggerFactory.getLogger(AggregatorsAccumulator.class); - - private static final String ACCUMULATOR_NAME = "Beam.Aggregators"; - - private static volatile NamedAggregatorsAccumulator instance = null; - - /** Init aggregators accumulator if it has not been initiated. This method is idempotent. */ - public static void init(JavaSparkContext jsc) { - if (instance == null) { - synchronized (AggregatorsAccumulator.class) { - if (instance == null) { - NamedAggregators namedAggregators = new NamedAggregators(); - NamedAggregatorsAccumulator accumulator = - new NamedAggregatorsAccumulator(namedAggregators); - jsc.sc().register(accumulator, ACCUMULATOR_NAME); - - instance = accumulator; - } - } - LOG.info("Instantiated aggregators accumulator: {}", instance.value()); - } - } - - public static NamedAggregatorsAccumulator getInstance() { - if (instance == null) { - throw new IllegalStateException("Aggregrators accumulator has not been instantiated"); - } else { - return instance; - } - } - - @VisibleForTesting - public static void clear() { - synchronized (AggregatorsAccumulator.class) { - instance = null; - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregators.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregators.java deleted file mode 100644 index de53458b3149..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregators.java +++ /dev/null @@ -1,113 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.aggregators; - -import java.io.Serializable; -import java.util.Map; -import java.util.TreeMap; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; - -/** - * This class wraps a map of named aggregators. Spark expects that all accumulators be declared - * before a job is launched. Beam allows aggregators to be used and incremented on the fly. We - * create a map of named aggregators and instantiate in the spark context before the job is - * launched. We can then add aggregators on the fly in Spark. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class NamedAggregators implements Serializable { - /** Map from aggregator name to current state. */ - private final Map> mNamedAggregators = new TreeMap<>(); - - /** Constructs a new NamedAggregators instance. */ - public NamedAggregators() {} - - /** - * @param name Name of aggregator to retrieve. - * @param typeClass Type class to cast the value to. - * @param Type to be returned. - * @return the value of the aggregator associated with the specified name, or null if - * the specified aggregator could not be found. - */ - public T getValue(String name, Class typeClass) { - final State state = mNamedAggregators.get(name); - return state != null ? typeClass.cast(state.render()) : null; - } - - /** @return a map of all the aggregator names and their rendered values */ - public Map renderAll() { - return ImmutableMap.copyOf(Maps.transformValues(mNamedAggregators, State::render)); - } - - /** - * Merges another NamedAggregators instance with this instance. - * - * @param other The other instance of named aggregators ot merge. - * @return This instance of Named aggregators with associated states updated to reflect the other - * instance's aggregators. - */ - public NamedAggregators merge(NamedAggregators other) { - for (Map.Entry> e : other.mNamedAggregators.entrySet()) { - String key = e.getKey(); - State otherValue = e.getValue(); - mNamedAggregators.merge(key, otherValue, NamedAggregators::merge); - } - return this; - } - - /** - * Helper method to merge States whose generic types aren't provably the same, so require some - * casting. - */ - @SuppressWarnings("unchecked") - private static State merge( - State s1, State s2) { - return ((State) s1).merge((State) s2); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - for (Map.Entry> e : mNamedAggregators.entrySet()) { - sb.append(e.getKey()).append(": ").append(e.getValue().render()).append(" "); - } - return sb.toString(); - } - - /** - * @param Input data type - * @param Intermediate data type (useful for averages) - * @param Output data type - */ - public interface State extends Serializable { - - /** @param element new element to update state */ - void update(InputT element); - - State merge(State other); - - InterT current(); - - OutputT render(); - - Combine.CombineFn getCombineFn(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregatorsAccumulator.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregatorsAccumulator.java deleted file mode 100644 index 992e63f47b8a..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregatorsAccumulator.java +++ /dev/null @@ -1,63 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.aggregators; - -import org.apache.spark.util.AccumulatorV2; - -/** {@link AccumulatorV2} implementation for {@link NamedAggregators}. */ -public class NamedAggregatorsAccumulator extends AccumulatorV2 { - private static final NamedAggregators empty = new NamedAggregators(); - - private NamedAggregators value; - - public NamedAggregatorsAccumulator(NamedAggregators value) { - this.value = value; - } - - @Override - public boolean isZero() { - return value.equals(empty); - } - - @Override - public NamedAggregatorsAccumulator copy() { - NamedAggregators newContainer = new NamedAggregators(); - newContainer.merge(value); - return new NamedAggregatorsAccumulator(newContainer); - } - - @Override - public void reset() { - this.value = new NamedAggregators(); - } - - @Override - public void add(NamedAggregators other) { - this.value.merge(other); - } - - @Override - public void merge(AccumulatorV2 other) { - this.value.merge(other.value()); - } - - @Override - public NamedAggregators value() { - return this.value; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/package-info.java deleted file mode 100644 index 11a87eed8970..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Provides internal utilities for implementing Beam aggregators using Spark accumulators. */ -package org.apache.beam.runners.spark.structuredstreaming.aggregators; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/examples/WordCount.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/examples/WordCount.java deleted file mode 100644 index dca43581f669..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/examples/WordCount.java +++ /dev/null @@ -1,132 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.examples; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.metrics.Counter; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; - -/** Duplicated from beam-examples-java to avoid dependency. */ -public class WordCount { - - /** - * Concept #2: You can make your pipeline code less verbose by defining your DoFns statically out- - * of-line. This DoFn tokenizes lines of text into individual words; we pass it to a ParDo in the - * pipeline. - */ - @SuppressWarnings("StringSplitter") - static class ExtractWordsFn extends DoFn { - private final Counter emptyLines = Metrics.counter(ExtractWordsFn.class, "emptyLines"); - - @ProcessElement - public void processElement(ProcessContext c) { - if (c.element().trim().isEmpty()) { - emptyLines.inc(); - } - - // Split the line into words. - String[] words = c.element().split("[^\\p{L}]+"); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - /** A SimpleFunction that converts a Word and Count into a printable string. */ - public static class FormatAsTextFn extends SimpleFunction, String> { - @Override - public String apply(KV input) { - return input.getKey() + ": " + input.getValue(); - } - } - - /** - * A PTransform that converts a PCollection containing lines of text into a PCollection of - * formatted word counts. - * - *

Concept #3: This is a custom composite transform that bundles two transforms (ParDo and - * Count) as a reusable PTransform subclass. Using composite transforms allows for easy reuse, - * modular testing, and an improved monitoring experience. - */ - public static class CountWords - extends PTransform, PCollection>> { - @Override - public PCollection> expand(PCollection lines) { - - // Convert lines of text into individual words. - PCollection words = lines.apply(ParDo.of(new ExtractWordsFn())); - - // Count the number of times each word occurs. - return words.apply(Count.perElement()); - } - } - - /** - * Options supported by {@link WordCount}. - * - *

Concept #4: Defining your own configuration options. Here, you can add your own arguments to - * be processed by the command-line parser, and specify default values for them. You can then - * access the options values in your pipeline code. - * - *

Inherits standard configuration options. - */ - public interface WordCountOptions extends PipelineOptions { - @Description("Path of the file to read from") - @Default.String("gs://beam-samples/shakespeare/kinglear.txt") - String getInputFile(); - - void setInputFile(String value); - - @Description("Path of the file to write to") - String getOutput(); - - void setOutput(String value); - } - - public static void main(String[] args) { - WordCountOptions options = - PipelineOptionsFactory.fromArgs(args).withValidation().as(WordCountOptions.class); - Pipeline p = Pipeline.create(options); - - // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the - // static FormatAsTextFn() to the ParDo transform. - p.apply("ReadLines", TextIO.read().from(options.getInputFile())) - .apply(new CountWords()) - .apply(MapElements.via(new FormatAsTextFn())) - .apply("WriteCounts", TextIO.write().to(options.getOutput())); - - p.run().waitUntilFinish(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/AggregatorMetric.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/AggregatorMetric.java deleted file mode 100644 index 74bea7f5255e..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/AggregatorMetric.java +++ /dev/null @@ -1,74 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricFilter; -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.runners.spark.structuredstreaming.aggregators.NamedAggregators; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** An adapter between the {@link NamedAggregators} and the Dropwizard {@link Metric} interface. */ -public class AggregatorMetric extends BeamMetricSet { - - private static final Logger LOG = LoggerFactory.getLogger(AggregatorMetric.class); - - private final NamedAggregators namedAggregators; - - private AggregatorMetric(NamedAggregators namedAggregators) { - this.namedAggregators = namedAggregators; - } - - public static AggregatorMetric of(NamedAggregators namedAggregators) { - return new AggregatorMetric(namedAggregators); - } - - @Override - public Map> getValue(String prefix, MetricFilter filter) { - Map> metrics = new HashMap<>(); - for (Map.Entry entry : namedAggregators.renderAll().entrySet()) { - String name = prefix + "." + entry.getKey(); - Object rawValue = entry.getValue(); - if (rawValue != null) { - try { - Gauge gauge = staticGauge(rawValue); - if (filter.matches(name, gauge)) { - metrics.put(name, gauge); - } - } catch (NumberFormatException e) { - LOG.warn( - "Metric `{}` of type {} can't be reported, conversion to double failed.", - name, - rawValue.getClass().getSimpleName(), - e); - } - } - } - return metrics; - } - - // Metric type is assumed to be compatible with Double - protected Gauge staticGauge(Object rawValue) throws NumberFormatException { - return rawValue instanceof Number - ? super.staticGauge((Number) rawValue) - : super.staticGauge(Double.parseDouble(rawValue.toString())); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/AggregatorMetricSource.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/AggregatorMetricSource.java deleted file mode 100644 index 406dba365cc6..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/AggregatorMetricSource.java +++ /dev/null @@ -1,49 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import com.codahale.metrics.MetricRegistry; -import org.apache.beam.runners.spark.structuredstreaming.aggregators.NamedAggregators; -import org.apache.spark.metrics.source.Source; - -/** - * A Spark {@link Source} that is tailored to expose an {@link AggregatorMetric}, wrapping an - * underlying {@link NamedAggregators} instance. - */ -public class AggregatorMetricSource implements Source { - private static final String METRIC_NAME = "Aggregators"; - - private final String name; - - private final MetricRegistry metricRegistry = new MetricRegistry(); - - public AggregatorMetricSource(final String name, final NamedAggregators aggregators) { - this.name = name; - metricRegistry.register(METRIC_NAME, AggregatorMetric.of(aggregators)); - } - - @Override - public String sourceName() { - return name; - } - - @Override - public MetricRegistry metricRegistry() { - return metricRegistry; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/BeamMetricSet.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/BeamMetricSet.java deleted file mode 100644 index 7095036f28a3..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/BeamMetricSet.java +++ /dev/null @@ -1,60 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.MetricFilter; -import java.util.Map; -import org.apache.beam.runners.spark.metrics.WithMetricsSupport; - -/** - * {@link BeamMetricSet} is a {@link Gauge} that returns a map of multiple metrics which get - * flattened in {@link WithMetricsSupport#getGauges()} for usage in {@link - * org.apache.spark.metrics.sink.Sink Spark metric sinks}. - * - *

Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry} - * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric Metrics} - * and require usage of {@link Gauge} here. - */ -// TODO: turn into MetricRegistry https://github.com/apache/beam/issues/22384 -abstract class BeamMetricSet implements Gauge>> { - - @Override - public final Map> getValue() { - return getValue("", MetricFilter.ALL); - } - - protected abstract Map> getValue(String prefix, MetricFilter filter); - - protected Gauge staticGauge(Number number) { - return new ConstantGauge(number.doubleValue()); - } - - private static class ConstantGauge implements Gauge { - private final double value; - - ConstantGauge(double value) { - this.value = value; - } - - @Override - public Double getValue() { - return value; - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/CompositeSource.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/CompositeSource.java deleted file mode 100644 index 8de06f76064f..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/CompositeSource.java +++ /dev/null @@ -1,45 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import com.codahale.metrics.MetricRegistry; -import org.apache.spark.metrics.source.Source; - -/** Composite source made up of several {@link MetricRegistry} instances. */ -public class CompositeSource implements Source { - private final String name; - private final MetricRegistry metricRegistry; - - public CompositeSource(final String name, MetricRegistry... metricRegistries) { - this.name = name; - this.metricRegistry = new MetricRegistry(); - for (MetricRegistry metricRegistry : metricRegistries) { - this.metricRegistry.registerAll(metricRegistry); - } - } - - @Override - public String sourceName() { - return name; - } - - @Override - public MetricRegistry metricRegistry() { - return metricRegistry; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsAccumulator.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsAccumulator.java deleted file mode 100644 index b319ae36487d..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsAccumulator.java +++ /dev/null @@ -1,76 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.util.AccumulatorV2; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * For resilience, {@link AccumulatorV2 Accumulators} are required to be wrapped in a Singleton. - * - * @see accumulatorsV2 - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class MetricsAccumulator { - private static final Logger LOG = LoggerFactory.getLogger(MetricsAccumulator.class); - - private static final String ACCUMULATOR_NAME = "Beam.Metrics"; - - private static volatile MetricsContainerStepMapAccumulator instance = null; - - /** Init metrics accumulator if it has not been initiated. This method is idempotent. */ - public static void init(JavaSparkContext jsc) { - if (instance == null) { - synchronized (MetricsAccumulator.class) { - if (instance == null) { - MetricsContainerStepMap metricsContainerStepMap = new SparkMetricsContainerStepMap(); - MetricsContainerStepMapAccumulator accumulator = - new MetricsContainerStepMapAccumulator(metricsContainerStepMap); - jsc.sc().register(accumulator, ACCUMULATOR_NAME); - - instance = accumulator; - } - } - LOG.info("Instantiated metrics accumulator: {}", instance.value()); - } else { - instance.reset(); - } - } - - public static MetricsContainerStepMapAccumulator getInstance() { - if (instance == null) { - throw new IllegalStateException("Metrics accumulator has not been instantiated"); - } else { - return instance; - } - } - - @VisibleForTesting - public static void clear() { - synchronized (MetricsAccumulator.class) { - instance = null; - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsContainerStepMapAccumulator.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsContainerStepMapAccumulator.java deleted file mode 100644 index 2d2a4ea17540..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsContainerStepMapAccumulator.java +++ /dev/null @@ -1,65 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.apache.spark.util.AccumulatorV2; - -/** {@link AccumulatorV2} implementation for {@link MetricsContainerStepMap}. */ -public class MetricsContainerStepMapAccumulator - extends AccumulatorV2 { - private static final MetricsContainerStepMap empty = new SparkMetricsContainerStepMap(); - - private MetricsContainerStepMap value; - - public MetricsContainerStepMapAccumulator(MetricsContainerStepMap value) { - this.value = value; - } - - @Override - public boolean isZero() { - return value.equals(empty); - } - - @Override - public MetricsContainerStepMapAccumulator copy() { - MetricsContainerStepMap newContainer = new SparkMetricsContainerStepMap(); - newContainer.updateAll(value); - return new MetricsContainerStepMapAccumulator(newContainer); - } - - @Override - public void reset() { - this.value = new SparkMetricsContainerStepMap(); - } - - @Override - public void add(MetricsContainerStepMap other) { - this.value.updateAll(other); - } - - @Override - public void merge(AccumulatorV2 other) { - this.value.updateAll(other.value()); - } - - @Override - public MetricsContainerStepMap value() { - return this.value; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetric.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetric.java deleted file mode 100644 index 0cecae4a25b9..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetric.java +++ /dev/null @@ -1,107 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Predicates.not; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricFilter; -import java.util.HashMap; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import javax.annotation.Nullable; -import org.apache.beam.sdk.metrics.DistributionResult; -import org.apache.beam.sdk.metrics.GaugeResult; -import org.apache.beam.sdk.metrics.MetricKey; -import org.apache.beam.sdk.metrics.MetricName; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.MetricResult; -import org.apache.beam.sdk.metrics.MetricResults; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams; - -/** - * An adapter between the {@link SparkMetricsContainerStepMap} and the Dropwizard {@link Metric} - * interface. - */ -class SparkBeamMetric extends BeamMetricSet { - - private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]"; - - @Override - public Map> getValue(String prefix, MetricFilter filter) { - MetricResults metricResults = - asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value()); - Map> metrics = new HashMap<>(); - MetricQueryResults allMetrics = metricResults.allMetrics(); - for (MetricResult metricResult : allMetrics.getCounters()) { - putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted()); - } - for (MetricResult metricResult : allMetrics.getDistributions()) { - DistributionResult result = metricResult.getAttempted(); - String baseName = renderName(prefix, metricResult); - putFiltered(metrics, filter, baseName + ".count", result.getCount()); - putFiltered(metrics, filter, baseName + ".sum", result.getSum()); - putFiltered(metrics, filter, baseName + ".min", result.getMin()); - putFiltered(metrics, filter, baseName + ".max", result.getMax()); - putFiltered(metrics, filter, baseName + ".mean", result.getMean()); - } - for (MetricResult metricResult : allMetrics.getGauges()) { - putFiltered( - metrics, - filter, - renderName(prefix, metricResult), - metricResult.getAttempted().getValue()); - } - return metrics; - } - - @VisibleForTesting - @SuppressWarnings("nullness") // ok to have nullable elements on stream - static String renderName(String prefix, MetricResult metricResult) { - MetricKey key = metricResult.getKey(); - MetricName name = key.metricName(); - String step = key.stepName(); - return Streams.concat( - Stream.of(prefix), // prefix is not cleaned, should it be? - Stream.of(stripSuffix(normalizePart(step))), - Stream.of(name.getNamespace(), name.getName()).map(SparkBeamMetric::normalizePart)) - .filter(not(Strings::isNullOrEmpty)) - .collect(Collectors.joining(".")); - } - - private static @Nullable String normalizePart(@Nullable String str) { - return str != null ? str.replaceAll(ILLEGAL_CHARACTERS, "_") : null; - } - - private static @Nullable String stripSuffix(@Nullable String str) { - return str != null && str.endsWith("_") ? str.substring(0, str.length() - 1) : str; - } - - private void putFiltered( - Map> metrics, MetricFilter filter, String name, Number value) { - Gauge metric = staticGauge(value); - if (filter.matches(name, metric)) { - metrics.put(name, metric); - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetricSource.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetricSource.java deleted file mode 100644 index 665dbc479c4d..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetricSource.java +++ /dev/null @@ -1,48 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import com.codahale.metrics.MetricRegistry; -import org.apache.spark.metrics.source.Source; - -/** - * A Spark {@link Source} that is tailored to expose a {@link SparkBeamMetric}, wrapping an - * underlying {@link org.apache.beam.sdk.metrics.MetricResults} instance. - */ -public class SparkBeamMetricSource implements Source { - private static final String METRIC_NAME = "Metrics"; - - private final String name; - - private final MetricRegistry metricRegistry = new MetricRegistry(); - - public SparkBeamMetricSource(final String name) { - this.name = name; - metricRegistry.register(METRIC_NAME, new SparkBeamMetric()); - } - - @Override - public String sourceName() { - return name; - } - - @Override - public MetricRegistry metricRegistry() { - return metricRegistry; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkMetricsContainerStepMap.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkMetricsContainerStepMap.java deleted file mode 100644 index 533dceb42e26..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkMetricsContainerStepMap.java +++ /dev/null @@ -1,43 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * Sole purpose of this class is to override {@link #toString()} of {@link MetricsContainerStepMap} - * in order to show meaningful metrics in Spark Web Interface. - */ -class SparkMetricsContainerStepMap extends MetricsContainerStepMap { - - @Override - public String toString() { - return asAttemptedOnlyMetricResults(this).toString(); - } - - @Override - public boolean equals(@Nullable Object o) { - return super.equals(o); - } - - @Override - public int hashCode() { - return super.hashCode(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/WithMetricsSupport.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/WithMetricsSupport.java deleted file mode 100644 index d48a229996f7..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/WithMetricsSupport.java +++ /dev/null @@ -1,91 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import com.codahale.metrics.Counter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.MetricFilter; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Timer; -import java.util.Map; -import java.util.SortedMap; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSortedMap; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Ordering; - -/** - * A {@link MetricRegistry} decorator-like that supports {@link BeamMetricSet}s as {@link Gauge - * Gauges}. - * - *

{@link MetricRegistry} is not an interface, so this is not a by-the-book decorator. That said, - * it delegates all metric related getters to the "decorated" instance. - */ -@SuppressWarnings({"rawtypes"}) // required by interface -public class WithMetricsSupport extends MetricRegistry { - - private final MetricRegistry internalMetricRegistry; - - private WithMetricsSupport(final MetricRegistry internalMetricRegistry) { - this.internalMetricRegistry = internalMetricRegistry; - } - - public static WithMetricsSupport forRegistry(final MetricRegistry metricRegistry) { - return new WithMetricsSupport(metricRegistry); - } - - @Override - public SortedMap getTimers(final MetricFilter filter) { - return internalMetricRegistry.getTimers(filter); - } - - @Override - public SortedMap getMeters(final MetricFilter filter) { - return internalMetricRegistry.getMeters(filter); - } - - @Override - public SortedMap getHistograms(final MetricFilter filter) { - return internalMetricRegistry.getHistograms(filter); - } - - @Override - public SortedMap getCounters(final MetricFilter filter) { - return internalMetricRegistry.getCounters(filter); - } - - @Override - public SortedMap getGauges(final MetricFilter filter) { - ImmutableSortedMap.Builder builder = - new ImmutableSortedMap.Builder<>(Ordering.from(String.CASE_INSENSITIVE_ORDER)); - - Map gauges = - internalMetricRegistry.getGauges( - (n, m) -> filter.matches(n, m) || m instanceof BeamMetricSet); - - for (Map.Entry entry : gauges.entrySet()) { - Gauge gauge = entry.getValue(); - if (gauge instanceof BeamMetricSet) { - builder.putAll(((BeamMetricSet) gauge).getValue(entry.getKey(), filter)); - } else { - builder.put(entry.getKey(), gauge); - } - } - return builder.build(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/package-info.java deleted file mode 100644 index 16a1a956e8e8..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Provides internal utilities for implementing Beam metrics using Spark accumulators. */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleCsvSink.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleCsvSink.java deleted file mode 100644 index c8f9139a2ebf..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleCsvSink.java +++ /dev/null @@ -1,86 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics.sink; - -import com.codahale.metrics.MetricRegistry; -import java.util.Properties; -import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric; -import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport; -import org.apache.spark.SecurityManager; -import org.apache.spark.metrics.sink.Sink; - -/** - * A {@link Sink} for Spark's - * metric system that is tailored to report {@link AggregatorMetric}s to a CSV file. - * - *

The sink is configured using Spark configuration parameters, for example: - * - *

{@code
- * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleCsvSink"
- * "spark.metrics.conf.*.sink.csv.directory"=""
- * "spark.metrics.conf.*.sink.csv.period"=10
- * "spark.metrics.conf.*.sink.csv.unit"=seconds
- * }
- */ -public class CodahaleCsvSink implements Sink { - - // Initialized reflectively as done by Spark's MetricsSystem - private final org.apache.spark.metrics.sink.CsvSink delegate; - - /** Constructor for Spark 3.1.x and earlier. */ - public CodahaleCsvSink( - final Properties properties, - final MetricRegistry metricRegistry, - final SecurityManager securityMgr) { - try { - delegate = - org.apache.spark.metrics.sink.CsvSink.class - .getConstructor(Properties.class, MetricRegistry.class, SecurityManager.class) - .newInstance(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr); - } catch (ReflectiveOperationException ex) { - throw new RuntimeException(ex); - } - } - - /** Constructor for Spark 3.2.x and later. */ - public CodahaleCsvSink(final Properties properties, final MetricRegistry metricRegistry) { - try { - delegate = - org.apache.spark.metrics.sink.CsvSink.class - .getConstructor(Properties.class, MetricRegistry.class) - .newInstance(properties, WithMetricsSupport.forRegistry(metricRegistry)); - } catch (ReflectiveOperationException ex) { - throw new RuntimeException(ex); - } - } - - @Override - public void start() { - delegate.start(); - } - - @Override - public void stop() { - delegate.stop(); - } - - @Override - public void report() { - delegate.report(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java deleted file mode 100644 index 5640c965740b..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java +++ /dev/null @@ -1,89 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics.sink; - -import com.codahale.metrics.MetricRegistry; -import java.util.Properties; -import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric; -import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport; -import org.apache.spark.SecurityManager; -import org.apache.spark.metrics.sink.Sink; - -/** - * A {@link Sink} for Spark's - * metric system that is tailored to report {@link AggregatorMetric}s to Graphite. - * - *

The sink is configured using Spark configuration parameters, for example: - * - *

{@code
- * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
- * "spark.metrics.conf.*.sink.graphite.host"=""
- * "spark.metrics.conf.*.sink.graphite.port"=
- * "spark.metrics.conf.*.sink.graphite.period"=10
- * "spark.metrics.conf.*.sink.graphite.unit"=seconds
- * "spark.metrics.conf.*.sink.graphite.prefix"=""
- * "spark.metrics.conf.*.sink.graphite.regex"=""
- * }
- */ -public class CodahaleGraphiteSink implements Sink { - - // Initialized reflectively as done by Spark's MetricsSystem - private final org.apache.spark.metrics.sink.GraphiteSink delegate; - - /** Constructor for Spark 3.1.x and earlier. */ - public CodahaleGraphiteSink( - final Properties properties, - final MetricRegistry metricRegistry, - final org.apache.spark.SecurityManager securityMgr) { - try { - delegate = - org.apache.spark.metrics.sink.GraphiteSink.class - .getConstructor(Properties.class, MetricRegistry.class, SecurityManager.class) - .newInstance(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr); - } catch (ReflectiveOperationException ex) { - throw new RuntimeException(ex); - } - } - - /** Constructor for Spark 3.2.x and later. */ - public CodahaleGraphiteSink(final Properties properties, final MetricRegistry metricRegistry) { - try { - delegate = - org.apache.spark.metrics.sink.GraphiteSink.class - .getConstructor(Properties.class, MetricRegistry.class) - .newInstance(properties, WithMetricsSupport.forRegistry(metricRegistry)); - } catch (ReflectiveOperationException ex) { - throw new RuntimeException(ex); - } - } - - @Override - public void start() { - delegate.start(); - } - - @Override - public void stop() { - delegate.stop(); - } - - @Override - public void report() { - delegate.report(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/package-info.java deleted file mode 100644 index 427e5441c579..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Spark sinks that supports beam metrics and aggregators. */ -package org.apache.beam.runners.spark.structuredstreaming.metrics.sink; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/package-info.java deleted file mode 100644 index aefeb282f8f4..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Internal implementation of the Beam runner for Apache Spark. */ -package org.apache.beam.runners.spark.structuredstreaming; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/AbstractTranslationContext.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/AbstractTranslationContext.java deleted file mode 100644 index aed287ba6d56..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/AbstractTranslationContext.java +++ /dev/null @@ -1,235 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -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.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VoidCoder; -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.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; -import org.apache.spark.api.java.function.ForeachFunction; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.ForeachWriter; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.streaming.DataStreamWriter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Base class that gives a context for {@link PTransform} translation: keeping track of the - * datasets, the {@link SparkSession}, the current transform being translated. - */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public abstract class AbstractTranslationContext { - - private static final Logger LOG = LoggerFactory.getLogger(AbstractTranslationContext.class); - - /** All the datasets of the DAG. */ - private final Map> datasets; - /** datasets that are not used as input to other datasets (leaves of the DAG). */ - private final Set> leaves; - - private final SerializablePipelineOptions serializablePipelineOptions; - - @SuppressFBWarnings("URF_UNREAD_FIELD") // make spotbugs happy - private AppliedPTransform currentTransform; - - @SuppressFBWarnings("URF_UNREAD_FIELD") // make spotbugs happy - private final SparkSession sparkSession; - - private final Map, Dataset> broadcastDataSets; - - public AbstractTranslationContext(SparkStructuredStreamingPipelineOptions options) { - this.sparkSession = SparkSessionFactory.getOrCreateSession(options); - this.serializablePipelineOptions = new SerializablePipelineOptions(options); - this.datasets = new HashMap<>(); - this.leaves = new HashSet<>(); - this.broadcastDataSets = new HashMap<>(); - } - - public SparkSession getSparkSession() { - return sparkSession; - } - - public SerializablePipelineOptions getSerializableOptions() { - return serializablePipelineOptions; - } - - // -------------------------------------------------------------------------------------------- - // Transforms methods - // -------------------------------------------------------------------------------------------- - public void setCurrentTransform(AppliedPTransform currentTransform) { - this.currentTransform = currentTransform; - } - - public AppliedPTransform getCurrentTransform() { - return currentTransform; - } - - // -------------------------------------------------------------------------------------------- - // Datasets methods - // -------------------------------------------------------------------------------------------- - @SuppressWarnings("unchecked") - public Dataset emptyDataset() { - return (Dataset) sparkSession.emptyDataset(EncoderHelpers.fromBeamCoder(VoidCoder.of())); - } - - @SuppressWarnings("unchecked") - public Dataset> getDataset(PValue value) { - Dataset dataset = datasets.get(value); - // assume that the Dataset is used as an input if retrieved here. So it is not a leaf anymore - leaves.remove(dataset); - return (Dataset>) dataset; - } - - /** - * TODO: All these 3 methods (putDataset*) are temporary and they are used only for generics type - * checking. We should unify them in the future. - */ - public void putDatasetWildcard(PValue value, Dataset> dataset) { - if (!datasets.containsKey(value)) { - datasets.put(value, dataset); - leaves.add(dataset); - } - } - - public void putDataset(PValue value, Dataset> dataset) { - if (!datasets.containsKey(value)) { - datasets.put(value, dataset); - leaves.add(dataset); - } - } - - public void setSideInputDataset( - PCollectionView value, Dataset> set) { - if (!broadcastDataSets.containsKey(value)) { - broadcastDataSets.put(value, set); - } - } - - @SuppressWarnings("unchecked") - public Dataset getSideInputDataSet(PCollectionView value) { - return (Dataset) broadcastDataSets.get(value); - } - - // -------------------------------------------------------------------------------------------- - // PCollections methods - // -------------------------------------------------------------------------------------------- - public PValue getInput() { - return Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); - } - - public Map, PCollection> getInputs() { - return currentTransform.getInputs(); - } - - public PValue getOutput() { - return Iterables.getOnlyElement(currentTransform.getOutputs().values()); - } - - public Map, PCollection> getOutputs() { - return currentTransform.getOutputs(); - } - - @SuppressWarnings("unchecked") - public Map, Coder> getOutputCoders() { - return currentTransform.getOutputs().entrySet().stream() - .filter(e -> e.getValue() instanceof PCollection) - .collect(Collectors.toMap(Map.Entry::getKey, e -> ((PCollection) e.getValue()).getCoder())); - } - - // -------------------------------------------------------------------------------------------- - // Pipeline methods - // -------------------------------------------------------------------------------------------- - - /** Starts the pipeline. */ - public void startPipeline() { - SparkStructuredStreamingPipelineOptions options = - serializablePipelineOptions.get().as(SparkStructuredStreamingPipelineOptions.class); - int datasetIndex = 0; - for (Dataset dataset : leaves) { - if (options.isStreaming()) { - // TODO: deal with Beam Discarding, Accumulating and Accumulating & Retracting outputmodes - // with DatastreamWriter.outputMode - DataStreamWriter dataStreamWriter = dataset.writeStream(); - // spark sets a default checkpoint dir if not set. - if (options.getCheckpointDir() != null) { - dataStreamWriter = - dataStreamWriter.option("checkpointLocation", options.getCheckpointDir()); - } - launchStreaming(dataStreamWriter.foreach(new NoOpForeachWriter<>())); - } else { - if (options.getTestMode()) { - LOG.debug("**** dataset {} catalyst execution plans ****", ++datasetIndex); - dataset.explain(true); - } - // apply a dummy fn just to apply foreach action that will trigger the pipeline run in - // spark - dataset.foreach((ForeachFunction) t -> {}); - } - } - } - - public abstract void launchStreaming(DataStreamWriter dataStreamWriter); - - public static void printDatasetContent(Dataset dataset) { - // cannot use dataset.show because dataset schema is binary so it will print binary - // code. - List windowedValues = dataset.collectAsList(); - for (WindowedValue windowedValue : windowedValues) { - LOG.debug("**** dataset content {} ****", windowedValue.toString()); - } - } - - private static class NoOpForeachWriter extends ForeachWriter { - - @Override - public boolean open(long partitionId, long epochId) { - return false; - } - - @Override - public void process(T value) { - // do nothing - } - - @Override - public void close(Throwable errorOrNull) { - // do nothing - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java deleted file mode 100644 index 0f851d9588d9..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java +++ /dev/null @@ -1,202 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation; - -import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.runners.spark.structuredstreaming.translation.batch.PipelineTranslatorBatch; -import org.apache.beam.runners.spark.structuredstreaming.translation.streaming.PipelineTranslatorStreaming; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.StreamingOptions; -import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PValue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts. - * It also does the pipeline preparation: mode detection, transforms replacement, classpath - * preparation. If we have a streaming job, it is instantiated as a {@link - * PipelineTranslatorStreaming}. If we have a batch job, it is instantiated as a {@link - * PipelineTranslatorBatch}. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults { - private int depth = 0; - private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class); - protected AbstractTranslationContext translationContext; - - // -------------------------------------------------------------------------------------------- - // Pipeline preparation methods - // -------------------------------------------------------------------------------------------- - public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) { - pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming())); - } - - /** - * Visit the pipeline to determine the translation mode (batch/streaming) and update options - * accordingly. - */ - public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) { - TranslationModeDetector detector = new TranslationModeDetector(); - pipeline.traverseTopologically(detector); - if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) { - options.setStreaming(true); - } - } - - /** The translation mode of the Beam Pipeline. */ - private enum TranslationMode { - - /** Uses the batch mode. */ - BATCH, - - /** Uses the streaming mode. */ - STREAMING - } - - /** Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline. */ - private static class TranslationModeDetector extends Pipeline.PipelineVisitor.Defaults { - private static final Logger LOG = LoggerFactory.getLogger(TranslationModeDetector.class); - - private TranslationMode translationMode; - - TranslationModeDetector(TranslationMode defaultMode) { - this.translationMode = defaultMode; - } - - TranslationModeDetector() { - this(TranslationMode.BATCH); - } - - TranslationMode getTranslationMode() { - return translationMode; - } - - @Override - public void visitValue(PValue value, TransformHierarchy.Node producer) { - if (translationMode.equals(TranslationMode.BATCH)) { - if (value instanceof PCollection - && ((PCollection) value).isBounded() == PCollection.IsBounded.UNBOUNDED) { - LOG.info( - "Found unbounded PCollection {}. Switching to streaming execution.", value.getName()); - translationMode = TranslationMode.STREAMING; - } - } - } - } - - // -------------------------------------------------------------------------------------------- - // Pipeline utility methods - // -------------------------------------------------------------------------------------------- - - /** - * Utility formatting method. - * - * @param n number of spaces to generate - * @return String with "|" followed by n spaces - */ - private static String genSpaces(int n) { - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < n; i++) { - builder.append("| "); - } - return builder.toString(); - } - - /** Get a {@link TransformTranslator} for the given {@link TransformHierarchy.Node}. */ - protected abstract TransformTranslator getTransformTranslator(TransformHierarchy.Node node); - - /** Apply the given TransformTranslator to the given node. */ - private > void applyTransformTranslator( - TransformHierarchy.Node node, TransformTranslator transformTranslator) { - // create the applied PTransform on the translationContext - translationContext.setCurrentTransform(node.toAppliedPTransform(getPipeline())); - - // avoid type capture - @SuppressWarnings("unchecked") - T typedTransform = (T) node.getTransform(); - @SuppressWarnings("unchecked") - TransformTranslator typedTransformTranslator = (TransformTranslator) transformTranslator; - - // apply the transformTranslator - typedTransformTranslator.translateTransform(typedTransform, translationContext); - } - - // -------------------------------------------------------------------------------------------- - // Pipeline visitor entry point - // -------------------------------------------------------------------------------------------- - - /** - * Translates the pipeline by passing this class as a visitor. - * - * @param pipeline The pipeline to be translated - */ - public void translate(Pipeline pipeline) { - LOG.debug("starting translation of the pipeline using {}", getClass().getName()); - pipeline.traverseTopologically(this); - } - - // -------------------------------------------------------------------------------------------- - // Pipeline Visitor Methods - // -------------------------------------------------------------------------------------------- - - @Override - public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { - LOG.debug("{} enterCompositeTransform- {}", genSpaces(depth), node.getFullName()); - depth++; - - TransformTranslator transformTranslator = getTransformTranslator(node); - - if (transformTranslator != null) { - applyTransformTranslator(node, transformTranslator); - LOG.debug("{} translated- {}", genSpaces(depth), node.getFullName()); - return CompositeBehavior.DO_NOT_ENTER_TRANSFORM; - } else { - return CompositeBehavior.ENTER_TRANSFORM; - } - } - - @Override - public void leaveCompositeTransform(TransformHierarchy.Node node) { - depth--; - LOG.debug("{} leaveCompositeTransform- {}", genSpaces(depth), node.getFullName()); - } - - @Override - public void visitPrimitiveTransform(TransformHierarchy.Node node) { - LOG.debug("{} visitPrimitiveTransform- {}", genSpaces(depth), node.getFullName()); - - // get the transformation corresponding to the node we are - // currently visiting and translate it into its Spark alternative. - TransformTranslator transformTranslator = getTransformTranslator(node); - if (transformTranslator == null) { - String transformUrn = PTransformTranslation.urnForTransform(node.getTransform()); - throw new UnsupportedOperationException( - "The transform " + transformUrn + " is currently not supported."); - } - applyTransformTranslator(node, transformTranslator); - } - - public AbstractTranslationContext getTranslationContext() { - return translationContext; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java deleted file mode 100644 index d8430f5f1300..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java +++ /dev/null @@ -1,71 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation; - -import java.util.List; -import javax.annotation.Nullable; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.spark.SparkConf; -import org.apache.spark.sql.SparkSession; - -public class SparkSessionFactory { - - /** - * Gets active {@link SparkSession} or creates one using {@link - * SparkStructuredStreamingPipelineOptions}. - */ - public static SparkSession getOrCreateSession(SparkStructuredStreamingPipelineOptions options) { - if (options.getUseActiveSparkSession()) { - return SparkSession.active(); - } - return sessionBuilder(options.getSparkMaster(), options.getAppName(), options.getFilesToStage()) - .getOrCreate(); - } - - /** Creates Spark session builder with some optimizations for local mode, e.g. in tests. */ - public static SparkSession.Builder sessionBuilder(String master) { - return sessionBuilder(master, null, null); - } - - private static SparkSession.Builder sessionBuilder( - String master, @Nullable String appName, @Nullable List jars) { - SparkConf sparkConf = new SparkConf(); - sparkConf.setMaster(master); - if (appName != null) { - sparkConf.setAppName(appName); - } - if (jars != null && !jars.isEmpty()) { - sparkConf.setJars(jars.toArray(new String[0])); - } - - // By default, Spark defines 200 as a number of sql partitions. This seems too much for local - // mode, so try to align with value of "sparkMaster" option in this case. - // We should not overwrite this value (or any user-defined spark configuration value) if the - // user has already configured it. - if (master != null - && master.startsWith("local[") - && System.getProperty("spark.sql.shuffle.partitions") == null) { - int numPartitions = - Integer.parseInt(master.substring("local[".length(), master.length() - 1)); - if (numPartitions > 0) { - sparkConf.set("spark.sql.shuffle.partitions", String.valueOf(numPartitions)); - } - } - return SparkSession.builder().config(sparkConf); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkTransformOverrides.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkTransformOverrides.java deleted file mode 100644 index 996f60cb7478..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkTransformOverrides.java +++ /dev/null @@ -1,56 +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. - */ -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.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; - -/** {@link PTransform} overrides for Spark runner. */ -@SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) -}) -class SparkTransformOverrides { - public static List getDefaultOverrides(boolean streaming) { - ImmutableList.Builder builder = ImmutableList.builder(); - // TODO: [https://github.com/apache/beam/issues/19107] Support @RequiresStableInput on Spark - // runner - builder.add( - PTransformOverride.of( - PTransformMatchers.requiresStableInputParDoMulti(), - UnsupportedOverrideFactory.withMessage( - "Spark runner currently doesn't support @RequiresStableInput annotation."))); - if (!streaming) { - builder - .add( - PTransformOverride.of( - PTransformMatchers.splittableParDo(), new SplittableParDo.OverrideFactory())) - .add( - PTransformOverride.of( - PTransformMatchers.urnEqualTo(PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN), - new SplittableParDoNaiveBounded.OverrideFactory())); - } - return builder.build(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java deleted file mode 100644 index 61580aed2192..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java +++ /dev/null @@ -1,31 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation; - -import java.io.Serializable; -import org.apache.beam.sdk.transforms.PTransform; - -/** Supports translation between a Beam transform, and Spark's operations on Datasets. */ -@SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) -}) -public interface TransformTranslator extends Serializable { - - /** Base class for translators of {@link PTransform}. */ - void translateTransform(TransformT transform, AbstractTranslationContext context); -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TranslationContext.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TranslationContext.java deleted file mode 100644 index ece050e5cc9d..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TranslationContext.java +++ /dev/null @@ -1,34 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation; - -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.spark.sql.streaming.DataStreamWriter; - -/** Subclass of {@link AbstractTranslationContext} that address spark breaking changes. */ -public class TranslationContext extends AbstractTranslationContext { - - public TranslationContext(SparkStructuredStreamingPipelineOptions options) { - super(options); - } - - @Override - public void launchStreaming(DataStreamWriter dataStreamWriter) { - dataStreamWriter.start(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorCombiner.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorCombiner.java deleted file mode 100644 index d0f46ea807c2..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorCombiner.java +++ /dev/null @@ -1,270 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; -import org.apache.spark.sql.Encoder; -import org.apache.spark.sql.expressions.Aggregator; -import org.joda.time.Instant; -import scala.Tuple2; - -/** An {@link Aggregator} for the Spark Batch Runner. - * The accumulator is a {@code Iterable> because an {@code InputT} can be in multiple windows. So, when accumulating {@code InputT} values, we create one accumulator per input window. - * */ -class AggregatorCombiner - extends Aggregator< - WindowedValue>, - Iterable>, - Iterable>> { - - private final Combine.CombineFn combineFn; - private WindowingStrategy windowingStrategy; - private TimestampCombiner timestampCombiner; - private Coder accumulatorCoder; - private IterableCoder> bufferEncoder; - private IterableCoder> outputCoder; - - public AggregatorCombiner( - Combine.CombineFn combineFn, - WindowingStrategy windowingStrategy, - Coder accumulatorCoder, - Coder outputCoder) { - this.combineFn = combineFn; - this.windowingStrategy = (WindowingStrategy) windowingStrategy; - this.timestampCombiner = windowingStrategy.getTimestampCombiner(); - this.accumulatorCoder = accumulatorCoder; - this.bufferEncoder = - IterableCoder.of( - WindowedValue.FullWindowedValueCoder.of( - accumulatorCoder, windowingStrategy.getWindowFn().windowCoder())); - this.outputCoder = - IterableCoder.of( - WindowedValue.FullWindowedValueCoder.of( - outputCoder, windowingStrategy.getWindowFn().windowCoder())); - } - - @Override - public Iterable> zero() { - return new ArrayList<>(); - } - - private Iterable> createAccumulator(WindowedValue> inputWv) { - // need to create an accumulator because combineFn can modify its input accumulator. - AccumT accumulator = combineFn.createAccumulator(); - AccumT accumT = combineFn.addInput(accumulator, inputWv.getValue().getValue()); - return Lists.newArrayList( - WindowedValue.of(accumT, inputWv.getTimestamp(), inputWv.getWindows(), inputWv.getPane())); - } - - @Override - public Iterable> reduce( - Iterable> accumulators, WindowedValue> inputWv) { - return merge(accumulators, createAccumulator(inputWv)); - } - - @Override - public Iterable> merge( - Iterable> accumulators1, - Iterable> accumulators2) { - - // merge the windows of all the accumulators - Iterable> accumulators = Iterables.concat(accumulators1, accumulators2); - Set accumulatorsWindows = collectAccumulatorsWindows(accumulators); - Map windowToMergeResult; - try { - windowToMergeResult = mergeWindows(windowingStrategy, accumulatorsWindows); - } catch (Exception e) { - throw new RuntimeException("Unable to merge accumulators windows", e); - } - - // group accumulators by their merged window - Map>> mergedWindowToAccumulators = new HashMap<>(); - for (WindowedValue accumulatorWv : accumulators) { - // Encode a version of the accumulator if it is in multiple windows. The combineFn is able to - // mutate the accumulator instance and this could lead to incorrect results if the same - // instance is merged across multiple windows so we decode a new instance as needed. This - // prevents issues during merging of accumulators. - byte[] encodedAccumT = null; - if (accumulatorWv.getWindows().size() > 1) { - try { - encodedAccumT = CoderUtils.encodeToByteArray(accumulatorCoder, accumulatorWv.getValue()); - } catch (CoderException e) { - throw new RuntimeException( - String.format( - "Unable to encode accumulator %s with coder %s.", - accumulatorWv.getValue(), accumulatorCoder), - e); - } - } - for (BoundedWindow accumulatorWindow : accumulatorWv.getWindows()) { - W mergedWindowForAccumulator = windowToMergeResult.get(accumulatorWindow); - mergedWindowForAccumulator = - (mergedWindowForAccumulator == null) - ? (W) accumulatorWindow - : mergedWindowForAccumulator; - - // Decode a copy of the accumulator when necessary. - AccumT accumT; - if (encodedAccumT != null) { - try { - accumT = CoderUtils.decodeFromByteArray(accumulatorCoder, encodedAccumT); - } catch (CoderException e) { - throw new RuntimeException( - String.format( - "Unable to encode accumulator %s with coder %s.", - accumulatorWv.getValue(), accumulatorCoder), - e); - } - } else { - accumT = accumulatorWv.getValue(); - } - - // we need only the timestamp and the AccumT, we create a tuple - Tuple2 accumAndInstant = - new Tuple2<>( - accumT, - timestampCombiner.assign(mergedWindowForAccumulator, accumulatorWv.getTimestamp())); - if (mergedWindowToAccumulators.get(mergedWindowForAccumulator) == null) { - mergedWindowToAccumulators.put( - mergedWindowForAccumulator, Lists.newArrayList(accumAndInstant)); - } else { - mergedWindowToAccumulators.get(mergedWindowForAccumulator).add(accumAndInstant); - } - } - } - // merge the accumulators for each mergedWindow - List> result = new ArrayList<>(); - for (Map.Entry>> entry : - mergedWindowToAccumulators.entrySet()) { - W mergedWindow = entry.getKey(); - List> accumsAndInstantsForMergedWindow = entry.getValue(); - - // we need to create the first accumulator because combineFn.mergerAccumulators can modify the - // first accumulator - AccumT first = combineFn.createAccumulator(); - Iterable accumulatorsToMerge = - Iterables.concat( - Collections.singleton(first), - accumsAndInstantsForMergedWindow.stream() - .map(x -> x._1()) - .collect(Collectors.toList())); - result.add( - WindowedValue.of( - combineFn.mergeAccumulators(accumulatorsToMerge), - timestampCombiner.combine( - accumsAndInstantsForMergedWindow.stream() - .map(x -> x._2()) - .collect(Collectors.toList())), - mergedWindow, - PaneInfo.NO_FIRING)); - } - return result; - } - - @Override - public Iterable> finish(Iterable> reduction) { - List> result = new ArrayList<>(); - for (WindowedValue windowedValue : reduction) { - result.add(windowedValue.withValue(combineFn.extractOutput(windowedValue.getValue()))); - } - return result; - } - - @Override - public Encoder>> bufferEncoder() { - return EncoderHelpers.fromBeamCoder(bufferEncoder); - } - - @Override - public Encoder>> outputEncoder() { - return EncoderHelpers.fromBeamCoder(outputCoder); - } - - private Set collectAccumulatorsWindows(Iterable> accumulators) { - Set windows = new HashSet<>(); - for (WindowedValue accumulator : accumulators) { - for (BoundedWindow untypedWindow : accumulator.getWindows()) { - @SuppressWarnings("unchecked") - W window = (W) untypedWindow; - windows.add(window); - } - } - return windows; - } - - private Map mergeWindows(WindowingStrategy windowingStrategy, Set windows) - throws Exception { - WindowFn windowFn = windowingStrategy.getWindowFn(); - - if (!windowingStrategy.needsMerge()) { - // Return an empty map, indicating that every window is not merged. - return Collections.emptyMap(); - } - - Map windowToMergeResult = new HashMap<>(); - windowFn.mergeWindows(new MergeContextImpl(windowFn, windows, windowToMergeResult)); - return windowToMergeResult; - } - - private class MergeContextImpl extends WindowFn.MergeContext { - - private Set windows; - private Map windowToMergeResult; - - MergeContextImpl(WindowFn windowFn, Set windows, Map windowToMergeResult) { - windowFn.super(); - this.windows = windows; - this.windowToMergeResult = windowToMergeResult; - } - - @Override - public Collection windows() { - return windows; - } - - @Override - public void merge(Collection toBeMerged, W mergeResult) throws Exception { - for (W w : toBeMerged) { - windowToMergeResult.put(w, mergeResult); - } - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java deleted file mode 100644 index 2b0cf8be9955..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java +++ /dev/null @@ -1,114 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.KVHelpers; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.transforms.Combine; -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.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.spark.api.java.function.FlatMapFunction; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.KeyValueGroupedDataset; -import scala.Tuple2; - -@SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) -}) -class CombinePerKeyTranslatorBatch - implements TransformTranslator< - PTransform>, PCollection>>> { - - @Override - public void translateTransform( - PTransform>, PCollection>> transform, - AbstractTranslationContext context) { - - Combine.PerKey combineTransform = (Combine.PerKey) transform; - @SuppressWarnings("unchecked") - final PCollection> input = (PCollection>) context.getInput(); - @SuppressWarnings("unchecked") - final PCollection> output = (PCollection>) context.getOutput(); - @SuppressWarnings("unchecked") - final Combine.CombineFn combineFn = - (Combine.CombineFn) combineTransform.getFn(); - WindowingStrategy windowingStrategy = input.getWindowingStrategy(); - - Dataset>> inputDataset = context.getDataset(input); - - KvCoder inputCoder = (KvCoder) input.getCoder(); - Coder keyCoder = inputCoder.getKeyCoder(); - KvCoder outputKVCoder = (KvCoder) output.getCoder(); - Coder outputCoder = outputKVCoder.getValueCoder(); - - KeyValueGroupedDataset>> groupedDataset = - inputDataset.groupByKey(KVHelpers.extractKey(), EncoderHelpers.fromBeamCoder(keyCoder)); - - Coder accumulatorCoder = null; - try { - accumulatorCoder = - combineFn.getAccumulatorCoder( - input.getPipeline().getCoderRegistry(), inputCoder.getValueCoder()); - } catch (CannotProvideCoderException e) { - throw new RuntimeException(e); - } - - Dataset>>> combinedDataset = - groupedDataset.agg( - new AggregatorCombiner( - combineFn, windowingStrategy, accumulatorCoder, outputCoder) - .toColumn()); - - // expand the list into separate elements and put the key back into the elements - WindowedValue.WindowedValueCoder> wvCoder = - WindowedValue.FullWindowedValueCoder.of( - outputKVCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); - Dataset>> outputDataset = - combinedDataset.flatMap( - (FlatMapFunction< - Tuple2>>, WindowedValue>>) - tuple2 -> { - K key = tuple2._1(); - Iterable> windowedValues = tuple2._2(); - List>> result = new ArrayList<>(); - for (WindowedValue windowedValue : windowedValues) { - KV kv = KV.of(key, windowedValue.getValue()); - result.add( - WindowedValue.of( - kv, - windowedValue.getTimestamp(), - windowedValue.getWindows(), - windowedValue.getPane())); - } - return result.iterator(); - }, - EncoderHelpers.fromBeamCoder(wvCoder)); - context.putDataset(output, outputDataset); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CreatePCollectionViewTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CreatePCollectionViewTranslatorBatch.java deleted file mode 100644 index ae1eeced3281..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CreatePCollectionViewTranslatorBatch.java +++ /dev/null @@ -1,60 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.IOException; -import org.apache.beam.runners.core.construction.CreatePCollectionViewTranslation; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -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.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.spark.sql.Dataset; - -class CreatePCollectionViewTranslatorBatch - implements TransformTranslator, PCollection>> { - - @Override - public void translateTransform( - PTransform, PCollection> transform, - AbstractTranslationContext context) { - - Dataset> inputDataSet = context.getDataset(context.getInput()); - - @SuppressWarnings("unchecked") - AppliedPTransform< - PCollection, - PCollection, - PTransform, PCollection>> - application = - (AppliedPTransform< - PCollection, - PCollection, - PTransform, PCollection>>) - context.getCurrentTransform(); - PCollectionView input; - try { - input = CreatePCollectionViewTranslation.getView(application); - } catch (IOException e) { - throw new RuntimeException(e); - } - context.setSideInputDataset(input, inputDataSet); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DatasetSourceBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DatasetSourceBatch.java deleted file mode 100644 index 766d143d6d7d..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DatasetSourceBatch.java +++ /dev/null @@ -1,160 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import static org.apache.beam.runners.spark.structuredstreaming.Constants.BEAM_SOURCE_OPTION; -import static org.apache.beam.runners.spark.structuredstreaming.Constants.DEFAULT_PARALLELISM; -import static org.apache.beam.runners.spark.structuredstreaming.Constants.PIPELINE_OPTIONS; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; - -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.serialization.Base64Serializer; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.RowHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.SchemaHelpers; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.ContinuousReadSupport; -import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; -import org.apache.spark.sql.sources.v2.ReadSupport; -import org.apache.spark.sql.sources.v2.reader.DataSourceReader; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.InputPartitionReader; -import org.apache.spark.sql.types.StructType; - -/** - * This is a spark structured streaming {@link DataSourceV2} implementation that wraps an {@link - * BoundedSource}.As Continuous streaming is tagged experimental in spark (no aggregation support + - * no exactly once guaranty), this class does no implement {@link ContinuousReadSupport}. - */ -public class DatasetSourceBatch implements DataSourceV2, ReadSupport { - - @Override - public DataSourceReader createReader(DataSourceOptions options) { - return new DatasetReader<>(options); - } - - /** This class is mapped to Beam {@link BoundedSource}. */ - private static class DatasetReader implements DataSourceReader, Serializable { - - private int numPartitions; - private BoundedSource source; - private SerializablePipelineOptions serializablePipelineOptions; - - @SuppressWarnings("unchecked") - private DatasetReader(DataSourceOptions options) { - if (!options.get(BEAM_SOURCE_OPTION).isPresent()) { - throw new RuntimeException("Beam source was not set in DataSource options"); - } - this.source = - Base64Serializer.deserializeUnchecked( - options.get(BEAM_SOURCE_OPTION).get(), BoundedSource.class); - - if (!options.get(DEFAULT_PARALLELISM).isPresent()) { - throw new RuntimeException("Spark default parallelism was not set in DataSource options"); - } - this.numPartitions = Integer.parseInt(options.get(DEFAULT_PARALLELISM).get()); - checkArgument(numPartitions > 0, "Number of partitions must be greater than zero."); - - if (!options.get(PIPELINE_OPTIONS).isPresent()) { - throw new RuntimeException("Beam pipelineOptions were not set in DataSource options"); - } - this.serializablePipelineOptions = - new SerializablePipelineOptions(options.get(PIPELINE_OPTIONS).get()); - } - - @Override - public StructType readSchema() { - // TODO: find a way to extend schema with a WindowedValue schema - return SchemaHelpers.binarySchema(); - } - - @Override - public List> planInputPartitions() { - PipelineOptions options = serializablePipelineOptions.get(); - List> result = new ArrayList<>(); - long desiredSizeBytes; - try { - desiredSizeBytes = source.getEstimatedSizeBytes(options) / numPartitions; - List> splits = source.split(desiredSizeBytes, options); - for (BoundedSource split : splits) { - result.add( - (InputPartition) - () -> new DatasetPartitionReader<>(split, serializablePipelineOptions)); - } - return result; - - } catch (Exception e) { - throw new RuntimeException( - "Error in splitting BoundedSource " + source.getClass().getCanonicalName(), e); - } - } - } - - /** This class can be mapped to Beam {@link BoundedReader}. */ - private static class DatasetPartitionReader implements InputPartitionReader { - private boolean started; - private boolean closed; - private final BoundedSource source; - private BoundedReader reader; - - DatasetPartitionReader( - BoundedSource source, SerializablePipelineOptions serializablePipelineOptions) { - this.started = false; - this.closed = false; - this.source = source; - // reader is not serializable so lazy initialize it - try { - reader = source.createReader(serializablePipelineOptions.get().as(PipelineOptions.class)); - } catch (IOException e) { - throw new RuntimeException("Error creating BoundedReader ", e); - } - } - - @Override - public boolean next() throws IOException { - if (!started) { - started = true; - return reader.start(); - } else { - return !closed && reader.advance(); - } - } - - @Override - public InternalRow get() { - WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow( - reader.getCurrent(), reader.getCurrentTimestamp()); - return RowHelpers.storeWindowedValueInRow(windowedValue, source.getOutputCoder()); - } - - @Override - public void close() throws IOException { - closed = true; - reader.close(); - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnFunction.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnFunction.java deleted file mode 100644 index 42a809fdd970..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnFunction.java +++ /dev/null @@ -1,164 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -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.spark.structuredstreaming.metrics.MetricsContainerStepMapAccumulator; -import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.NoOpStepContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.SparkSideInputReader; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.SideInputBroadcast; -import org.apache.beam.runners.spark.structuredstreaming.translation.utils.CachedSideInputReader; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnSchemaInformation; -import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.LinkedListMultimap; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap; -import org.apache.spark.api.java.function.MapPartitionsFunction; -import scala.Tuple2; - -/** - * Encapsulates a {@link DoFn} inside a Spark {@link - * org.apache.spark.api.java.function.MapPartitionsFunction}. - * - *

We get a mapping from {@link org.apache.beam.sdk.values.TupleTag} to output index and must tag - * all outputs with the output number. Afterwards a filter will filter out those elements that are - * not to be in a specific output. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class DoFnFunction - implements MapPartitionsFunction, Tuple2, WindowedValue>> { - - private final MetricsContainerStepMapAccumulator metricsAccum; - private final String stepName; - private final DoFn doFn; - private transient boolean wasSetupCalled; - private final WindowingStrategy windowingStrategy; - private final Map, WindowingStrategy> sideInputs; - private final SerializablePipelineOptions serializableOptions; - private final List> additionalOutputTags; - private final TupleTag mainOutputTag; - private final Coder inputCoder; - private final Map, Coder> outputCoderMap; - private final SideInputBroadcast broadcastStateData; - private DoFnSchemaInformation doFnSchemaInformation; - private Map> sideInputMapping; - - public DoFnFunction( - MetricsContainerStepMapAccumulator metricsAccum, - String stepName, - DoFn doFn, - WindowingStrategy windowingStrategy, - Map, WindowingStrategy> sideInputs, - SerializablePipelineOptions serializableOptions, - List> additionalOutputTags, - TupleTag mainOutputTag, - Coder inputCoder, - Map, Coder> outputCoderMap, - SideInputBroadcast broadcastStateData, - DoFnSchemaInformation doFnSchemaInformation, - Map> sideInputMapping) { - this.metricsAccum = metricsAccum; - this.stepName = stepName; - this.doFn = doFn; - this.windowingStrategy = windowingStrategy; - this.sideInputs = sideInputs; - this.serializableOptions = serializableOptions; - this.additionalOutputTags = additionalOutputTags; - this.mainOutputTag = mainOutputTag; - this.inputCoder = inputCoder; - this.outputCoderMap = outputCoderMap; - this.broadcastStateData = broadcastStateData; - this.doFnSchemaInformation = doFnSchemaInformation; - this.sideInputMapping = sideInputMapping; - } - - @Override - public Iterator, WindowedValue>> call(Iterator> iter) - throws Exception { - if (!wasSetupCalled && iter.hasNext()) { - DoFnInvokers.tryInvokeSetupFor(doFn, serializableOptions.get()); - wasSetupCalled = true; - } - - DoFnOutputManager outputManager = new DoFnOutputManager(); - - DoFnRunner doFnRunner = - DoFnRunners.simpleRunner( - serializableOptions.get(), - doFn, - CachedSideInputReader.of(new SparkSideInputReader(sideInputs, broadcastStateData)), - outputManager, - mainOutputTag, - additionalOutputTags, - new NoOpStepContext(), - inputCoder, - outputCoderMap, - windowingStrategy, - doFnSchemaInformation, - sideInputMapping); - - DoFnRunnerWithMetrics doFnRunnerWithMetrics = - new DoFnRunnerWithMetrics<>(stepName, doFnRunner, metricsAccum); - - return new ProcessContext<>( - doFn, doFnRunnerWithMetrics, outputManager, Collections.emptyIterator()) - .processPartition(iter) - .iterator(); - } - - private class DoFnOutputManager - implements ProcessContext.ProcessOutputManager, WindowedValue>> { - - private final Multimap, WindowedValue> outputs = LinkedListMultimap.create(); - - @Override - public void clear() { - outputs.clear(); - } - - @Override - public Iterator, WindowedValue>> iterator() { - Iterator, WindowedValue>> entryIter = outputs.entries().iterator(); - return Iterators.transform(entryIter, this.entryToTupleFn()); - } - - private Function, Tuple2> entryToTupleFn() { - return en -> new Tuple2<>(en.getKey(), en.getValue()); - } - - @Override - public synchronized void output(TupleTag tag, WindowedValue output) { - outputs.put(tag, output); - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java deleted file mode 100644 index b80ec87d3c56..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java +++ /dev/null @@ -1,104 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.Closeable; -import java.io.IOException; -import org.apache.beam.runners.core.DoFnRunner; -import org.apache.beam.runners.core.metrics.MetricsContainerImpl; -import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsContainerStepMapAccumulator; -import org.apache.beam.sdk.metrics.MetricsContainer; -import org.apache.beam.sdk.metrics.MetricsEnvironment; -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.joda.time.Instant; - -/** DoFnRunner decorator which registers {@link MetricsContainerImpl}. */ -class DoFnRunnerWithMetrics implements DoFnRunner { - private final DoFnRunner delegate; - private final String stepName; - private final MetricsContainerStepMapAccumulator metricsAccum; - - DoFnRunnerWithMetrics( - String stepName, - DoFnRunner delegate, - MetricsContainerStepMapAccumulator metricsAccum) { - this.delegate = delegate; - this.stepName = stepName; - this.metricsAccum = metricsAccum; - } - - @Override - public DoFn getFn() { - return delegate.getFn(); - } - - @Override - public void startBundle() { - try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer(metricsContainer())) { - delegate.startBundle(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void processElement(final WindowedValue elem) { - try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer(metricsContainer())) { - delegate.processElement(elem); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void onTimer( - final String timerId, - final String timerFamilyId, - KeyT key, - final BoundedWindow window, - final Instant timestamp, - final Instant outputTimestamp, - final TimeDomain timeDomain) { - try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer(metricsContainer())) { - delegate.onTimer(timerId, timerFamilyId, key, window, timestamp, outputTimestamp, timeDomain); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void finishBundle() { - try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer(metricsContainer())) { - delegate.finishBundle(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { - delegate.onWindowExpiration(window, timestamp, key); - } - - private MetricsContainer metricsContainer() { - return metricsAccum.value().getContainer(stepName); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java deleted file mode 100644 index db361f7753e1..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java +++ /dev/null @@ -1,65 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; - -import java.util.Collection; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.PValue; -import org.apache.spark.sql.Dataset; - -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class FlattenTranslatorBatch - implements TransformTranslator, PCollection>> { - - @Override - public void translateTransform( - PTransform, PCollection> transform, - AbstractTranslationContext context) { - Collection> pcollectionList = context.getInputs().values(); - Dataset> result = null; - if (pcollectionList.isEmpty()) { - result = context.emptyDataset(); - } else { - for (PValue pValue : pcollectionList) { - checkArgument( - pValue instanceof PCollection, - "Got non-PCollection input to flatten: %s of type %s", - pValue, - pValue.getClass().getSimpleName()); - @SuppressWarnings("unchecked") - PCollection pCollection = (PCollection) pValue; - Dataset> current = context.getDataset(pCollection); - if (result == null) { - result = current; - } else { - result = result.union(current); - } - } - } - context.putDataset(context.getOutput(), result); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java deleted file mode 100644 index 6391ba4600cf..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java +++ /dev/null @@ -1,90 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.Serializable; -import org.apache.beam.runners.core.InMemoryStateInternals; -import org.apache.beam.runners.core.StateInternals; -import org.apache.beam.runners.core.StateInternalsFactory; -import org.apache.beam.runners.core.SystemReduceFn; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.GroupAlsoByWindowViaOutputBufferFn; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.KVHelpers; -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.PTransform; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.KeyValueGroupedDataset; - -class GroupByKeyTranslatorBatch - implements TransformTranslator< - PTransform>, PCollection>>>> { - - @Override - public void translateTransform( - PTransform>, PCollection>>> transform, - AbstractTranslationContext context) { - - @SuppressWarnings("unchecked") - final PCollection> inputPCollection = (PCollection>) context.getInput(); - Dataset>> input = context.getDataset(inputPCollection); - WindowingStrategy windowingStrategy = inputPCollection.getWindowingStrategy(); - KvCoder kvCoder = (KvCoder) inputPCollection.getCoder(); - Coder valueCoder = kvCoder.getValueCoder(); - - // group by key only - Coder keyCoder = kvCoder.getKeyCoder(); - KeyValueGroupedDataset>> groupByKeyOnly = - input.groupByKey(KVHelpers.extractKey(), EncoderHelpers.fromBeamCoder(keyCoder)); - - // group also by windows - WindowedValue.FullWindowedValueCoder>> outputCoder = - WindowedValue.FullWindowedValueCoder.of( - KvCoder.of(keyCoder, IterableCoder.of(valueCoder)), - windowingStrategy.getWindowFn().windowCoder()); - Dataset>>> output = - groupByKeyOnly.flatMapGroups( - new GroupAlsoByWindowViaOutputBufferFn<>( - windowingStrategy, - new InMemoryStateInternalsFactory<>(), - SystemReduceFn.buffering(valueCoder), - context.getSerializableOptions()), - EncoderHelpers.fromBeamCoder(outputCoder)); - - context.putDataset(context.getOutput(), output); - } - - /** - * In-memory state internals factory. - * - * @param State key type. - */ - static class InMemoryStateInternalsFactory implements StateInternalsFactory, Serializable { - @Override - public StateInternals stateInternalsForKey(K key) { - return InMemoryStateInternals.forKey(key); - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java deleted file mode 100644 index 65f496c772ba..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java +++ /dev/null @@ -1,49 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.util.Collections; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.spark.sql.Dataset; - -public class ImpulseTranslatorBatch - implements TransformTranslator>> { - - @Override - public void translateTransform( - PTransform> transform, AbstractTranslationContext context) { - Coder> windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE); - Dataset> dataset = - context - .getSparkSession() - .createDataset( - Collections.singletonList(WindowedValue.valueInGlobalWindow(new byte[0])), - EncoderHelpers.fromBeamCoder(windowedValueCoder)); - context.putDataset(context.getOutput(), dataset); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java deleted file mode 100644 index 52c2d5ae6420..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java +++ /dev/null @@ -1,257 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsAccumulator; -import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsContainerStepMapAccumulator; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.CoderHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.MultiOutputCoder; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.SideInputBroadcast; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnSchemaInformation; -import org.apache.beam.sdk.transforms.PTransform; -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.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FilterFunction; -import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.sql.Dataset; -import scala.Tuple2; - -/** - * TODO: Add support for state and timers. - * - * @param - * @param - */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class ParDoTranslatorBatch - implements TransformTranslator, PCollectionTuple>> { - - @Override - public void translateTransform( - PTransform, PCollectionTuple> transform, - AbstractTranslationContext context) { - String stepName = context.getCurrentTransform().getFullName(); - - // Check for not supported advanced features - // TODO: add support of Splittable DoFn - DoFn doFn = getDoFn(context); - checkState( - !DoFnSignatures.isSplittable(doFn), - "Not expected to directly translate splittable DoFn, should have been overridden: %s", - doFn); - - // TODO: add support of states and timers - checkState( - !DoFnSignatures.isStateful(doFn), "States and timers are not supported for the moment."); - - checkState( - !DoFnSignatures.requiresTimeSortedInput(doFn), - "@RequiresTimeSortedInput is not " + "supported for the moment"); - - DoFnSchemaInformation doFnSchemaInformation = - ParDoTranslation.getSchemaInformation(context.getCurrentTransform()); - - // Init main variables - PValue input = context.getInput(); - Dataset> inputDataSet = context.getDataset(input); - Map, PCollection> outputs = context.getOutputs(); - TupleTag mainOutputTag = getTupleTag(context); - List> outputTags = new ArrayList<>(outputs.keySet()); - WindowingStrategy windowingStrategy = - ((PCollection) input).getWindowingStrategy(); - Coder inputCoder = ((PCollection) input).getCoder(); - Coder windowCoder = windowingStrategy.getWindowFn().windowCoder(); - - // construct a map from side input to WindowingStrategy so that - // the DoFn runner can map main-input windows to side input windows - List> sideInputs = getSideInputs(context); - Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); - for (PCollectionView sideInput : sideInputs) { - sideInputStrategies.put(sideInput, sideInput.getPCollection().getWindowingStrategy()); - } - - SideInputBroadcast broadcastStateData = createBroadcastSideInputs(sideInputs, context); - - Map, Coder> outputCoderMap = context.getOutputCoders(); - MetricsContainerStepMapAccumulator metricsAccum = MetricsAccumulator.getInstance(); - - List> additionalOutputTags = new ArrayList<>(); - for (TupleTag tag : outputTags) { - if (!tag.equals(mainOutputTag)) { - additionalOutputTags.add(tag); - } - } - - Map> sideInputMapping = - ParDoTranslation.getSideInputMapping(context.getCurrentTransform()); - @SuppressWarnings("unchecked") - DoFnFunction doFnWrapper = - new DoFnFunction( - metricsAccum, - stepName, - doFn, - windowingStrategy, - sideInputStrategies, - context.getSerializableOptions(), - additionalOutputTags, - mainOutputTag, - inputCoder, - outputCoderMap, - broadcastStateData, - doFnSchemaInformation, - sideInputMapping); - - MultiOutputCoder multipleOutputCoder = - MultiOutputCoder.of(SerializableCoder.of(TupleTag.class), outputCoderMap, windowCoder); - Dataset, WindowedValue>> allOutputs = - inputDataSet.mapPartitions(doFnWrapper, EncoderHelpers.fromBeamCoder(multipleOutputCoder)); - if (outputs.entrySet().size() > 1) { - allOutputs.persist(); - for (Map.Entry, PCollection> output : outputs.entrySet()) { - pruneOutputFilteredByTag(context, allOutputs, output, windowCoder); - } - } else { - Coder outputCoder = ((PCollection) outputs.get(mainOutputTag)).getCoder(); - Coder> windowedValueCoder = - (Coder>) (Coder) WindowedValue.getFullCoder(outputCoder, windowCoder); - Dataset> outputDataset = - allOutputs.map( - (MapFunction, WindowedValue>, WindowedValue>) - value -> value._2, - EncoderHelpers.fromBeamCoder(windowedValueCoder)); - context.putDatasetWildcard(outputs.entrySet().iterator().next().getValue(), outputDataset); - } - } - - private static SideInputBroadcast createBroadcastSideInputs( - List> sideInputs, AbstractTranslationContext context) { - JavaSparkContext jsc = - JavaSparkContext.fromSparkContext(context.getSparkSession().sparkContext()); - - SideInputBroadcast sideInputBroadcast = new SideInputBroadcast(); - for (PCollectionView sideInput : sideInputs) { - Coder windowCoder = - sideInput.getPCollection().getWindowingStrategy().getWindowFn().windowCoder(); - - Coder> windowedValueCoder = - (Coder>) - (Coder) - WindowedValue.getFullCoder(sideInput.getPCollection().getCoder(), windowCoder); - Dataset> broadcastSet = context.getSideInputDataSet(sideInput); - List> valuesList = broadcastSet.collectAsList(); - List codedValues = new ArrayList<>(); - for (WindowedValue v : valuesList) { - codedValues.add(CoderHelpers.toByteArray(v, windowedValueCoder)); - } - - sideInputBroadcast.add( - sideInput.getTagInternal().getId(), jsc.broadcast(codedValues), windowedValueCoder); - } - return sideInputBroadcast; - } - - private List> getSideInputs(AbstractTranslationContext context) { - List> sideInputs; - try { - sideInputs = ParDoTranslation.getSideInputs(context.getCurrentTransform()); - } catch (IOException e) { - throw new RuntimeException(e); - } - return sideInputs; - } - - private TupleTag getTupleTag(AbstractTranslationContext context) { - TupleTag mainOutputTag; - try { - mainOutputTag = ParDoTranslation.getMainOutputTag(context.getCurrentTransform()); - } catch (IOException e) { - throw new RuntimeException(e); - } - return mainOutputTag; - } - - @SuppressWarnings("unchecked") - private DoFn getDoFn(AbstractTranslationContext context) { - DoFn doFn; - try { - doFn = (DoFn) ParDoTranslation.getDoFn(context.getCurrentTransform()); - } catch (IOException e) { - throw new RuntimeException(e); - } - return doFn; - } - - private void pruneOutputFilteredByTag( - AbstractTranslationContext context, - Dataset, WindowedValue>> allOutputs, - Map.Entry, PCollection> output, - Coder windowCoder) { - Dataset, WindowedValue>> filteredDataset = - allOutputs.filter(new DoFnFilterFunction(output.getKey())); - Coder> windowedValueCoder = - (Coder>) - (Coder) - WindowedValue.getFullCoder( - ((PCollection) output.getValue()).getCoder(), windowCoder); - Dataset> outputDataset = - filteredDataset.map( - (MapFunction, WindowedValue>, WindowedValue>) - value -> value._2, - EncoderHelpers.fromBeamCoder(windowedValueCoder)); - context.putDatasetWildcard(output.getValue(), outputDataset); - } - - static class DoFnFilterFunction implements FilterFunction, WindowedValue>> { - - private final TupleTag key; - - DoFnFilterFunction(TupleTag key) { - this.key = key; - } - - @Override - public boolean call(Tuple2, WindowedValue> value) { - return value._1.equals(key); - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java deleted file mode 100644 index 5789db6cd304..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java +++ /dev/null @@ -1,101 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.translation.PipelineTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.TranslationContext; -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.Flatten; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.Impulse; -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.windowing.Window; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * {@link PipelineTranslator} for executing a {@link Pipeline} in Spark in batch mode. This contains - * only the components specific to batch: registry of batch {@link TransformTranslator} and registry - * lookup code. - */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class PipelineTranslatorBatch extends PipelineTranslator { - - // -------------------------------------------------------------------------------------------- - // Transform Translator Registry - // -------------------------------------------------------------------------------------------- - - @SuppressWarnings("rawtypes") - private static final Map, TransformTranslator> TRANSFORM_TRANSLATORS = - new HashMap<>(); - - // TODO the ability to have more than one TransformTranslator per URN - // that could be dynamically chosen by a predicated that evaluates based on PCollection - // obtainable though node.getInputs.getValue() - // See - // https://github.com/seznam/euphoria/blob/master/euphoria-spark/src/main/java/cz/seznam/euphoria/spark/SparkFlowTranslator.java#L83 - // And - // https://github.com/seznam/euphoria/blob/master/euphoria-spark/src/main/java/cz/seznam/euphoria/spark/SparkFlowTranslator.java#L106 - - static { - TRANSFORM_TRANSLATORS.put(Impulse.class, new ImpulseTranslatorBatch()); - TRANSFORM_TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch()); - TRANSFORM_TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); - - // TODO: Do we need to have a dedicated translator for {@code Reshuffle} if it's deprecated? - // TRANSFORM_TRANSLATORS.put(Reshuffle.class, new ReshuffleTranslatorBatch()); - - TRANSFORM_TRANSLATORS.put(Flatten.PCollections.class, new FlattenTranslatorBatch()); - - TRANSFORM_TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslatorBatch()); - - TRANSFORM_TRANSLATORS.put(ParDo.MultiOutput.class, new ParDoTranslatorBatch()); - - TRANSFORM_TRANSLATORS.put( - SplittableParDo.PrimitiveBoundedRead.class, new ReadSourceTranslatorBatch()); - - TRANSFORM_TRANSLATORS.put( - View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch()); - } - - public PipelineTranslatorBatch(SparkStructuredStreamingPipelineOptions options) { - translationContext = new TranslationContext(options); - } - - /** Returns a translator for the given node, if it is possible, otherwise null. */ - @Override - protected TransformTranslator getTransformTranslator(TransformHierarchy.Node node) { - @Nullable PTransform transform = node.getTransform(); - // Root of the graph is null - if (transform == null) { - return null; - } - return TRANSFORM_TRANSLATORS.get(transform.getClass()); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ProcessContext.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ProcessContext.java deleted file mode 100644 index db64bfd19f39..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ProcessContext.java +++ /dev/null @@ -1,138 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.util.ArrayList; -import java.util.Iterator; -import org.apache.beam.runners.core.DoFnRunner; -import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator; - -/** Spark runner process context processes Spark partitions using Beam's {@link DoFnRunner}. */ -class ProcessContext { - - private final DoFn doFn; - private final DoFnRunner doFnRunner; - private final ProcessOutputManager outputManager; - private final Iterator timerDataIterator; - - ProcessContext( - DoFn doFn, - DoFnRunner doFnRunner, - ProcessOutputManager outputManager, - Iterator timerDataIterator) { - - this.doFn = doFn; - this.doFnRunner = doFnRunner; - this.outputManager = outputManager; - this.timerDataIterator = timerDataIterator; - } - - Iterable processPartition(Iterator> partition) { - - // skip if partition is empty. - if (!partition.hasNext()) { - return new ArrayList<>(); - } - - // process the partition; finishBundle() is called from within the output iterator. - return this.getOutputIterable(partition, doFnRunner); - } - - private void clearOutput() { - outputManager.clear(); - } - - private Iterator getOutputIterator() { - return outputManager.iterator(); - } - - private Iterable getOutputIterable( - final Iterator> iter, - final DoFnRunner doFnRunner) { - return () -> new ProcCtxtIterator(iter, doFnRunner); - } - - interface ProcessOutputManager extends OutputManager, Iterable { - void clear(); - } - - private class ProcCtxtIterator extends AbstractIterator { - - private final Iterator> inputIterator; - private final DoFnRunner doFnRunner; - private Iterator outputIterator; - private boolean isBundleStarted; - private boolean isBundleFinished; - - ProcCtxtIterator( - Iterator> iterator, DoFnRunner doFnRunner) { - this.inputIterator = iterator; - this.doFnRunner = doFnRunner; - this.outputIterator = getOutputIterator(); - } - - @Override - protected OutputT computeNext() { - try { - // Process each element from the (input) iterator, which produces, zero, one or more - // output elements (of type V) in the output iterator. Note that the output - // collection (and iterator) is reset between each call to processElement, so the - // collection only holds the output values for each call to processElement, rather - // than for the whole partition (which would use too much memory). - if (!isBundleStarted) { - isBundleStarted = true; - // call startBundle() before beginning to process the partition. - doFnRunner.startBundle(); - } - - while (true) { - if (outputIterator.hasNext()) { - return outputIterator.next(); - } - - clearOutput(); - if (inputIterator.hasNext()) { - // grab the next element and process it. - doFnRunner.processElement(inputIterator.next()); - outputIterator = getOutputIterator(); - } else if (timerDataIterator.hasNext()) { - outputIterator = getOutputIterator(); - } else { - // no more input to consume, but finishBundle can produce more output - if (!isBundleFinished) { - isBundleFinished = true; - doFnRunner.finishBundle(); - outputIterator = getOutputIterator(); - continue; // try to consume outputIterator from start of loop - } - DoFnInvokers.invokerFor(doFn).invokeTeardown(); - return endOfData(); - } - } - } catch (final RuntimeException re) { - DoFnInvokers.invokerFor(doFn).invokeTeardown(); - throw re; - } - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java deleted file mode 100644 index ebeb8a96eda4..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java +++ /dev/null @@ -1,88 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import static org.apache.beam.runners.spark.structuredstreaming.Constants.BEAM_SOURCE_OPTION; -import static org.apache.beam.runners.spark.structuredstreaming.Constants.DEFAULT_PARALLELISM; -import static org.apache.beam.runners.spark.structuredstreaming.Constants.PIPELINE_OPTIONS; - -import java.io.IOException; -import org.apache.beam.runners.core.construction.ReadTranslation; -import org.apache.beam.runners.core.serialization.Base64Serializer; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.RowHelpers; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - -class ReadSourceTranslatorBatch - implements TransformTranslator>> { - - private static final String sourceProviderClass = DatasetSourceBatch.class.getCanonicalName(); - - @SuppressWarnings("unchecked") - @Override - public void translateTransform( - PTransform> transform, AbstractTranslationContext context) { - AppliedPTransform, PTransform>> rootTransform = - (AppliedPTransform, PTransform>>) - context.getCurrentTransform(); - - BoundedSource source; - try { - source = ReadTranslation.boundedSourceFromTransform(rootTransform); - } catch (IOException e) { - throw new RuntimeException(e); - } - SparkSession sparkSession = context.getSparkSession(); - - String serializedSource = Base64Serializer.serializeUnchecked(source); - Dataset rowDataset = - sparkSession - .read() - .format(sourceProviderClass) - .option(BEAM_SOURCE_OPTION, serializedSource) - .option( - DEFAULT_PARALLELISM, - String.valueOf(context.getSparkSession().sparkContext().defaultParallelism())) - .option(PIPELINE_OPTIONS, context.getSerializableOptions().toString()) - .load(); - - // extract windowedValue from Row - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of( - source.getOutputCoder(), GlobalWindow.Coder.INSTANCE); - - Dataset> dataset = - rowDataset.map( - RowHelpers.extractWindowedValueFromRowMapFunction(windowedValueCoder), - EncoderHelpers.fromBeamCoder(windowedValueCoder)); - - PCollection output = (PCollection) context.getOutput(); - context.putDataset(output, dataset); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java deleted file mode 100644 index a88d5454667f..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java +++ /dev/null @@ -1,30 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.sdk.transforms.Reshuffle; - -/** TODO: Should be removed if {@link Reshuffle} won't be translated. */ -class ReshuffleTranslatorBatch implements TransformTranslator> { - - @Override - public void translateTransform( - Reshuffle transform, AbstractTranslationContext context) {} -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java deleted file mode 100644 index 875a983b4017..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java +++ /dev/null @@ -1,61 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.WindowingHelpers; -import org.apache.beam.sdk.transforms.PTransform; -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.values.PCollection; -import org.apache.spark.sql.Dataset; - -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class WindowAssignTranslatorBatch - implements TransformTranslator, PCollection>> { - - @Override - public void translateTransform( - PTransform, PCollection> transform, AbstractTranslationContext context) { - - Window.Assign assignTransform = (Window.Assign) transform; - @SuppressWarnings("unchecked") - final PCollection input = (PCollection) context.getInput(); - @SuppressWarnings("unchecked") - final PCollection output = (PCollection) context.getOutput(); - - Dataset> inputDataset = context.getDataset(input); - if (WindowingHelpers.skipAssignWindows(assignTransform, context)) { - context.putDataset(output, inputDataset); - } else { - WindowFn windowFn = assignTransform.getWindowFn(); - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(input.getCoder(), windowFn.windowCoder()); - Dataset> outputDataset = - inputDataset.map( - WindowingHelpers.assignWindowsMapFunction(windowFn), - EncoderHelpers.fromBeamCoder(windowedValueCoder)); - context.putDataset(output, outputDataset); - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java deleted file mode 100644 index 77ec5c1269a3..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java +++ /dev/null @@ -1,173 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import org.apache.beam.runners.core.InMemoryTimerInternals; -import org.apache.beam.runners.core.OutputWindowedValue; -import org.apache.beam.runners.core.ReduceFnRunner; -import org.apache.beam.runners.core.StateInternals; -import org.apache.beam.runners.core.StateInternalsFactory; -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.values.KV; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.spark.api.java.function.FlatMapGroupsFunction; -import org.joda.time.Instant; - -/** A FlatMap function that groups by windows in batch mode using {@link ReduceFnRunner}. */ -public class GroupAlsoByWindowViaOutputBufferFn - implements FlatMapGroupsFunction< - K, WindowedValue>, WindowedValue>>> { - - private final WindowingStrategy windowingStrategy; - private final StateInternalsFactory stateInternalsFactory; - private final SystemReduceFn, Iterable, W> reduceFn; - private final SerializablePipelineOptions options; - - public GroupAlsoByWindowViaOutputBufferFn( - WindowingStrategy windowingStrategy, - StateInternalsFactory stateInternalsFactory, - SystemReduceFn, Iterable, W> reduceFn, - SerializablePipelineOptions options) { - this.windowingStrategy = windowingStrategy; - this.stateInternalsFactory = stateInternalsFactory; - this.reduceFn = reduceFn; - this.options = options; - } - - @Override - public Iterator>>> call( - K key, Iterator>> iterator) throws Exception { - - // we have to materialize the Iterator because ReduceFnRunner.processElements expects - // to have all elements to merge the windows between each other. - // possible OOM even though the spark framework spills to disk if a given group is too large to - // fit in memory. - ArrayList> values = new ArrayList<>(); - while (iterator.hasNext()) { - WindowedValue> wv = iterator.next(); - values.add(wv.withValue(wv.getValue().getValue())); - } - - // ------ based on GroupAlsoByWindowsViaOutputBufferDoFn ------// - - // Used with Batch, we know that all the data is available for this key. We can't use the - // timer manager from the context because it doesn't exist. So we create one and emulate the - // watermark, knowing that we have all data and it is in timestamp order. - InMemoryTimerInternals timerInternals = new InMemoryTimerInternals(); - timerInternals.advanceProcessingTime(Instant.now()); - timerInternals.advanceSynchronizedProcessingTime(Instant.now()); - StateInternals stateInternals = stateInternalsFactory.stateInternalsForKey(key); - GABWOutputWindowedValue outputter = new GABWOutputWindowedValue<>(); - - ReduceFnRunner, W> reduceFnRunner = - new ReduceFnRunner<>( - key, - windowingStrategy, - ExecutableTriggerStateMachine.create( - TriggerStateMachines.stateMachineForTrigger( - TriggerTranslation.toProto(windowingStrategy.getTrigger()))), - stateInternals, - timerInternals, - outputter, - new UnsupportedSideInputReader("GroupAlsoByWindow"), - reduceFn, - options.get()); - - // Process the grouped values. - reduceFnRunner.processElements(values); - - // Finish any pending windows by advancing the input watermark to infinity. - timerInternals.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE); - - // Finally, advance the processing time to infinity to fire any timers. - timerInternals.advanceProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE); - timerInternals.advanceSynchronizedProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE); - - fireEligibleTimers(timerInternals, reduceFnRunner); - - reduceFnRunner.persist(); - - return outputter.getOutputs().iterator(); - } - - private void fireEligibleTimers( - InMemoryTimerInternals timerInternals, - ReduceFnRunner, W> reduceFnRunner) - throws Exception { - List timers = new ArrayList<>(); - while (true) { - TimerInternals.TimerData timer; - while ((timer = timerInternals.removeNextEventTimer()) != null) { - timers.add(timer); - } - while ((timer = timerInternals.removeNextProcessingTimer()) != null) { - timers.add(timer); - } - while ((timer = timerInternals.removeNextSynchronizedProcessingTimer()) != null) { - timers.add(timer); - } - if (timers.isEmpty()) { - break; - } - reduceFnRunner.onTimers(timers); - timers.clear(); - } - } - - private static class GABWOutputWindowedValue - implements OutputWindowedValue>> { - private final List>>> outputs = new ArrayList<>(); - - @Override - public void outputWindowedValue( - KV> output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - outputs.add(WindowedValue.of(output, timestamp, windows, pane)); - } - - @Override - public void outputWindowedValue( - TupleTag tag, - AdditionalOutputT output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - throw new UnsupportedOperationException("GroupAlsoByWindow should not use tagged outputs."); - } - - Iterable>>> getOutputs() { - return outputs; - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/NoOpStepContext.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/NoOpStepContext.java deleted file mode 100644 index 25e6f112a3f3..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/NoOpStepContext.java +++ /dev/null @@ -1,36 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions; - -import org.apache.beam.runners.core.StateInternals; -import org.apache.beam.runners.core.StepContext; -import org.apache.beam.runners.core.TimerInternals; - -/** A {@link StepContext} for Spark Batch Runner execution. */ -public class NoOpStepContext implements StepContext { - - @Override - public StateInternals stateInternals() { - throw new UnsupportedOperationException("stateInternals is not supported"); - } - - @Override - public TimerInternals timerInternals() { - throw new UnsupportedOperationException("timerInternals is not supported"); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java deleted file mode 100644 index a9897d710910..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java +++ /dev/null @@ -1,186 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions; - -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.beam.runners.core.InMemoryMultimapSideInputView; -import org.apache.beam.runners.core.SideInputReader; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.CoderHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.SideInputBroadcast; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.transforms.Materializations; -import org.apache.beam.sdk.transforms.Materializations.IterableView; -import org.apache.beam.sdk.transforms.Materializations.MultimapView; -import org.apache.beam.sdk.transforms.ViewFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** A {@link SideInputReader} for the Spark Batch Runner. */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class SparkSideInputReader implements SideInputReader { - private static final Set SUPPORTED_MATERIALIZATIONS = - ImmutableSet.of( - Materializations.ITERABLE_MATERIALIZATION_URN, - Materializations.MULTIMAP_MATERIALIZATION_URN); - - private final Map, WindowingStrategy> sideInputs; - private final SideInputBroadcast broadcastStateData; - - public SparkSideInputReader( - Map, WindowingStrategy> indexByView, - SideInputBroadcast broadcastStateData) { - for (PCollectionView view : indexByView.keySet()) { - checkArgument( - SUPPORTED_MATERIALIZATIONS.contains(view.getViewFn().getMaterialization().getUrn()), - "This handler is only capable of dealing with %s materializations " - + "but was asked to handle %s for PCollectionView with tag %s.", - SUPPORTED_MATERIALIZATIONS, - view.getViewFn().getMaterialization().getUrn(), - view.getTagInternal().getId()); - } - sideInputs = new HashMap<>(); - for (Map.Entry, WindowingStrategy> entry : indexByView.entrySet()) { - sideInputs.put(entry.getKey().getTagInternal(), entry.getValue()); - } - this.broadcastStateData = broadcastStateData; - } - - @Override - public @Nullable T get(PCollectionView view, BoundedWindow window) { - checkNotNull(view, "View passed to sideInput cannot be null"); - TupleTag tag = view.getTagInternal(); - checkNotNull(sideInputs.get(tag), "Side input for " + view + " not available."); - - List sideInputsValues = - (List) broadcastStateData.getBroadcastValue(tag.getId()).getValue(); - Coder coder = broadcastStateData.getCoder(tag.getId()); - - List> decodedValues = new ArrayList<>(); - for (byte[] value : sideInputsValues) { - decodedValues.add((WindowedValue) CoderHelpers.fromByteArray(value, coder)); - } - - Map sideInputs = initializeBroadcastVariable(decodedValues, view); - T result = sideInputs.get(window); - if (result == null) { - switch (view.getViewFn().getMaterialization().getUrn()) { - case Materializations.ITERABLE_MATERIALIZATION_URN: - { - ViewFn viewFn = (ViewFn) view.getViewFn(); - return viewFn.apply(() -> Collections.emptyList()); - } - case Materializations.MULTIMAP_MATERIALIZATION_URN: - { - ViewFn viewFn = (ViewFn) view.getViewFn(); - return viewFn.apply(InMemoryMultimapSideInputView.empty()); - } - default: - throw new IllegalStateException( - String.format( - "Unknown side input materialization format requested '%s'", - view.getViewFn().getMaterialization().getUrn())); - } - } - return result; - } - - @Override - public boolean contains(PCollectionView view) { - return sideInputs.containsKey(view.getTagInternal()); - } - - @Override - public boolean isEmpty() { - return sideInputs.isEmpty(); - } - - private Map initializeBroadcastVariable( - Iterable> inputValues, PCollectionView view) { - - // first partition into windows - Map>> partitionedElements = new HashMap<>(); - for (WindowedValue value : inputValues) { - for (BoundedWindow window : value.getWindows()) { - List> windowedValues = - partitionedElements.computeIfAbsent(window, k -> new ArrayList<>()); - windowedValues.add(value); - } - } - - Map resultMap = new HashMap<>(); - - for (Map.Entry>> elements : - partitionedElements.entrySet()) { - - switch (view.getViewFn().getMaterialization().getUrn()) { - case Materializations.ITERABLE_MATERIALIZATION_URN: - { - ViewFn viewFn = (ViewFn) view.getViewFn(); - resultMap.put( - elements.getKey(), - viewFn.apply( - () -> - elements.getValue().stream() - .map(WindowedValue::getValue) - .collect(Collectors.toList()))); - } - break; - case Materializations.MULTIMAP_MATERIALIZATION_URN: - { - ViewFn viewFn = (ViewFn) view.getViewFn(); - Coder keyCoder = ((KvCoder) view.getCoderInternal()).getKeyCoder(); - resultMap.put( - elements.getKey(), - viewFn.apply( - InMemoryMultimapSideInputView.fromIterable( - keyCoder, - (Iterable) - elements.getValue().stream() - .map(WindowedValue::getValue) - .collect(Collectors.toList())))); - } - break; - default: - throw new IllegalStateException( - String.format( - "Unknown side input materialization format requested '%s'", - view.getViewFn().getMaterialization().getUrn())); - } - } - - return resultMap; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/package-info.java deleted file mode 100644 index 1f03bac21240..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Internal implementation of the Beam runner for Apache Spark. */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/package-info.java deleted file mode 100644 index 6d3ce5aa723f..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Internal utilities to translate Beam pipelines to Spark batching. */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/CoderHelpers.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/CoderHelpers.java deleted file mode 100644 index fe3f39ef51e9..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/CoderHelpers.java +++ /dev/null @@ -1,63 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import org.apache.beam.sdk.coders.Coder; - -/** Serialization utility class. */ -public final class CoderHelpers { - private CoderHelpers() {} - - /** - * Utility method for serializing an object using the specified coder. - * - * @param value Value to serialize. - * @param coder Coder to serialize with. - * @param type of value that is serialized - * @return Byte array representing serialized object. - */ - public static byte[] toByteArray(T value, Coder coder) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try { - coder.encode(value, baos); - } catch (IOException e) { - throw new IllegalStateException("Error encoding value: " + value, e); - } - return baos.toByteArray(); - } - - /** - * Utility method for deserializing a byte array using the specified coder. - * - * @param serialized bytearray to be deserialized. - * @param coder Coder to deserialize with. - * @param Type of object to be returned. - * @return Deserialized object. - */ - public static T fromByteArray(byte[] serialized, Coder coder) { - ByteArrayInputStream bais = new ByteArrayInputStream(serialized); - try { - return coder.decode(bais); - } catch (IOException e) { - throw new IllegalStateException("Error decoding bytes for coder: " + coder, e); - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java deleted file mode 100644 index 2b86ec839c9e..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java +++ /dev/null @@ -1,49 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.listOf; -import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.seqOf; - -import org.apache.spark.sql.Encoder; -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; -import org.apache.spark.sql.catalyst.expressions.Expression; -import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke; -import org.apache.spark.sql.types.DataType; -import scala.reflect.ClassTag$; - -public class EncoderFactory { - - static Encoder create( - Expression serializer, Expression deserializer, Class clazz) { - return new ExpressionEncoder<>( - SchemaHelpers.binarySchema(), - false, - listOf(serializer), - deserializer, - ClassTag$.MODULE$.apply(clazz)); - } - - /** - * Invoke method {@code fun} on Class {@code cls}, immediately propagating {@code null} if any - * input arg is {@code null}. - */ - static Expression invokeIfNotNull(Class cls, String fun, DataType type, Expression... args) { - return new StaticInvoke(cls, type, fun, seqOf(args), true, true); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java deleted file mode 100644 index 68738cf03080..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java +++ /dev/null @@ -1,71 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import static org.apache.spark.sql.types.DataTypes.BinaryType; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.spark.sql.Encoder; -import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal; -import org.apache.spark.sql.catalyst.expressions.BoundReference; -import org.apache.spark.sql.catalyst.expressions.Expression; -import org.apache.spark.sql.catalyst.expressions.Literal; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.ObjectType; -import org.checkerframework.checker.nullness.qual.NonNull; - -public class EncoderHelpers { - private static final DataType OBJECT_TYPE = new ObjectType(Object.class); - - /** - * Wrap a Beam coder into a Spark Encoder using Catalyst Expression Encoders (which uses java code - * generation). - */ - public static Encoder fromBeamCoder(Coder coder) { - Class clazz = coder.getEncodedTypeDescriptor().getRawType(); - // Class T could be private, therefore use OBJECT_TYPE to not risk an IllegalAccessError - return EncoderFactory.create( - beamSerializer(rootRef(OBJECT_TYPE, true), coder), - beamDeserializer(rootCol(BinaryType), coder), - clazz); - } - - /** Catalyst Expression that serializes elements using Beam {@link Coder}. */ - private static Expression beamSerializer(Expression obj, Coder coder) { - Expression[] args = {obj, lit(coder, Coder.class)}; - return EncoderFactory.invokeIfNotNull(CoderHelpers.class, "toByteArray", BinaryType, args); - } - - /** Catalyst Expression that deserializes elements using Beam {@link Coder}. */ - private static Expression beamDeserializer(Expression bytes, Coder coder) { - Expression[] args = {bytes, lit(coder, Coder.class)}; - return EncoderFactory.invokeIfNotNull(CoderHelpers.class, "fromByteArray", OBJECT_TYPE, args); - } - - private static Expression rootRef(DataType dt, boolean nullable) { - return new BoundReference(0, dt, nullable); - } - - private static Expression rootCol(DataType dt) { - return new GetColumnByOrdinal(0, dt); - } - - private static Literal lit(T obj, Class cls) { - return Literal.fromObject(obj, new ObjectType(cls)); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/KVHelpers.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/KVHelpers.java deleted file mode 100644 index 2406c0f49ab5..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/KVHelpers.java +++ /dev/null @@ -1,31 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.KV; -import org.apache.spark.api.java.function.MapFunction; - -/** Helper functions for working with {@link org.apache.beam.sdk.values.KV}. */ -public final class KVHelpers { - - /** A Spark {@link MapFunction} for extracting the key out of a {@link KV} for GBK for example. */ - public static MapFunction>, K> extractKey() { - return wv -> wv.getValue().getKey(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/MultiOutputCoder.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/MultiOutputCoder.java deleted file mode 100644 index f77fcea67960..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/MultiOutputCoder.java +++ /dev/null @@ -1,84 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Map; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; -import scala.Tuple2; - -/** - * Coder to serialize and deserialize {@code}Tuple2, WindowedValue{/@code} to be used - * in spark encoders while applying {@link org.apache.beam.sdk.transforms.DoFn}. - * - * @param type of the elements in the collection - */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class MultiOutputCoder extends CustomCoder, WindowedValue>> { - Coder tupleTagCoder; - Map, Coder> coderMap; - Coder windowCoder; - - public static MultiOutputCoder of( - Coder tupleTagCoder, - Map, Coder> coderMap, - Coder windowCoder) { - return new MultiOutputCoder(tupleTagCoder, coderMap, windowCoder); - } - - private MultiOutputCoder( - Coder tupleTagCoder, - Map, Coder> coderMap, - Coder windowCoder) { - this.tupleTagCoder = tupleTagCoder; - this.coderMap = coderMap; - this.windowCoder = windowCoder; - } - - @Override - public void encode(Tuple2, WindowedValue> tuple2, OutputStream outStream) - throws IOException { - TupleTag tupleTag = tuple2._1(); - tupleTagCoder.encode(tupleTag, outStream); - Coder valueCoder = (Coder) coderMap.get(tupleTag); - WindowedValue.FullWindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(valueCoder, windowCoder); - wvCoder.encode(tuple2._2(), outStream); - } - - @Override - public Tuple2, WindowedValue> decode(InputStream inStream) - throws CoderException, IOException { - TupleTag tupleTag = (TupleTag) tupleTagCoder.decode(inStream); - Coder valueCoder = (Coder) coderMap.get(tupleTag); - WindowedValue.FullWindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(valueCoder, windowCoder); - WindowedValue wv = wvCoder.decode(inStream); - return Tuple2.apply(tupleTag, wv); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/RowHelpers.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/RowHelpers.java deleted file mode 100644 index 9b5d5da2b2cd..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/RowHelpers.java +++ /dev/null @@ -1,75 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import static scala.collection.JavaConversions.asScalaBuffer; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.InternalRow; - -/** Helper functions for working with {@link Row}. */ -public final class RowHelpers { - - /** - * A Spark {@link MapFunction} for extracting a {@link WindowedValue} from a Row in which the - * {@link WindowedValue} was serialized to bytes using its {@link - * WindowedValue.WindowedValueCoder}. - * - * @param The type of the object. - * @return A {@link MapFunction} that accepts a {@link Row} and returns its {@link WindowedValue}. - */ - public static MapFunction> extractWindowedValueFromRowMapFunction( - WindowedValue.WindowedValueCoder windowedValueCoder) { - return (MapFunction>) - value -> { - // there is only one value put in each Row by the InputPartitionReader - byte[] bytes = (byte[]) value.get(0); - return windowedValueCoder.decode(new ByteArrayInputStream(bytes)); - }; - } - - /** - * Serialize a windowedValue to bytes using windowedValueCoder {@link - * WindowedValue.FullWindowedValueCoder} and stores it an InternalRow. - */ - public static InternalRow storeWindowedValueInRow( - WindowedValue windowedValue, Coder coder) { - List list = new ArrayList<>(); - // serialize the windowedValue to bytes array to comply with dataset binary schema - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); - ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - try { - windowedValueCoder.encode(windowedValue, byteArrayOutputStream); - byte[] bytes = byteArrayOutputStream.toByteArray(); - list.add(bytes); - } catch (IOException e) { - throw new RuntimeException(e); - } - return InternalRow.apply(asScalaBuffer(list).toList()); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SchemaHelpers.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SchemaHelpers.java deleted file mode 100644 index 71dca5264dd8..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SchemaHelpers.java +++ /dev/null @@ -1,39 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -/** A {@link SchemaHelpers} for the Spark Batch Runner. */ -public class SchemaHelpers { - private static final StructType BINARY_SCHEMA = - new StructType( - new StructField[] { - StructField.apply("binaryStructField", DataTypes.BinaryType, true, Metadata.empty()) - }); - - public static StructType binarySchema() { - // we use a binary schema for now because: - // using a empty schema raises a indexOutOfBoundsException - // using a NullType schema stores null in the elements - return BINARY_SCHEMA; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SideInputBroadcast.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SideInputBroadcast.java deleted file mode 100644 index 9a397f98cea1..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SideInputBroadcast.java +++ /dev/null @@ -1,49 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.coders.Coder; -import org.apache.spark.broadcast.Broadcast; - -/** Broadcast helper for side inputs. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class SideInputBroadcast implements Serializable { - - private final Map> bcast = new HashMap<>(); - private final Map> coder = new HashMap<>(); - - public SideInputBroadcast() {} - - public void add(String key, Broadcast bcast, Coder coder) { - this.bcast.put(key, bcast); - this.coder.put(key, coder); - } - - public Broadcast getBroadcastValue(String key) { - return bcast.get(key); - } - - public Coder getCoder(String key) { - return coder.get(key); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/WindowingHelpers.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/WindowingHelpers.java deleted file mode 100644 index 5085eb9f7964..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/WindowingHelpers.java +++ /dev/null @@ -1,82 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import java.util.Collection; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -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.values.PCollection; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; -import org.apache.spark.api.java.function.MapFunction; -import org.joda.time.Instant; - -/** Helper functions for working with windows. */ -public final class WindowingHelpers { - - /** - * Checks if the window transformation should be applied or skipped. - * - *

Avoid running assign windows if both source and destination are global window or if the user - * has not specified the WindowFn (meaning they are just messing with triggering or allowed - * lateness). - */ - @SuppressWarnings("unchecked") - public static boolean skipAssignWindows( - Window.Assign transform, AbstractTranslationContext context) { - WindowFn windowFnToApply = (WindowFn) transform.getWindowFn(); - PCollection input = (PCollection) context.getInput(); - WindowFn windowFnOfInput = input.getWindowingStrategy().getWindowFn(); - return windowFnToApply == null - || (windowFnOfInput instanceof GlobalWindows && windowFnToApply instanceof GlobalWindows); - } - - public static - MapFunction, WindowedValue> assignWindowsMapFunction( - WindowFn windowFn) { - return (MapFunction, WindowedValue>) - windowedValue -> { - final BoundedWindow boundedWindow = Iterables.getOnlyElement(windowedValue.getWindows()); - final T element = windowedValue.getValue(); - final Instant timestamp = windowedValue.getTimestamp(); - Collection windows = - windowFn.assignWindows( - windowFn.new AssignContext() { - - @Override - public T element() { - return element; - } - - @Override - public Instant timestamp() { - return timestamp; - } - - @Override - public BoundedWindow window() { - return boundedWindow; - } - }); - return WindowedValue.of(element, timestamp, windows, windowedValue.getPane()); - }; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/package-info.java deleted file mode 100644 index 7079eadfbe26..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Internal helpers to translate Beam pipelines to Spark streaming. */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/package-info.java deleted file mode 100644 index 2754ac500039..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Internal translators for running Beam pipelines on Spark. */ -package org.apache.beam.runners.spark.structuredstreaming.translation; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/DatasetSourceStreaming.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/DatasetSourceStreaming.java deleted file mode 100644 index c91ab8a1af60..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/DatasetSourceStreaming.java +++ /dev/null @@ -1,260 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.streaming; - -import static org.apache.beam.runners.spark.structuredstreaming.Constants.BEAM_SOURCE_OPTION; -import static org.apache.beam.runners.spark.structuredstreaming.Constants.DEFAULT_PARALLELISM; -import static org.apache.beam.runners.spark.structuredstreaming.Constants.PIPELINE_OPTIONS; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import org.apache.beam.runners.core.construction.SerializablePipelineOptions; -import org.apache.beam.runners.core.serialization.Base64Serializer; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.RowHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.SchemaHelpers; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.ContinuousReadSupport; -import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; -import org.apache.spark.sql.sources.v2.MicroBatchReadSupport; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.InputPartitionReader; -import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader; -import org.apache.spark.sql.sources.v2.reader.streaming.Offset; -import org.apache.spark.sql.types.StructType; - -/** - * This is a spark structured streaming {@link DataSourceV2} implementation that wraps an {@link - * UnboundedSource}. - * - *

As Continuous streaming is tagged experimental in spark (no aggregation support + no exactly - * once guaranty), this class does no implement {@link ContinuousReadSupport}. - * - *

Spark {@link Offset}s are ignored because: - * - *

    - *
  • resuming from checkpoint is supported by the Beam framework through {@link CheckpointMark} - *
  • {@link DatasetSourceStreaming} is a generic wrapper that could wrap a Beam {@link - * UnboundedSource} that cannot specify offset ranges - *
- * - * So, no matter the offset range specified by the spark framework, the Beam source will resume from - * its {@link CheckpointMark} in case of failure. - */ -@SuppressFBWarnings("SE_BAD_FIELD") // make spotbugs happy -@SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) -}) -class DatasetSourceStreaming implements DataSourceV2, MicroBatchReadSupport { - - @Override - public MicroBatchReader createMicroBatchReader( - Optional schema, String checkpointLocation, DataSourceOptions options) { - return new DatasetMicroBatchReader(options); - } - - /** This class is mapped to Beam {@link UnboundedSource}. */ - private static class DatasetMicroBatchReader< - T, CheckpointMarkT extends UnboundedSource.CheckpointMark> - implements MicroBatchReader, Serializable { - - private int numPartitions; - private UnboundedSource source; - private SerializablePipelineOptions serializablePipelineOptions; - - private final List partitionReaders = new ArrayList<>(); - - @SuppressWarnings("unchecked") - private DatasetMicroBatchReader(DataSourceOptions options) { - if (!options.get(BEAM_SOURCE_OPTION).isPresent()) { - throw new RuntimeException("Beam source was not set in DataSource options"); - } - this.source = - Base64Serializer.deserializeUnchecked( - options.get(BEAM_SOURCE_OPTION).get(), UnboundedSource.class); - - if (!options.get(DEFAULT_PARALLELISM).isPresent()) { - throw new RuntimeException("Spark default parallelism was not set in DataSource options"); - } - this.numPartitions = Integer.parseInt(options.get(DEFAULT_PARALLELISM).get()); - checkArgument(numPartitions > 0, "Number of partitions must be greater than zero."); - - if (!options.get(PIPELINE_OPTIONS).isPresent()) { - throw new RuntimeException("Beam pipelineOptions were not set in DataSource options"); - } - this.serializablePipelineOptions = - new SerializablePipelineOptions(options.get(PIPELINE_OPTIONS).get()); - } - - @Override - public void setOffsetRange(Optional start, Optional end) { - // offsets are ignored see javadoc - } - - @Override - public Offset getStartOffset() { - return EMPTY_OFFSET; - } - - @Override - public Offset getEndOffset() { - return EMPTY_OFFSET; - } - - @Override - public Offset deserializeOffset(String json) { - return EMPTY_OFFSET; - } - - @Override - public void commit(Offset end) { - // offsets are ignored see javadoc - for (DatasetPartitionReader partitionReader : partitionReaders) { - try { - // TODO: is checkpointMark stored in reliable storage ? - partitionReader.reader.getCheckpointMark().finalizeCheckpoint(); - } catch (IOException e) { - throw new RuntimeException( - String.format( - "Commit of Offset %s failed, checkpointMark %s finalizeCheckpoint() failed", - end, partitionReader.reader.getCheckpointMark())); - } - } - } - - @Override - public void stop() { - try { - for (DatasetPartitionReader partitionReader : partitionReaders) { - if (partitionReader.started) { - partitionReader.close(); - } - } - } catch (IOException e) { - throw new RuntimeException("Error closing " + this + "partitionReaders", e); - } - } - - @Override - public StructType readSchema() { - // TODO: find a way to extend schema with a WindowedValue schema - return SchemaHelpers.binarySchema(); - } - - @Override - public List> planInputPartitions() { - PipelineOptions options = serializablePipelineOptions.get(); - List> result = new ArrayList<>(); - try { - List> splits = - source.split(numPartitions, options); - for (UnboundedSource split : splits) { - result.add( - new InputPartition() { - - @Override - public InputPartitionReader createPartitionReader() { - DatasetPartitionReader datasetPartitionReader; - datasetPartitionReader = - new DatasetPartitionReader<>(split, serializablePipelineOptions); - partitionReaders.add(datasetPartitionReader); - return datasetPartitionReader; - } - }); - } - return result; - - } catch (Exception e) { - throw new RuntimeException( - "Error in splitting UnboundedSource " + source.getClass().getCanonicalName(), e); - } - } - } - - /** This class can be mapped to Beam {@link BoundedSource.BoundedReader}. */ - private static class DatasetPartitionReader< - T, CheckpointMarkT extends UnboundedSource.CheckpointMark> - implements InputPartitionReader { - private boolean started; - private boolean closed; - private final UnboundedSource source; - private UnboundedSource.UnboundedReader reader; - - DatasetPartitionReader( - UnboundedSource source, - SerializablePipelineOptions serializablePipelineOptions) { - this.started = false; - this.closed = false; - this.source = source; - // reader is not serializable so lazy initialize it - try { - reader = - // In - // https://blog.yuvalitzchakov.com/exploring-stateful-streaming-with-spark-structured-streaming/ - // "Structured Streaming stores and retrieves the offsets on our behalf when re-running - // the application meaning we no longer have to store them externally." - source.createReader(serializablePipelineOptions.get(), null); - } catch (IOException e) { - throw new RuntimeException("Error creating UnboundedReader ", e); - } - } - - @Override - public boolean next() throws IOException { - // TODO deal with watermark - if (!started) { - started = true; - return reader.start(); - } else { - return !closed && reader.advance(); - } - } - - @Override - public InternalRow get() { - WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow( - reader.getCurrent(), reader.getCurrentTimestamp()); - return RowHelpers.storeWindowedValueInRow(windowedValue, source.getOutputCoder()); - } - - @Override - public void close() throws IOException { - closed = true; - reader.close(); - } - } - - private static final Offset EMPTY_OFFSET = - new Offset() { - @Override - public String json() { - return "{offset : -1}"; - } - }; -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/PipelineTranslatorStreaming.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/PipelineTranslatorStreaming.java deleted file mode 100644 index 73d99efa4630..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/PipelineTranslatorStreaming.java +++ /dev/null @@ -1,93 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.streaming; - -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.runners.core.construction.SplittableParDo; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.translation.PipelineTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.TranslationContext; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.PTransform; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * {@link PipelineTranslator} for executing a {@link Pipeline} in Spark in streaming mode. This - * contains only the components specific to streaming: registry of streaming {@link - * TransformTranslator} and registry lookup code. - */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class PipelineTranslatorStreaming extends PipelineTranslator { - // -------------------------------------------------------------------------------------------- - // Transform Translator Registry - // -------------------------------------------------------------------------------------------- - - @SuppressWarnings("rawtypes") - private static final Map, TransformTranslator> TRANSFORM_TRANSLATORS = - new HashMap<>(); - - // TODO the ability to have more than one TransformTranslator per URN - // that could be dynamically chosen by a predicated that evaluates based on PCollection - // obtainable though node.getInputs.getValue() - // See - // https://github.com/seznam/euphoria/blob/master/euphoria-spark/src/main/java/cz/seznam/euphoria/spark/SparkFlowTranslator.java#L83 - // And - // https://github.com/seznam/euphoria/blob/master/euphoria-spark/src/main/java/cz/seznam/euphoria/spark/SparkFlowTranslator.java#L106 - - static { - // TRANSFORM_TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch()); - // TRANSFORM_TRANSLATORS.put(Combine.Globally.class, new CombineGloballyTranslatorBatch()); - // TRANSFORM_TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); - - // TODO: Do we need to have a dedicated translator for {@code Reshuffle} if it's deprecated? - // TRANSFORM_TRANSLATORS.put(Reshuffle.class, new ReshuffleTranslatorBatch()); - - // TRANSFORM_TRANSLATORS.put(Flatten.PCollections.class, new FlattenTranslatorBatch()); - // - // TRANSFORM_TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslatorBatch()); - // - // TRANSFORM_TRANSLATORS.put(ParDo.MultiOutput.class, new ParDoTranslatorBatch()); - - TRANSFORM_TRANSLATORS.put( - SplittableParDo.PrimitiveUnboundedRead.class, new ReadSourceTranslatorStreaming()); - - // TRANSFORM_TRANSLATORS - // .put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch()); - } - - public PipelineTranslatorStreaming(SparkStructuredStreamingPipelineOptions options) { - translationContext = new TranslationContext(options); - } - - /** Returns a translator for the given node, if it is possible, otherwise null. */ - @Override - protected TransformTranslator getTransformTranslator(TransformHierarchy.Node node) { - @Nullable PTransform transform = node.getTransform(); - // Root of the graph is null - if (transform == null) { - return null; - } - return TRANSFORM_TRANSLATORS.get(transform.getClass()); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/ReadSourceTranslatorStreaming.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/ReadSourceTranslatorStreaming.java deleted file mode 100644 index 8abc8771a4e8..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/ReadSourceTranslatorStreaming.java +++ /dev/null @@ -1,87 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.streaming; - -import static org.apache.beam.runners.spark.structuredstreaming.Constants.BEAM_SOURCE_OPTION; -import static org.apache.beam.runners.spark.structuredstreaming.Constants.DEFAULT_PARALLELISM; -import static org.apache.beam.runners.spark.structuredstreaming.Constants.PIPELINE_OPTIONS; - -import java.io.IOException; -import org.apache.beam.runners.core.construction.ReadTranslation; -import org.apache.beam.runners.core.serialization.Base64Serializer; -import org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext; -import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers; -import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.RowHelpers; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - -class ReadSourceTranslatorStreaming - implements TransformTranslator>> { - - private static final String sourceProviderClass = DatasetSourceStreaming.class.getCanonicalName(); - - @SuppressWarnings("unchecked") - @Override - public void translateTransform( - PTransform> transform, AbstractTranslationContext context) { - AppliedPTransform, PTransform>> rootTransform = - (AppliedPTransform, PTransform>>) - context.getCurrentTransform(); - - UnboundedSource source; - try { - source = ReadTranslation.unboundedSourceFromTransform(rootTransform); - } catch (IOException e) { - throw new RuntimeException(e); - } - SparkSession sparkSession = context.getSparkSession(); - - String serializedSource = Base64Serializer.serializeUnchecked(source); - Dataset rowDataset = - sparkSession - .readStream() - .format(sourceProviderClass) - .option(BEAM_SOURCE_OPTION, serializedSource) - .option( - DEFAULT_PARALLELISM, - String.valueOf(context.getSparkSession().sparkContext().defaultParallelism())) - .option(PIPELINE_OPTIONS, context.getSerializableOptions().toString()) - .load(); - - // extract windowedValue from Row - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of( - source.getOutputCoder(), GlobalWindow.Coder.INSTANCE); - Dataset> dataset = - rowDataset.map( - RowHelpers.extractWindowedValueFromRowMapFunction(windowedValueCoder), - EncoderHelpers.fromBeamCoder(windowedValueCoder)); - - PCollection output = (PCollection) context.getOutput(); - context.putDataset(output, dataset); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/package-info.java deleted file mode 100644 index 67f3613e056b..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Internal utilities to translate Beam pipelines to Spark streaming. */ -package org.apache.beam.runners.spark.structuredstreaming.translation.streaming; diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/CachedSideInputReader.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/CachedSideInputReader.java deleted file mode 100644 index d2e4751c8117..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/CachedSideInputReader.java +++ /dev/null @@ -1,93 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.utils; - -import java.util.concurrent.ExecutionException; -import org.apache.beam.runners.core.SideInputReader; -import org.apache.beam.runners.spark.structuredstreaming.translation.utils.SideInputStorage.Key; -import org.apache.beam.runners.spark.structuredstreaming.translation.utils.SideInputStorage.Value; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache; -import org.apache.spark.util.SizeEstimator; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** {@link SideInputReader} that caches materialized views. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class CachedSideInputReader implements SideInputReader { - - private static final Logger LOG = LoggerFactory.getLogger(CachedSideInputReader.class); - - /** - * Create a new cached {@link SideInputReader}. - * - * @param delegate wrapped reader - * @return cached reader - */ - public static CachedSideInputReader of(SideInputReader delegate) { - return new CachedSideInputReader(delegate); - } - - /** Wrapped {@link SideInputReader} which results will be cached. */ - private final SideInputReader delegate; - - private CachedSideInputReader(SideInputReader delegate) { - this.delegate = delegate; - } - - @Override - public @Nullable T get(PCollectionView view, BoundedWindow window) { - @SuppressWarnings("unchecked") - final Cache, Value> materializedCasted = - (Cache) SideInputStorage.getMaterializedSideInputs(); - - Key sideInputKey = new Key<>(view, window); - - try { - Value cachedResult = - materializedCasted.get( - sideInputKey, - () -> { - final T result = delegate.get(view, window); - LOG.debug( - "Caching de-serialized side input for {} of size [{}B] in memory.", - sideInputKey, - SizeEstimator.estimate(result)); - - return new Value<>(result); - }); - return cachedResult.getValue(); - } catch (ExecutionException e) { - throw new RuntimeException(e.getCause()); - } - } - - @Override - public boolean contains(PCollectionView view) { - return delegate.contains(view); - } - - @Override - public boolean isEmpty() { - return delegate.isEmpty(); - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/ScalaInterop.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/ScalaInterop.java deleted file mode 100644 index c5bc71af6026..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/ScalaInterop.java +++ /dev/null @@ -1,40 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.utils; - -import scala.collection.Seq; -import scala.collection.immutable.List; -import scala.collection.immutable.Nil$; -import scala.collection.mutable.WrappedArray; - -/** Utilities for easier interoperability with the Spark Scala API. */ -public class ScalaInterop { - private ScalaInterop() {} - - public static Seq seqOf(T... t) { - return new WrappedArray.ofRef<>(t); - } - - public static Seq listOf(T t) { - return emptyList().$colon$colon(t); - } - - public static List emptyList() { - return (List) Nil$.MODULE$; - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/SideInputStorage.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/SideInputStorage.java deleted file mode 100644 index 4febddd4f883..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/SideInputStorage.java +++ /dev/null @@ -1,107 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.utils; - -import java.util.Objects; -import java.util.concurrent.TimeUnit; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * Cache deserialized side inputs for executor so every task doesn't need to deserialize them again. - * Side inputs are stored in {@link Cache} with 5 minutes expireAfterAccess. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class SideInputStorage { - - /** JVM deserialized side input cache. */ - private static final Cache, Value> materializedSideInputs = - CacheBuilder.newBuilder().expireAfterAccess(5, TimeUnit.MINUTES).build(); - - static Cache, Value> getMaterializedSideInputs() { - return materializedSideInputs; - } - - /** - * Composite key of {@link PCollectionView} and {@link BoundedWindow} used to identify - * materialized results. - * - * @param type of result - */ - public static class Key { - - private final PCollectionView view; - private final BoundedWindow window; - - Key(PCollectionView view, BoundedWindow window) { - this.view = view; - this.window = window; - } - - @Override - public boolean equals(@Nullable Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Key key = (Key) o; - return Objects.equals(view, key.view) && Objects.equals(window, key.window); - } - - @Override - public int hashCode() { - return Objects.hash(view, window); - } - - @Override - public String toString() { - String pName = view.getPCollection() != null ? view.getPCollection().getName() : "Unknown"; - return "Key{" - + "view=" - + view.getTagInternal() - + " of PCollection[" - + pName - + "], window=" - + window - + '}'; - } - } - - /** - * Null value is not allowed in guava's Cache and is valid in SideInput so we use wrapper for - * cache value. - */ - public static class Value { - final T value; - - Value(T value) { - this.value = value; - } - - public T getValue() { - return value; - } - } -} diff --git a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/package-info.java b/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/package-info.java deleted file mode 100644 index 470bef88fb4b..000000000000 --- a/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Internal utils to translate Beam pipelines to Spark streaming. */ -package org.apache.beam.runners.spark.structuredstreaming.translation.utils; diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java deleted file mode 100644 index 33eef26dddda..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java +++ /dev/null @@ -1,88 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming; - -import static java.util.stream.Collectors.toMap; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.Map; -import javax.annotation.Nullable; -import org.apache.beam.runners.spark.structuredstreaming.translation.SparkSessionFactory; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.values.KV; -import org.apache.spark.sql.SparkSession; -import org.junit.rules.ExternalResource; -import org.junit.runner.Description; -import org.junit.runners.model.Statement; - -public class SparkSessionRule extends ExternalResource implements Serializable { - private transient SparkSession.Builder builder; - private transient @Nullable SparkSession session = null; - - public SparkSessionRule(String sparkMaster, Map sparkConfig) { - builder = SparkSessionFactory.sessionBuilder(sparkMaster); - sparkConfig.forEach(builder::config); - } - - public SparkSessionRule(KV... sparkConfig) { - this("local[2]", sparkConfig); - } - - public SparkSessionRule(String sparkMaster, KV... sparkConfig) { - this(sparkMaster, Arrays.stream(sparkConfig).collect(toMap(KV::getKey, KV::getValue))); - } - - public SparkSession getSession() { - if (session == null) { - throw new IllegalStateException("SparkSession not available"); - } - return session; - } - - public PipelineOptions createPipelineOptions() { - return configure(TestPipeline.testingPipelineOptions()); - } - - public PipelineOptions configure(PipelineOptions options) { - SparkStructuredStreamingPipelineOptions opts = - options.as(SparkStructuredStreamingPipelineOptions.class); - opts.setUseActiveSparkSession(true); - opts.setRunner(SparkStructuredStreamingRunner.class); - opts.setTestMode(true); - return opts; - } - - @Override - public Statement apply(Statement base, Description description) { - builder.appName(description.getDisplayName()); - return super.apply(base, description); - } - - @Override - protected void before() throws Throwable { - session = builder.getOrCreate(); - } - - @Override - protected void after() { - getSession().stop(); - session = null; - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunnerRegistrarTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunnerRegistrarTest.java deleted file mode 100644 index 30d8297809b9..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunnerRegistrarTest.java +++ /dev/null @@ -1,70 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -import java.util.ServiceLoader; -import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test {@link SparkStructuredStreamingRunnerRegistrar}. */ -@RunWith(JUnit4.class) -public class SparkStructuredStreamingRunnerRegistrarTest { - @Test - public void testOptions() { - assertEquals( - ImmutableList.of(SparkStructuredStreamingPipelineOptions.class), - new SparkStructuredStreamingRunnerRegistrar.Options().getPipelineOptions()); - } - - @Test - public void testRunners() { - assertEquals( - ImmutableList.of(SparkStructuredStreamingRunner.class), - new SparkStructuredStreamingRunnerRegistrar.Runner().getPipelineRunners()); - } - - @Test - public void testServiceLoaderForOptions() { - for (PipelineOptionsRegistrar registrar : - Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) { - if (registrar instanceof SparkStructuredStreamingRunnerRegistrar.Options) { - return; - } - } - fail("Expected to find " + SparkStructuredStreamingRunnerRegistrar.Options.class); - } - - @Test - public void testServiceLoaderForRunner() { - for (PipelineRunnerRegistrar registrar : - Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) { - if (registrar instanceof SparkStructuredStreamingRunnerRegistrar.Runner) { - return; - } - } - fail("Expected to find " + SparkStructuredStreamingRunnerRegistrar.Runner.class); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java deleted file mode 100644 index b44df7bf101b..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java +++ /dev/null @@ -1,225 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming; - -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.fail; - -import java.io.Serializable; -import org.apache.beam.runners.spark.io.CreateStream; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.joda.time.Duration; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestName; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** This suite tests that various scenarios result in proper states of the pipeline. */ -@RunWith(JUnit4.class) -public class StructuredStreamingPipelineStateTest implements Serializable { - - private static class MyCustomException extends RuntimeException { - - MyCustomException(final String message) { - super(message); - } - } - - private final transient SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - - @Rule public transient TestName testName = new TestName(); - - private static final String FAILED_THE_BATCH_INTENTIONALLY = "Failed the batch intentionally"; - - private ParDo.SingleOutput printParDo(final String prefix) { - return ParDo.of( - new DoFn() { - - @ProcessElement - public void processElement(final ProcessContext c) { - System.out.println(prefix + " " + c.element()); - } - }); - } - - private PTransform> getValues( - final SparkStructuredStreamingPipelineOptions options) { - final boolean doNotSyncWithWatermark = false; - return options.isStreaming() - ? CreateStream.of(StringUtf8Coder.of(), Duration.millis(1), doNotSyncWithWatermark) - .nextBatch("one", "two") - : Create.of("one", "two"); - } - - private SparkStructuredStreamingPipelineOptions getStreamingOptions() { - options.setRunner(SparkStructuredStreamingRunner.class); - options.setStreaming(true); - return options; - } - - private SparkStructuredStreamingPipelineOptions getBatchOptions() { - options.setRunner(SparkStructuredStreamingRunner.class); - options.setStreaming(false); // explicit because options is reused throughout the test. - return options; - } - - private Pipeline getPipeline(final SparkStructuredStreamingPipelineOptions options) { - - final Pipeline pipeline = Pipeline.create(options); - final String name = testName.getMethodName() + "(isStreaming=" + options.isStreaming() + ")"; - - pipeline.apply(getValues(options)).setCoder(StringUtf8Coder.of()).apply(printParDo(name)); - - return pipeline; - } - - private void testFailedPipeline(final SparkStructuredStreamingPipelineOptions options) - throws Exception { - - SparkStructuredStreamingPipelineResult result = null; - - try { - final Pipeline pipeline = Pipeline.create(options); - pipeline - .apply(getValues(options)) - .setCoder(StringUtf8Coder.of()) - .apply( - MapElements.via( - new SimpleFunction() { - - @Override - public String apply(final String input) { - throw new MyCustomException(FAILED_THE_BATCH_INTENTIONALLY); - } - })); - - result = (SparkStructuredStreamingPipelineResult) pipeline.run(); - result.waitUntilFinish(); - } catch (final Exception e) { - assertThat(e, instanceOf(Pipeline.PipelineExecutionException.class)); - assertThat(e.getCause(), instanceOf(MyCustomException.class)); - assertThat(e.getCause().getMessage(), is(FAILED_THE_BATCH_INTENTIONALLY)); - assertThat(result.getState(), is(PipelineResult.State.FAILED)); - result.cancel(); - return; - } - - fail("An injected failure did not affect the pipeline as expected."); - } - - private void testTimeoutPipeline(final SparkStructuredStreamingPipelineOptions options) - throws Exception { - - final Pipeline pipeline = getPipeline(options); - - final SparkStructuredStreamingPipelineResult result = - (SparkStructuredStreamingPipelineResult) pipeline.run(); - - result.waitUntilFinish(Duration.millis(1)); - - assertThat(result.getState(), is(PipelineResult.State.RUNNING)); - - result.cancel(); - } - - private void testCanceledPipeline(final SparkStructuredStreamingPipelineOptions options) - throws Exception { - - final Pipeline pipeline = getPipeline(options); - - final SparkStructuredStreamingPipelineResult result = - (SparkStructuredStreamingPipelineResult) pipeline.run(); - - result.cancel(); - - assertThat(result.getState(), is(PipelineResult.State.CANCELLED)); - } - - private void testRunningPipeline(final SparkStructuredStreamingPipelineOptions options) - throws Exception { - - final Pipeline pipeline = getPipeline(options); - - final SparkStructuredStreamingPipelineResult result = - (SparkStructuredStreamingPipelineResult) pipeline.run(); - - assertThat(result.getState(), is(PipelineResult.State.RUNNING)); - - result.cancel(); - } - - @Ignore("TODO: Reactivate with streaming.") - @Test - public void testStreamingPipelineRunningState() throws Exception { - testRunningPipeline(getStreamingOptions()); - } - - @Test - public void testBatchPipelineRunningState() throws Exception { - testRunningPipeline(getBatchOptions()); - } - - @Ignore("TODO: Reactivate with streaming.") - @Test - public void testStreamingPipelineCanceledState() throws Exception { - testCanceledPipeline(getStreamingOptions()); - } - - @Test - public void testBatchPipelineCanceledState() throws Exception { - testCanceledPipeline(getBatchOptions()); - } - - @Ignore("TODO: Reactivate with streaming.") - @Test - public void testStreamingPipelineFailedState() throws Exception { - testFailedPipeline(getStreamingOptions()); - } - - @Test - public void testBatchPipelineFailedState() throws Exception { - testFailedPipeline(getBatchOptions()); - } - - @Ignore("TODO: Reactivate with streaming.") - @Test - public void testStreamingPipelineTimeoutState() throws Exception { - testTimeoutPipeline(getStreamingOptions()); - } - - @Test - public void testBatchPipelineTimeoutState() throws Exception { - testTimeoutPipeline(getBatchOptions()); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetrics.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetrics.java deleted file mode 100644 index f994f7712b32..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetrics.java +++ /dev/null @@ -1,80 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.aggregators.metrics.sink; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.MetricFilter; -import com.codahale.metrics.MetricRegistry; -import java.util.Collection; -import java.util.Properties; -import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; -import org.apache.spark.metrics.sink.Sink; - -/** An in-memory {@link Sink} implementation for tests. */ -public class InMemoryMetrics implements Sink { - - private static WithMetricsSupport extendedMetricsRegistry; - private static MetricRegistry internalMetricRegistry; - - // Constructor for Spark 3.1 - @SuppressWarnings("UnusedParameters") - public InMemoryMetrics( - final Properties properties, - final MetricRegistry metricRegistry, - final org.apache.spark.SecurityManager securityMgr) { - extendedMetricsRegistry = WithMetricsSupport.forRegistry(metricRegistry); - internalMetricRegistry = metricRegistry; - } - - // Constructor for Spark >= 3.2 - @SuppressWarnings("UnusedParameters") - public InMemoryMetrics(final Properties properties, final MetricRegistry metricRegistry) { - extendedMetricsRegistry = WithMetricsSupport.forRegistry(metricRegistry); - internalMetricRegistry = metricRegistry; - } - - @SuppressWarnings({"TypeParameterUnusedInFormals", "rawtypes"}) - public static T valueOf(final String name) { - // this might fail in case we have multiple aggregators with the same suffix after - // the last dot, but it should be good enough for tests. - if (extendedMetricsRegistry != null) { - Collection matches = - extendedMetricsRegistry.getGauges((n, m) -> n.endsWith(name)).values(); - return matches.isEmpty() ? null : (T) Iterables.getOnlyElement(matches).getValue(); - } else { - return null; - } - } - - @SuppressWarnings("WeakerAccess") - public static void clearAll() { - if (internalMetricRegistry != null) { - internalMetricRegistry.removeMatching(MetricFilter.ALL); - } - } - - @Override - public void start() {} - - @Override - public void stop() {} - - @Override - public void report() {} -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetricsSinkRule.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetricsSinkRule.java deleted file mode 100644 index f1b996eaf21b..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetricsSinkRule.java +++ /dev/null @@ -1,28 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.aggregators.metrics.sink; - -import org.junit.rules.ExternalResource; - -/** A rule that cleans the {@link InMemoryMetrics} after the tests has finished. */ -class InMemoryMetricsSinkRule extends ExternalResource { - @Override - protected void before() throws Throwable { - InMemoryMetrics.clearAll(); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java deleted file mode 100644 index 2f02656dc37e..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java +++ /dev/null @@ -1,73 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.aggregators.metrics.sink; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; - -import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule; -import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExternalResource; - -/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */ -public class SparkMetricsSinkTest { - - @ClassRule - public static final SparkSessionRule SESSION = - new SparkSessionRule( - KV.of("spark.metrics.conf.*.sink.memory.class", InMemoryMetrics.class.getName())); - - @Rule public final ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule(); - - @Rule - public final TestPipeline pipeline = TestPipeline.fromOptions(SESSION.createPipelineOptions()); - - private static final ImmutableList WORDS = - ImmutableList.of("hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"); - private static final ImmutableSet EXPECTED_COUNTS = - ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2"); - - @Test - public void testInBatchMode() throws Exception { - assertThat(InMemoryMetrics.valueOf("emptyLines"), is(nullValue())); - - final PCollection output = - pipeline - .apply(Create.of(WORDS).withCoder(StringUtf8Coder.of())) - .apply(new WordCount.CountWords()) - .apply(MapElements.via(new WordCount.FormatAsTextFn())); - - PAssert.that(output).containsInAnyOrder(EXPECTED_COUNTS); - pipeline.run(); - - assertThat(InMemoryMetrics.valueOf("emptyLines"), is(1d)); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetricTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetricTest.java deleted file mode 100644 index fd0aa35e5c8d..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/metrics/SparkBeamMetricTest.java +++ /dev/null @@ -1,59 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.metrics; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; - -import org.apache.beam.sdk.metrics.MetricKey; -import org.apache.beam.sdk.metrics.MetricName; -import org.apache.beam.sdk.metrics.MetricResult; -import org.junit.Test; - -/** Test BeamMetric. */ -public class SparkBeamMetricTest { - @Test - public void testRenderName() { - MetricResult metricResult = - MetricResult.create( - MetricKey.create( - "myStep.one.two(three)", MetricName.named("myNameSpace//", "myName()")), - 123, - 456); - String renderedName = SparkBeamMetric.renderName("", metricResult); - assertThat( - "Metric name was not rendered correctly", - renderedName, - equalTo("myStep_one_two_three.myNameSpace__.myName__")); - } - - @Test - public void testRenderNameWithPrefix() { - MetricResult metricResult = - MetricResult.create( - MetricKey.create( - "myStep.one.two(three)", MetricName.named("myNameSpace//", "myName()")), - 123, - 456); - String renderedName = SparkBeamMetric.renderName("prefix", metricResult); - assertThat( - "Metric name was not rendered correctly", - renderedName, - equalTo("prefix.myStep_one_two_three.myNameSpace__.myName__")); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineTest.java deleted file mode 100644 index 52e60a3db545..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineTest.java +++ /dev/null @@ -1,186 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.SerializableBiFunction; -import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; -import org.apache.beam.sdk.transforms.windowing.SlidingWindows; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TimestampedValue; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test class for beam to spark {@link org.apache.beam.sdk.transforms.Combine} translation. */ -@RunWith(JUnit4.class) -public class CombineTest implements Serializable { - private static Pipeline pipeline; - - @BeforeClass - public static void beforeClass() { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - options.setTestMode(true); - pipeline = Pipeline.create(options); - } - - @Test - public void testCombineGlobally() { - PCollection input = - pipeline.apply(Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)).apply(Sum.integersGlobally()); - PAssert.that(input).containsInAnyOrder(55); - // uses combine per key - pipeline.run(); - } - - @Test - public void testCombineGloballyPreservesWindowing() { - PCollection input = - pipeline - .apply( - Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2)), - TimestampedValue.of(3, new Instant(11)), - TimestampedValue.of(4, new Instant(3)), - TimestampedValue.of(5, new Instant(11)), - TimestampedValue.of(6, new Instant(12)))) - .apply(Window.into(FixedWindows.of(Duration.millis(10)))) - .apply(Combine.globally(Sum.ofIntegers()).withoutDefaults()); - PAssert.that(input).containsInAnyOrder(7, 14); - } - - @Test - public void testCombinePerKey() { - List> elems = new ArrayList<>(); - elems.add(KV.of(1, 1)); - elems.add(KV.of(1, 3)); - elems.add(KV.of(1, 5)); - elems.add(KV.of(2, 2)); - elems.add(KV.of(2, 4)); - elems.add(KV.of(2, 6)); - - PCollection> input = - pipeline.apply(Create.of(elems)).apply(Sum.integersPerKey()); - PAssert.that(input).containsInAnyOrder(KV.of(1, 9), KV.of(2, 12)); - pipeline.run(); - } - - @Test - public void testCombinePerKeyPreservesWindowing() { - PCollection> input = - pipeline - .apply( - Create.timestamped( - TimestampedValue.of(KV.of(1, 1), new Instant(1)), - TimestampedValue.of(KV.of(1, 3), new Instant(2)), - TimestampedValue.of(KV.of(1, 5), new Instant(11)), - TimestampedValue.of(KV.of(2, 2), new Instant(3)), - TimestampedValue.of(KV.of(2, 4), new Instant(11)), - TimestampedValue.of(KV.of(2, 6), new Instant(12)))) - .apply(Window.into(FixedWindows.of(Duration.millis(10)))) - .apply(Sum.integersPerKey()); - PAssert.that(input).containsInAnyOrder(KV.of(1, 4), KV.of(1, 5), KV.of(2, 2), KV.of(2, 10)); - pipeline.run(); - } - - @Test - public void testCombinePerKeyWithSlidingWindows() { - PCollection> input = - pipeline - .apply( - Create.timestamped( - TimestampedValue.of(KV.of(1, 1), new Instant(1)), - TimestampedValue.of(KV.of(1, 3), new Instant(2)), - TimestampedValue.of(KV.of(1, 5), new Instant(3)), - TimestampedValue.of(KV.of(1, 2), new Instant(1)), - TimestampedValue.of(KV.of(1, 4), new Instant(2)), - TimestampedValue.of(KV.of(1, 6), new Instant(3)))) - .apply(Window.into(SlidingWindows.of(Duration.millis(3)).every(Duration.millis(1)))) - .apply(Sum.integersPerKey()); - PAssert.that(input) - .containsInAnyOrder( - KV.of(1, 1 + 2), - KV.of(1, 1 + 2 + 3 + 4), - KV.of(1, 1 + 3 + 5 + 2 + 4 + 6), - KV.of(1, 3 + 4 + 5 + 6), - KV.of(1, 5 + 6)); - pipeline.run(); - } - - @Test - public void testBinaryCombineWithSlidingWindows() { - PCollection input = - pipeline - .apply( - Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(3, new Instant(2)), - TimestampedValue.of(5, new Instant(3)))) - .apply(Window.into(SlidingWindows.of(Duration.millis(3)).every(Duration.millis(1)))) - .apply( - Combine.globally( - Combine.BinaryCombineFn.of( - (SerializableBiFunction) - (integer1, integer2) -> integer1 > integer2 ? integer1 : integer2)) - .withoutDefaults()); - PAssert.that(input).containsInAnyOrder(1, 3, 5, 5, 5); - pipeline.run(); - } - - @Test - public void testCountPerElementWithSlidingWindows() { - PCollection input = - pipeline - .apply( - Create.timestamped( - TimestampedValue.of("a", new Instant(1)), - TimestampedValue.of("a", new Instant(2)), - TimestampedValue.of("b", new Instant(3)), - TimestampedValue.of("b", new Instant(4)))) - .apply(Window.into(SlidingWindows.of(Duration.millis(2)).every(Duration.millis(1)))); - PCollection> output = input.apply(Count.perElement()); - PAssert.that(output) - .containsInAnyOrder( - KV.of("a", 1L), - KV.of("a", 2L), - KV.of("a", 1L), - KV.of("b", 1L), - KV.of("b", 2L), - KV.of("b", 1L)); - pipeline.run(); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ComplexSourceTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ComplexSourceTest.java deleted file mode 100644 index 0175d03f8753..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ComplexSourceTest.java +++ /dev/null @@ -1,86 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.io.PrintStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.values.PCollection; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test class for beam to spark source translation. */ -@RunWith(JUnit4.class) -public class ComplexSourceTest implements Serializable { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - private static File file; - private static List lines = createLines(30); - - private static Pipeline pipeline; - - @BeforeClass - public static void beforeClass() throws IOException { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - options.setTestMode(true); - pipeline = Pipeline.create(options); - file = createFile(lines); - } - - @Test - public void testBoundedSource() { - PCollection input = pipeline.apply(TextIO.read().from(file.getPath())); - PAssert.that(input).containsInAnyOrder(lines); - pipeline.run(); - } - - private static File createFile(List lines) throws IOException { - File file = TEMPORARY_FOLDER.newFile(); - OutputStream outputStream = new FileOutputStream(file); - try (PrintStream writer = new PrintStream(outputStream)) { - for (String line : lines) { - writer.println(line); - } - } - return file; - } - - private static List createLines(int size) { - List lines = new ArrayList<>(); - for (int i = 0; i < size; ++i) { - lines.add("word" + i); - } - return lines; - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTest.java deleted file mode 100644 index e126d06e6852..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTest.java +++ /dev/null @@ -1,59 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.Serializable; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionList; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test class for beam to spark flatten translation. */ -@RunWith(JUnit4.class) -public class FlattenTest implements Serializable { - private static Pipeline pipeline; - - @BeforeClass - public static void beforeClass() { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - options.setTestMode(true); - pipeline = Pipeline.create(options); - } - - @Test - public void testFlatten() { - PCollection input1 = pipeline.apply(Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - PCollection input2 = pipeline.apply(Create.of(11, 12, 13, 14, 15, 16, 17, 18, 19, 20)); - PCollectionList pcs = PCollectionList.of(input1).and(input2); - PCollection input = pcs.apply(Flatten.pCollections()); - PAssert.that(input) - .containsInAnyOrder(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20); - pipeline.run(); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java deleted file mode 100644 index 07850232853a..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java +++ /dev/null @@ -1,124 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import static org.apache.beam.sdk.testing.SerializableMatchers.containsInAnyOrder; -import static org.hamcrest.MatcherAssert.assertThat; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.GroupByKey; -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.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test class for beam to spark {@link ParDo} translation. */ -@RunWith(JUnit4.class) -public class GroupByKeyTest implements Serializable { - private static Pipeline pipeline; - - @BeforeClass - public static void beforeClass() { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - options.setTestMode(true); - pipeline = Pipeline.create(options); - } - - @Test - public void testGroupByKeyPreservesWindowing() { - pipeline - .apply( - Create.timestamped( - TimestampedValue.of(KV.of(1, 1), new Instant(1)), - TimestampedValue.of(KV.of(1, 3), new Instant(2)), - TimestampedValue.of(KV.of(1, 5), new Instant(11)), - TimestampedValue.of(KV.of(2, 2), new Instant(3)), - TimestampedValue.of(KV.of(2, 4), new Instant(11)), - TimestampedValue.of(KV.of(2, 6), new Instant(12)))) - .apply(Window.into(FixedWindows.of(Duration.millis(10)))) - .apply(GroupByKey.create()) - // do manual assertion for windows because Passert do not support multiple kv with same key - // (because multiple windows) - .apply( - ParDo.of( - new DoFn>, KV>>() { - - @ProcessElement - public void processElement(ProcessContext context) { - KV> element = context.element(); - if (element.getKey() == 1) { - if (Iterables.size(element.getValue()) == 2) { - assertThat(element.getValue(), containsInAnyOrder(1, 3)); // window [0-10) - } else { - assertThat(element.getValue(), containsInAnyOrder(5)); // window [10-20) - } - } else { // key == 2 - if (Iterables.size(element.getValue()) == 2) { - assertThat(element.getValue(), containsInAnyOrder(4, 6)); // window [10-20) - } else { - assertThat(element.getValue(), containsInAnyOrder(2)); // window [0-10) - } - } - context.output(element); - } - })); - pipeline.run(); - } - - @Test - public void testGroupByKey() { - List> elems = new ArrayList<>(); - elems.add(KV.of(1, 1)); - elems.add(KV.of(1, 3)); - elems.add(KV.of(1, 5)); - elems.add(KV.of(2, 2)); - elems.add(KV.of(2, 4)); - elems.add(KV.of(2, 6)); - - PCollection>> input = - pipeline.apply(Create.of(elems)).apply(GroupByKey.create()); - PAssert.thatMap(input) - .satisfies( - results -> { - assertThat(results.get(1), containsInAnyOrder(1, 3, 5)); - assertThat(results.get(2), containsInAnyOrder(2, 4, 6)); - return null; - }); - pipeline.run(); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTest.java deleted file mode 100644 index 16d9a8b7fa87..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTest.java +++ /dev/null @@ -1,153 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.Serializable; -import java.util.List; -import java.util.Map; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test class for beam to spark {@link ParDo} translation. */ -@RunWith(JUnit4.class) -public class ParDoTest implements Serializable { - private static Pipeline pipeline; - - @BeforeClass - public static void beforeClass() { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - options.setTestMode(true); - pipeline = Pipeline.create(options); - } - - @Test - public void testPardo() { - PCollection input = - pipeline.apply(Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)).apply(ParDo.of(PLUS_ONE_DOFN)); - PAssert.that(input).containsInAnyOrder(2, 3, 4, 5, 6, 7, 8, 9, 10, 11); - pipeline.run(); - } - - @Test - public void testTwoPardoInRow() { - PCollection input = - pipeline - .apply(Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) - .apply(ParDo.of(PLUS_ONE_DOFN)) - .apply(ParDo.of(PLUS_ONE_DOFN)); - PAssert.that(input).containsInAnyOrder(3, 4, 5, 6, 7, 8, 9, 10, 11, 12); - pipeline.run(); - } - - @Test - public void testSideInputAsList() { - PCollectionView> sideInputView = - pipeline.apply("Create sideInput", Create.of(1, 2, 3)).apply(View.asList()); - PCollection input = - pipeline - .apply("Create input", Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) - .apply( - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - List sideInputValue = c.sideInput(sideInputView); - if (!sideInputValue.contains(c.element())) { - c.output(c.element()); - } - } - }) - .withSideInputs(sideInputView)); - PAssert.that(input).containsInAnyOrder(4, 5, 6, 7, 8, 9, 10); - pipeline.run(); - } - - @Test - public void testSideInputAsSingleton() { - PCollectionView sideInputView = - pipeline.apply("Create sideInput", Create.of(1)).apply(View.asSingleton()); - - PCollection input = - pipeline - .apply("Create input", Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) - .apply( - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - Integer sideInputValue = c.sideInput(sideInputView); - if (!sideInputValue.equals(c.element())) { - c.output(c.element()); - } - } - }) - .withSideInputs(sideInputView)); - - PAssert.that(input).containsInAnyOrder(2, 3, 4, 5, 6, 7, 8, 9, 10); - pipeline.run(); - } - - @Test - public void testSideInputAsMap() { - PCollectionView> sideInputView = - pipeline - .apply("Create sideInput", Create.of(KV.of("key1", 1), KV.of("key2", 2))) - .apply(View.asMap()); - PCollection input = - pipeline - .apply("Create input", Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) - .apply( - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - Map sideInputValue = c.sideInput(sideInputView); - if (!sideInputValue.containsKey("key" + c.element())) { - c.output(c.element()); - } - } - }) - .withSideInputs(sideInputView)); - PAssert.that(input).containsInAnyOrder(3, 4, 5, 6, 7, 8, 9, 10); - pipeline.run(); - } - - private static final DoFn PLUS_ONE_DOFN = - new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element() + 1); - } - }; -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/SimpleSourceTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/SimpleSourceTest.java deleted file mode 100644 index 70cdca630b9b..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/SimpleSourceTest.java +++ /dev/null @@ -1,53 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.Serializable; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.values.PCollection; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test class for beam to spark source translation. */ -@RunWith(JUnit4.class) -public class SimpleSourceTest implements Serializable { - private static Pipeline pipeline; - - @BeforeClass - public static void beforeClass() { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - options.setTestMode(true); - pipeline = Pipeline.create(options); - } - - @Test - public void testBoundedSource() { - PCollection input = pipeline.apply(Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - PAssert.that(input).containsInAnyOrder(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - pipeline.run(); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java deleted file mode 100644 index b8b41010a24b..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java +++ /dev/null @@ -1,69 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.batch; - -import java.io.Serializable; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -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.values.PCollection; -import org.apache.beam.sdk.values.TimestampedValue; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test class for beam to spark window assign translation. */ -@RunWith(JUnit4.class) -public class WindowAssignTest implements Serializable { - private static Pipeline pipeline; - - @BeforeClass - public static void beforeClass() { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - options.setTestMode(true); - pipeline = Pipeline.create(options); - } - - @Test - public void testWindowAssign() { - PCollection input = - pipeline - .apply( - Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2)), - TimestampedValue.of(3, new Instant(3)), - TimestampedValue.of(4, new Instant(10)), - TimestampedValue.of(5, new Instant(11)))) - .apply(Window.into(FixedWindows.of(Duration.millis(10)))) - .apply(Sum.integersGlobally().withoutDefaults()); - PAssert.that(input).containsInAnyOrder(6, 9); - pipeline.run(); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java deleted file mode 100644 index c8a8fba8d281..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java +++ /dev/null @@ -1,98 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.helpers; - -import static java.util.Arrays.asList; -import static org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers.fromBeamCoder; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.DelegateCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoder; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test of the wrapping of Beam Coders as Spark ExpressionEncoders. */ -@RunWith(JUnit4.class) -public class EncoderHelpersTest { - - @ClassRule public static SparkSessionRule sessionRule = new SparkSessionRule(); - - private Dataset createDataset(List data, Encoder encoder) { - Dataset ds = sessionRule.getSession().createDataset(data, encoder); - ds.printSchema(); - return ds; - } - - @Test - public void beamCoderToSparkEncoderTest() { - List data = Arrays.asList(1, 2, 3); - Dataset dataset = createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of())); - assertEquals(data, dataset.collectAsList()); - } - - @Test - public void testBeamEncoderOfPrivateType() { - // Verify concrete types are not used in coder generation. - // In case of private types this would cause an IllegalAccessError. - List data = asList(new PrivateString("1"), new PrivateString("2")); - Dataset dataset = createDataset(data, fromBeamCoder(PrivateString.CODER)); - assertThat(dataset.collect(), equalTo(data.toArray())); - } - - private static class PrivateString { - private static final Coder CODER = - DelegateCoder.of( - StringUtf8Coder.of(), - str -> str.string, - PrivateString::new, - new TypeDescriptor() {}); - - private final String string; - - public PrivateString(String string) { - this.string = string; - } - - @Override - public boolean equals(Object o) { - if (o == null || getClass() != o.getClass()) { - return false; - } - PrivateString that = (PrivateString) o; - return Objects.equals(string, that.string); - } - - @Override - public int hashCode() { - return Objects.hash(string); - } - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java deleted file mode 100644 index a06d2cec1e9e..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java +++ /dev/null @@ -1,57 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.translation.streaming; - -import java.io.Serializable; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; -import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.GenerateSequence; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Test class for beam to spark source translation. */ -@RunWith(JUnit4.class) -public class SimpleSourceTest implements Serializable { - private static Pipeline pipeline; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @BeforeClass - public static void beforeClass() { - SparkStructuredStreamingPipelineOptions options = - PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); - options.setRunner(SparkStructuredStreamingRunner.class); - options.setTestMode(true); - pipeline = Pipeline.create(options); - } - - @Ignore - @Test - public void testUnboundedSource() { - // produces an unbounded PCollection of longs from 0 to Long.MAX_VALUE which elements - // have processing time as event timestamps. - pipeline.apply(GenerateSequence.from(0L)); - pipeline.run(); - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/SerializationDebugger.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/SerializationDebugger.java deleted file mode 100644 index b384b9b9d35d..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/SerializationDebugger.java +++ /dev/null @@ -1,115 +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. - */ -package org.apache.beam.runners.spark.structuredstreaming.utils; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.ObjectOutputStream; -import java.io.OutputStream; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.List; - -/** A {@code SerializationDebugger} for Spark Runner. */ -public class SerializationDebugger { - - public static void testSerialization(Object object, File to) throws IOException { - DebuggingObjectOutputStream out = new DebuggingObjectOutputStream(new FileOutputStream(to)); - try { - out.writeObject(object); - } catch (Exception e) { - throw new RuntimeException("Serialization error. Path to bad object: " + out.getStack(), e); - } - } - - private static class DebuggingObjectOutputStream extends ObjectOutputStream { - - private static final Field DEPTH_FIELD; - - static { - try { - DEPTH_FIELD = ObjectOutputStream.class.getDeclaredField("depth"); - DEPTH_FIELD.setAccessible(true); - } catch (NoSuchFieldException e) { - throw new AssertionError(e); - } - } - - final List stack = new ArrayList<>(); - - /** - * Indicates whether or not OOS has tried to write an IOException (presumably as the result of a - * serialization error) to the stream. - */ - boolean broken = false; - - DebuggingObjectOutputStream(OutputStream out) throws IOException { - super(out); - enableReplaceObject(true); - } - - /** Abuse {@code replaceObject()} as a hook to maintain our stack. */ - @Override - protected Object replaceObject(Object o) { - // ObjectOutputStream writes serialization - // exceptions to the stream. Ignore - // everything after that so we don't lose - // the path to a non-serializable object. So - // long as the user doesn't write an - // IOException as the root object, we're OK. - int currentDepth = currentDepth(); - if (o instanceof IOException && currentDepth == 0) { - broken = true; - } - if (!broken) { - truncate(currentDepth); - stack.add(o); - } - return o; - } - - private void truncate(int depth) { - while (stack.size() > depth) { - pop(); - } - } - - private Object pop() { - return stack.remove(stack.size() - 1); - } - - /** Returns a 0-based depth within the object graph of the current object being serialized. */ - private int currentDepth() { - try { - Integer oneBased = ((Integer) DEPTH_FIELD.get(this)); - return oneBased - 1; - } catch (IllegalAccessException e) { - throw new AssertionError(e); - } - } - - /** - * Returns the path to the last object serialized. If an exception occurred, this should be the - * path to the non-serializable object. - */ - List getStack() { - return stack; - } - } -} diff --git a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/package-info.java b/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/package-info.java deleted file mode 100644 index 3d7da111a9c4..000000000000 --- a/runners/spark/2/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Testing utils for spark structured streaming runner. */ -package org.apache.beam.runners.spark.structuredstreaming.utils; diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index b3cbb9f6dac6..7d820c28f83f 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -164,9 +164,7 @@ dependencies { implementation library.java.jackson_annotations implementation library.java.slf4j_api implementation library.java.joda_time - if (project.property("spark_version").startsWith("3.")) { - implementation library.java.commons_lang3 - } + implementation library.java.commons_lang3 implementation library.java.args4j implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") @@ -177,15 +175,9 @@ dependencies { provided "$component:$spark_version" } permitUnusedDeclared "org.apache.spark:spark-network-common_$spark_scala_version:$spark_version" - if (project.property("spark_scala_version").equals("2.11")) { - implementation "io.dropwizard.metrics:metrics-core:3.1.5" // version used by Spark 2.4 - compileOnly "org.scala-lang:scala-library:2.11.12" - runtimeOnly library.java.jackson_module_scala_2_11 - } else { - implementation "io.dropwizard.metrics:metrics-core:4.1.1" // version used by Spark 3.1 - compileOnly "org.scala-lang:scala-library:2.12.15" - runtimeOnly library.java.jackson_module_scala_2_12 - } + implementation "io.dropwizard.metrics:metrics-core:4.1.1" // version used by Spark 3.1 + compileOnly "org.scala-lang:scala-library:2.12.15" + runtimeOnly library.java.jackson_module_scala_2_12 // Force paranamer 2.8 to avoid issues when using Scala 2.12 runtimeOnly "com.thoughtworks.paranamer:paranamer:2.8" provided library.java.hadoop_common diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java index 4b714b655818..9f9465ccde8f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java @@ -143,12 +143,6 @@ private static JavaSparkContext createSparkContext(SparkPipelineOptions options) conf.setAppName(options.getAppName()); // register immutable collections serializers because the SDK uses them. conf.set("spark.kryo.registrator", SparkRunnerKryoRegistrator.class.getName()); - JavaSparkContext jsc = new JavaSparkContext(conf); - if (jsc.sc().version().startsWith("2")) { - LOG.warn( - "Support for Spark 2 is deprecated, this runner will be removed in a few releases.\n" - + "Spark 2 is reaching its EOL, consider migrating to Spark 3."); - } - return jsc; + return new JavaSparkContext(conf); } } 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 08b418fa3666..657ad0c42b64 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 @@ -46,7 +46,6 @@ import org.apache.beam.runners.spark.stateful.SparkGroupAlsoByWindowViaWindowSet; import org.apache.beam.runners.spark.translation.streaming.UnboundedDataset; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder; -import org.apache.beam.runners.spark.util.SparkCompat; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -67,6 +66,7 @@ import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaInputDStream; import scala.Tuple2; +import scala.collection.JavaConverters; /** Translates an unbounded portable pipeline into a Spark job. */ @SuppressWarnings({ @@ -328,7 +328,7 @@ private static void translateFlatten( } } // Unify streams into a single stream. - unifiedStreams = SparkCompat.joinStreams(context.getStreamingContext(), dStreams); + unifiedStreams = context.getStreamingContext().union(JavaConverters.asScalaBuffer(dStreams)); } context.pushDataset( 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 4cd73585e46d..7fd0f9155d3e 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 @@ -39,7 +39,6 @@ import org.apache.beam.runners.spark.metrics.MetricsContainerStepMapAccumulator; import org.apache.beam.runners.spark.util.ByteArray; import org.apache.beam.runners.spark.util.SideInputBroadcast; -import org.apache.beam.runners.spark.util.SparkCompat; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -82,6 +81,7 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.storage.StorageLevel; import org.checkerframework.checker.nullness.qual.Nullable; @@ -343,8 +343,15 @@ public void evaluate( vaCoder, windowingStrategy); + FlatMapFunction< + SparkCombineFn.WindowedAccumulator, InputT, AccumT, ?>, + WindowedValue> + flatMapFunction = + windowedAccumulator -> + sparkCombineFn.extractOutputStream(windowedAccumulator).iterator(); + JavaPairRDD> kwvs = - SparkCompat.extractOutput(accumulatePerKey, sparkCombineFn); + accumulatePerKey.flatMapValues(flatMapFunction); JavaRDD>> outRdd = kwvs.map(new TranslationUtils.FromPairFunction()) .map(new TranslationUtils.ToKVByWindowInValueFunction<>()); 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 55048f0f6fdc..2abeeb7c33ff 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 @@ -56,7 +56,6 @@ import org.apache.beam.runners.spark.translation.TranslationUtils; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder; import org.apache.beam.runners.spark.util.SideInputBroadcast; -import org.apache.beam.runners.spark.util.SparkCompat; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.testing.TestStream; @@ -95,6 +94,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.apache.spark.streaming.dstream.ConstantInputDStream; import org.checkerframework.checker.nullness.qual.Nullable; +import scala.collection.JavaConverters; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; @@ -301,7 +301,7 @@ public void evaluate(Flatten.PCollections transform, EvaluationContext contex } // start by unifying streams into a single stream. JavaDStream> unifiedStreams = - SparkCompat.joinStreams(context.getStreamingContext(), dStreams); + context.getStreamingContext().union(JavaConverters.asScalaBuffer(dStreams)); context.putDataset(transform, new UnboundedDataset<>(unifiedStreams, streamingSources)); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkCompat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkCompat.java deleted file mode 100644 index 17bafd5ff521..000000000000 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkCompat.java +++ /dev/null @@ -1,117 +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. - */ -package org.apache.beam.runners.spark.util; - -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.beam.runners.spark.translation.SparkCombineFn; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.KV; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.function.FlatMapFunction; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; - -/** A set of functions to provide API compatibility between Spark 2 and Spark 3. */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class SparkCompat { - private SparkCompat() {} - - /** - * Union of dStreams in the given StreamingContext. - * - *

This is required because the API to join (union) DStreams is different among Spark versions. - * See https://issues.apache.org/jira/browse/SPARK-25737 - */ - public static JavaDStream> joinStreams( - JavaStreamingContext streamingContext, List>> dStreams) { - try { - if (streamingContext.sparkContext().version().startsWith("3")) { - // This invokes by reflection the equivalent of: - // return streamingContext.union( - // JavaConverters.asScalaIteratorConverter(dStreams.iterator()).asScala().toSeq()); - Method method = streamingContext.getClass().getDeclaredMethod("union", JavaDStream[].class); - Object result = - method.invoke(streamingContext, new Object[] {dStreams.toArray(new JavaDStream[0])}); - return (JavaDStream>) result; - } - // This invokes by reflection the equivalent of: - // return streamingContext.union(dStreams.remove(0), dStreams); - Method method = - streamingContext.getClass().getDeclaredMethod("union", JavaDStream.class, List.class); - Object result = method.invoke(streamingContext, dStreams.remove(0), dStreams); - return (JavaDStream>) result; - } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { - throw new RuntimeException("Error invoking Spark union", e); - } - } - - /** - * Extracts the output for a given collection of WindowedAccumulators. - * - *

This is required because the API of JavaPairRDD.flatMapValues is different among Spark - * versions. See https://issues.apache.org/jira/browse/SPARK-19287 - */ - public static JavaPairRDD> extractOutput( - JavaPairRDD, InputT, AccumT, ?>> - accumulatePerKey, - SparkCombineFn, InputT, AccumT, OutputT> sparkCombineFn) { - try { - if (accumulatePerKey.context().version().startsWith("3")) { - FlatMapFunction< - SparkCombineFn.WindowedAccumulator, InputT, AccumT, ?>, - WindowedValue> - flatMapFunction = - (FlatMapFunction< - SparkCombineFn.WindowedAccumulator, InputT, AccumT, ?>, - WindowedValue>) - windowedAccumulator -> - sparkCombineFn.extractOutputStream(windowedAccumulator).iterator(); - // This invokes by reflection the equivalent of: - // return accumulatePerKey.flatMapValues(flatMapFunction); - Method method = - accumulatePerKey.getClass().getDeclaredMethod("flatMapValues", FlatMapFunction.class); - Object result = method.invoke(accumulatePerKey, flatMapFunction); - return (JavaPairRDD>) result; - } - - Function< - SparkCombineFn.WindowedAccumulator, InputT, AccumT, ?>, - Iterable>> - flatMapFunction = - windowedAccumulator -> - sparkCombineFn - .extractOutputStream(windowedAccumulator) - .collect(Collectors.toList()); - // This invokes by reflection the equivalent of: - // return accumulatePerKey.flatMapValues(flatMapFunction); - Method method = - accumulatePerKey.getClass().getDeclaredMethod("flatMapValues", Function.class); - Object result = method.invoke(accumulatePerKey, flatMapFunction); - return (JavaPairRDD>) result; - } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { - throw new RuntimeException("Error invoking Spark flatMapValues", e); - } - } -} diff --git a/sdks/java/testing/watermarks/build.gradle b/sdks/java/testing/watermarks/build.gradle index 0a1b137c5b75..c6c2a50279cc 100644 --- a/sdks/java/testing/watermarks/build.gradle +++ b/sdks/java/testing/watermarks/build.gradle @@ -39,7 +39,6 @@ def runnerDependency = (project.hasProperty(runnerProperty) ? project.getProperty(runnerProperty) : ":runners:direct-java") -def shouldProvideSpark = ":runners:spark".equals(runnerDependency) def isDataflowRunner = ":runners:google-cloud-dataflow-java".equals(runnerDependency) def runnerConfiguration = ":runners:direct-java".equals(runnerDependency) ? "shadow" : null @@ -74,24 +73,6 @@ dependencies { gradleRun project(project.path) gradleRun project(path: runnerDependency, configuration: runnerConfiguration) - - // The Spark runner requires the user to provide a Spark dependency. For self-contained - // runs with the Spark runner, we can provide such a dependency. This is deliberately phrased - // to not hardcode any runner other than :runners:direct-java - if (shouldProvideSpark) { - gradleRun library.java.spark_streaming - gradleRun library.java.spark_core, { - exclude group:"org.slf4j", module:"jul-to-slf4j" - } - gradleRun library.java.spark_sql - } -} - -if (shouldProvideSpark) { - configurations.gradleRun { - // Using Spark runner causes a StackOverflowError if slf4j-jdk14 is on the classpath - exclude group: "org.slf4j", module: "slf4j-jdk14" - } } task run(type: JavaExec) { diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 1bcce558384a..37bdb106038e 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -1535,9 +1535,8 @@ def _add_argparse_args(cls, parser): parser.add_argument( '--spark_version', default='3', - choices=['3', '2'], - help='Spark major version to use. ' - 'Note, Spark 2 support is deprecated') + choices=['3'], + help='Spark major version to use.') class TestOptions(PipelineOptions): diff --git a/sdks/python/apache_beam/runners/portability/spark_runner.py b/sdks/python/apache_beam/runners/portability/spark_runner.py index b4c46c0dac06..c06d1b9c1a2c 100644 --- a/sdks/python/apache_beam/runners/portability/spark_runner.py +++ b/sdks/python/apache_beam/runners/portability/spark_runner.py @@ -93,9 +93,7 @@ def path_to_jar(self): return self._jar else: if self._spark_version == '2': - return self.path_to_beam_jar( - ':runners:spark:2:job-server:shadowJar', - artifact_id='beam-runners-spark-job-server') + raise ValueError('Support for Spark 2 was dropped.') return self.path_to_beam_jar(':runners:spark:3:job-server:shadowJar') def java_arguments( diff --git a/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py index 97fa6b629cee..f754b4c330ad 100644 --- a/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py +++ b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py @@ -74,9 +74,7 @@ def executable_jar(self): url = self._executable_jar else: if self._spark_version == '2': - url = job_server.JavaJarJobServer.path_to_beam_jar( - ':runners:spark:2:job-server:shadowJar', - artifact_id='beam-runners-spark-job-server') + raise ValueError('Support for Spark 2 was dropped.') else: url = job_server.JavaJarJobServer.path_to_beam_jar( ':runners:spark:3:job-server:shadowJar') diff --git a/settings.gradle.kts b/settings.gradle.kts index 4455981a3fd7..feb1c3609dad 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -108,9 +108,6 @@ include(":runners:java-job-service") include(":runners:jet") include(":runners:local-java") include(":runners:portability:java") -include(":runners:spark:2") -include(":runners:spark:2:job-server") -include(":runners:spark:2:job-server:container") include(":runners:spark:3") include(":runners:spark:3:job-server") include(":runners:spark:3:job-server:container") diff --git a/website/www/site/content/en/documentation/runners/spark.md b/website/www/site/content/en/documentation/runners/spark.md index 15cf6cf5ac7c..3b166c077e0c 100644 --- a/website/www/site/content/en/documentation/runners/spark.md +++ b/website/www/site/content/en/documentation/runners/spark.md @@ -68,7 +68,7 @@ the portable Runner. For more information on portability, please visit the ## Spark Runner prerequisites and setup The Spark runner currently supports Spark's 3.1.x branch. -> **Note:** Support for Spark 2.4.x is deprecated and will be dropped with the release of Beam 2.44.0 (or soon after). +> **Note:** Support for Spark 2.4.x was deprecated as of Beam 2.41.0 and finally dropped with the release of Beam 2.46.0. {{< paragraph class="language-java" >}} You can add a dependency on the latest version of the Spark runner by adding to your pom.xml the following: