You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by mm...@apache.org on 2023/02/08 13:03:18 UTC

[beam] branch master updated: [Spark runner] Removal of Spark 2 runner support (closes #25259) (#25263)

This is an automated email from the ASF dual-hosted git repository.

mmack pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 063eea63345 [Spark runner] Removal of Spark 2 runner support (closes #25259) (#25263)
063eea63345 is described below

commit 063eea6334504b42ab4a9d852af99d3694e27bc1
Author: Moritz Mack <mm...@talend.com>
AuthorDate: Wed Feb 8 14:03:10 2023 +0100

    [Spark runner] Removal of Spark 2 runner support (closes #25259) (#25263)
---
 ...ommit_CrossLanguageValidatesRunner_Spark.groovy |  51 ----
 .../job_PostCommit_Java_Examples_Spark.groovy      |   1 -
 ...PortableValidatesRunner_Spark2_Streaming.groovy |  43 ----
 ...Java_PortableValidatesRunner_Spark_Batch.groovy |   2 -
 ...ob_PostCommit_Java_ValidatesRunner_Spark.groovy |   1 -
 ...ValidatesRunner_SparkStructuredStreaming.groovy |   1 -
 CHANGES.md                                         |   1 +
 build.gradle.kts                                   |   5 -
 .../org/apache/beam/gradle/BeamModulePlugin.groovy |   3 -
 runners/spark/2/build.gradle                       |  33 ---
 runners/spark/2/job-server/build.gradle            |  31 ---
 runners/spark/2/job-server/container/build.gradle  |  27 ---
 .../spark/structuredstreaming/Constants.java       |  25 --
 .../SparkStructuredStreamingPipelineOptions.java   |  42 ----
 .../SparkStructuredStreamingPipelineResult.java    | 134 ----------
 .../SparkStructuredStreamingRunner.java            | 228 -----------------
 .../SparkStructuredStreamingRunnerRegistrar.java   |  54 -----
 .../aggregators/AggregatorsAccumulator.java        |  73 ------
 .../aggregators/NamedAggregators.java              | 113 ---------
 .../aggregators/NamedAggregatorsAccumulator.java   |  63 -----
 .../aggregators/package-info.java                  |  20 --
 .../structuredstreaming/examples/WordCount.java    | 132 ----------
 .../metrics/AggregatorMetric.java                  |  74 ------
 .../metrics/AggregatorMetricSource.java            |  49 ----
 .../structuredstreaming/metrics/BeamMetricSet.java |  60 -----
 .../metrics/CompositeSource.java                   |  45 ----
 .../metrics/MetricsAccumulator.java                |  76 ------
 .../MetricsContainerStepMapAccumulator.java        |  65 -----
 .../metrics/SparkBeamMetric.java                   | 107 --------
 .../metrics/SparkBeamMetricSource.java             |  48 ----
 .../metrics/SparkMetricsContainerStepMap.java      |  43 ----
 .../metrics/WithMetricsSupport.java                |  91 -------
 .../structuredstreaming/metrics/package-info.java  |  20 --
 .../metrics/sink/CodahaleCsvSink.java              |  86 -------
 .../metrics/sink/CodahaleGraphiteSink.java         |  89 -------
 .../metrics/sink/package-info.java                 |  20 --
 .../spark/structuredstreaming/package-info.java    |  20 --
 .../translation/AbstractTranslationContext.java    | 235 ------------------
 .../translation/PipelineTranslator.java            | 202 ---------------
 .../translation/SparkSessionFactory.java           |  71 ------
 .../translation/SparkTransformOverrides.java       |  56 -----
 .../translation/TransformTranslator.java           |  31 ---
 .../translation/TranslationContext.java            |  34 ---
 .../translation/batch/AggregatorCombiner.java      | 270 ---------------------
 .../batch/CombinePerKeyTranslatorBatch.java        | 114 ---------
 .../CreatePCollectionViewTranslatorBatch.java      |  60 -----
 .../translation/batch/DatasetSourceBatch.java      | 160 ------------
 .../translation/batch/DoFnFunction.java            | 164 -------------
 .../translation/batch/DoFnRunnerWithMetrics.java   | 104 --------
 .../translation/batch/FlattenTranslatorBatch.java  |  65 -----
 .../batch/GroupByKeyTranslatorBatch.java           |  90 -------
 .../translation/batch/ImpulseTranslatorBatch.java  |  49 ----
 .../translation/batch/ParDoTranslatorBatch.java    | 257 --------------------
 .../translation/batch/PipelineTranslatorBatch.java | 101 --------
 .../translation/batch/ProcessContext.java          | 138 -----------
 .../batch/ReadSourceTranslatorBatch.java           |  88 -------
 .../batch/ReshuffleTranslatorBatch.java            |  30 ---
 .../batch/WindowAssignTranslatorBatch.java         |  61 -----
 .../GroupAlsoByWindowViaOutputBufferFn.java        | 173 -------------
 .../batch/functions/NoOpStepContext.java           |  36 ---
 .../batch/functions/SparkSideInputReader.java      | 186 --------------
 .../translation/batch/functions/package-info.java  |  20 --
 .../translation/batch/package-info.java            |  20 --
 .../translation/helpers/CoderHelpers.java          |  63 -----
 .../translation/helpers/EncoderFactory.java        |  49 ----
 .../translation/helpers/EncoderHelpers.java        |  71 ------
 .../translation/helpers/KVHelpers.java             |  31 ---
 .../translation/helpers/MultiOutputCoder.java      |  84 -------
 .../translation/helpers/RowHelpers.java            |  75 ------
 .../translation/helpers/SchemaHelpers.java         |  39 ---
 .../translation/helpers/SideInputBroadcast.java    |  49 ----
 .../translation/helpers/WindowingHelpers.java      |  82 -------
 .../translation/helpers/package-info.java          |  20 --
 .../translation/package-info.java                  |  20 --
 .../streaming/DatasetSourceStreaming.java          | 260 --------------------
 .../streaming/PipelineTranslatorStreaming.java     |  93 -------
 .../streaming/ReadSourceTranslatorStreaming.java   |  87 -------
 .../translation/streaming/package-info.java        |  20 --
 .../translation/utils/CachedSideInputReader.java   |  93 -------
 .../translation/utils/ScalaInterop.java            |  40 ---
 .../translation/utils/SideInputStorage.java        | 107 --------
 .../translation/utils/package-info.java            |  20 --
 .../structuredstreaming/SparkSessionRule.java      |  88 -------
 ...parkStructuredStreamingRunnerRegistrarTest.java |  70 ------
 .../StructuredStreamingPipelineStateTest.java      | 225 -----------------
 .../aggregators/metrics/sink/InMemoryMetrics.java  |  80 ------
 .../metrics/sink/InMemoryMetricsSinkRule.java      |  28 ---
 .../metrics/sink/SparkMetricsSinkTest.java         |  73 ------
 .../metrics/SparkBeamMetricTest.java               |  59 -----
 .../translation/batch/CombineTest.java             | 186 --------------
 .../translation/batch/ComplexSourceTest.java       |  86 -------
 .../translation/batch/FlattenTest.java             |  59 -----
 .../translation/batch/GroupByKeyTest.java          | 124 ----------
 .../translation/batch/ParDoTest.java               | 153 ------------
 .../translation/batch/SimpleSourceTest.java        |  53 ----
 .../translation/batch/WindowAssignTest.java        |  69 ------
 .../translation/helpers/EncoderHelpersTest.java    |  98 --------
 .../translation/streaming/SimpleSourceTest.java    |  57 -----
 .../utils/SerializationDebugger.java               | 115 ---------
 .../structuredstreaming/utils/package-info.java    |  20 --
 runners/spark/spark_runner.gradle                  |  16 +-
 .../spark/translation/SparkContextFactory.java     |   8 +-
 .../SparkStreamingPortablePipelineTranslator.java  |   4 +-
 .../spark/translation/TransformTranslator.java     |  11 +-
 .../streaming/StreamingTransformTranslator.java    |   4 +-
 .../beam/runners/spark/util/SparkCompat.java       | 117 ---------
 sdks/java/testing/watermarks/build.gradle          |  19 --
 .../python/apache_beam/options/pipeline_options.py |   5 +-
 .../runners/portability/spark_runner.py            |   4 +-
 .../portability/spark_uber_jar_job_server.py       |   4 +-
 settings.gradle.kts                                |   3 -
 .../site/content/en/documentation/runners/spark.md |   2 +-
 112 files changed, 24 insertions(+), 7910 deletions(-)

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 2cf4edb88f1..00000000000
--- 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 10dff586028..f4af7dea6c0 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 14e1ff3d607..00000000000
--- 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 ce7340f64da..ba08cc2e440 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 2509e6884d7..0cb6bd2d371 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 7253140faa5..31d5d5cf07b 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 bd7d0a67e8d..297ec87bfcf 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -68,6 +68,7 @@
 
 ## Breaking Changes
 
+* The deprecated SparkRunner for Spark 2 (see [2.41.0](#2410---2022-08-23)) was removed ([#25263](https://github.com/apache/beam/pull/25263)).
 * Python's BatchElements performs more aggressive batching in some cases,
   capping at 10 second rather than 1 second batches by default and excluding
   fixed cost in this computation to better handle cases where the fixed cost
diff --git a/build.gradle.kts b/build.gradle.kts
index b70d8591bf8..de7bf456ec1 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 414b3ccdc55..9674cc807b4 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<Project> {
     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<Project> {
         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 e943b01e347..00000000000
--- 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 adb9121d1d0..00000000000
--- 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 10cacb109a2..00000000000
--- 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 08c187ce6c6..00000000000
--- 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 3371a403b2c..00000000000
--- 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 1392ae8f0c7..00000000000
--- 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 98b068a9cce..00000000000
--- 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:
- *
- * <p>{@code Pipeline p = [logic for pipeline creation] SparkStructuredStreamingPipelineResult
- * result = (SparkStructuredStreamingPipelineResult) p.run(); }
- *
- * <p>To create a pipeline runner to run against a different spark cluster, with a custom master url
- * we would do the following:
- *
- * <p>{@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<SparkStructuredStreamingPipelineResult> {
-
-  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 fb759da960c..00000000000
--- 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}.
- *
- * <p>{@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<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.of(SparkStructuredStreamingRunner.class);
-    }
-  }
-
-  /** Registers the {@link SparkStructuredStreamingPipelineOptions}. */
-  @AutoService(PipelineOptionsRegistrar.class)
-  public static class Options implements PipelineOptionsRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineOptions>> 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 bdba5509a9b..00000000000
--- 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 <a
- *     href="https://spark.apache.org/docs/latest/api/java/org/apache/spark/util/AccumulatorV2.html">accumulatorsV2</a>
- */
-@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 de53458b314..00000000000
--- 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<String, State<?, ?, ?>> 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 <T> Type to be returned.
-   * @return the value of the aggregator associated with the specified name, or <code>null</code> if
-   *     the specified aggregator could not be found.
-   */
-  public <T> T getValue(String name, Class<T> 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 <b>rendered </b>values */
-  public Map<String, ?> 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<String, State<?, ?, ?>> 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 <InputT, InterT, OutputT> State<InputT, InterT, OutputT> merge(
-      State<?, ?, ?> s1, State<?, ?, ?> s2) {
-    return ((State<InputT, InterT, OutputT>) s1).merge((State<InputT, InterT, OutputT>) s2);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    for (Map.Entry<String, State<?, ?, ?>> e : mNamedAggregators.entrySet()) {
-      sb.append(e.getKey()).append(": ").append(e.getValue().render()).append(" ");
-    }
-    return sb.toString();
-  }
-
-  /**
-   * @param <InputT> Input data type
-   * @param <InterT> Intermediate data type (useful for averages)
-   * @param <OutputT> Output data type
-   */
-  public interface State<InputT, InterT, OutputT> extends Serializable {
-
-    /** @param element new element to update state */
-    void update(InputT element);
-
-    State<InputT, InterT, OutputT> merge(State<InputT, InterT, OutputT> other);
-
-    InterT current();
-
-    OutputT render();
-
-    Combine.CombineFn<InputT, InterT, OutputT> 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 992e63f47b8..00000000000
--- 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<NamedAggregators, NamedAggregators> {
-  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<NamedAggregators, NamedAggregators> 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 11a87eed897..00000000000
--- 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 dca43581f66..00000000000
--- 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<String, String> {
-    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<KV<String, Long>, String> {
-    @Override
-    public String apply(KV<String, Long> input) {
-      return input.getKey() + ": " + input.getValue();
-    }
-  }
-
-  /**
-   * A PTransform that converts a PCollection containing lines of text into a PCollection of
-   * formatted word counts.
-   *
-   * <p>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<String>, PCollection<KV<String, Long>>> {
-    @Override
-    public PCollection<KV<String, Long>> expand(PCollection<String> lines) {
-
-      // Convert lines of text into individual words.
-      PCollection<String> 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}.
-   *
-   * <p>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.
-   *
-   * <p>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 74bea7f5255..00000000000
--- 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<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
-    Map<String, Gauge<Double>> metrics = new HashMap<>();
-    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
-      String name = prefix + "." + entry.getKey();
-      Object rawValue = entry.getValue();
-      if (rawValue != null) {
-        try {
-          Gauge<Double> 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<Double> 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 406dba365cc..00000000000
--- 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 7095036f28a..00000000000
--- 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}.
- *
- * <p>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<Map<String, Gauge<Double>>> {
-
-  @Override
-  public final Map<String, Gauge<Double>> getValue() {
-    return getValue("", MetricFilter.ALL);
-  }
-
-  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
-
-  protected Gauge<Double> staticGauge(Number number) {
-    return new ConstantGauge(number.doubleValue());
-  }
-
-  private static class ConstantGauge implements Gauge<Double> {
-    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 8de06f76064..00000000000
--- 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 b319ae36487..00000000000
--- 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 <a
- *     href="https://spark.apache.org/docs/2.4.4/streaming-programming-guide.html#accumulators-broadcast-variables-and-checkpoints">accumulatorsV2</a>
- */
-@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 2d2a4ea1754..00000000000
--- 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<MetricsContainerStepMap, MetricsContainerStepMap> {
-  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<MetricsContainerStepMap, MetricsContainerStepMap> 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 0cecae4a25b..00000000000
--- 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<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    Map<String, Gauge<Double>> metrics = new HashMap<>();
-    MetricQueryResults allMetrics = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
-      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
-    }
-    for (MetricResult<DistributionResult> 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<GaugeResult> 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<String, Gauge<Double>> metrics, MetricFilter filter, String name, Number value) {
-    Gauge<Double> 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 665dbc479c4..00000000000
--- 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 533dceb42e2..00000000000
--- 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 d48a229996f..00000000000
--- 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}.
- *
- * <p>{@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<String, Timer> getTimers(final MetricFilter filter) {
-    return internalMetricRegistry.getTimers(filter);
-  }
-
-  @Override
-  public SortedMap<String, Meter> getMeters(final MetricFilter filter) {
-    return internalMetricRegistry.getMeters(filter);
-  }
-
-  @Override
-  public SortedMap<String, Histogram> getHistograms(final MetricFilter filter) {
-    return internalMetricRegistry.getHistograms(filter);
-  }
-
-  @Override
-  public SortedMap<String, Counter> getCounters(final MetricFilter filter) {
-    return internalMetricRegistry.getCounters(filter);
-  }
-
-  @Override
-  public SortedMap<String, Gauge> getGauges(final MetricFilter filter) {
-    ImmutableSortedMap.Builder<String, Gauge> builder =
-        new ImmutableSortedMap.Builder<>(Ordering.from(String.CASE_INSENSITIVE_ORDER));
-
-    Map<String, Gauge> gauges =
-        internalMetricRegistry.getGauges(
-            (n, m) -> filter.matches(n, m) || m instanceof BeamMetricSet);
-
-    for (Map.Entry<String, Gauge> 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 16a1a956e8e..00000000000
--- 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 c8f9139a2eb..00000000000
--- 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 <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
- * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
- *
- * <p>The sink is configured using Spark configuration parameters, for example:
- *
- * <pre>{@code
- * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleCsvSink"
- * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
- * "spark.metrics.conf.*.sink.csv.period"=10
- * "spark.metrics.conf.*.sink.csv.unit"=seconds
- * }</pre>
- */
-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 5640c965740..00000000000
--- 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 <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
- * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
- *
- * <p>The sink is configured using Spark configuration parameters, for example:
- *
- * <pre>{@code
- * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
- * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
- * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
- * "spark.metrics.conf.*.sink.graphite.period"=10
- * "spark.metrics.conf.*.sink.graphite.unit"=seconds
- * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
- * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
- * }</pre>
- */
-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 427e5441c57..00000000000
--- 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 aefeb282f8f..00000000000
--- 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 aed287ba6d5..00000000000
--- 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<PValue, Dataset<?>> datasets;
-  /** datasets that are not used as input to other datasets (leaves of the DAG). */
-  private final Set<Dataset<?>> 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<PCollectionView<?>, 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 <T> Dataset<T> emptyDataset() {
-    return (Dataset<T>) sparkSession.emptyDataset(EncoderHelpers.fromBeamCoder(VoidCoder.of()));
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T> Dataset<WindowedValue<T>> 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<WindowedValue<T>>) 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<WindowedValue<?>> dataset) {
-    if (!datasets.containsKey(value)) {
-      datasets.put(value, dataset);
-      leaves.add(dataset);
-    }
-  }
-
-  public <T> void putDataset(PValue value, Dataset<WindowedValue<T>> dataset) {
-    if (!datasets.containsKey(value)) {
-      datasets.put(value, dataset);
-      leaves.add(dataset);
-    }
-  }
-
-  public <ViewT, ElemT> void setSideInputDataset(
-      PCollectionView<ViewT> value, Dataset<WindowedValue<ElemT>> set) {
-    if (!broadcastDataSets.containsKey(value)) {
-      broadcastDataSets.put(value, set);
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T> Dataset<T> getSideInputDataSet(PCollectionView<?> value) {
-    return (Dataset<T>) broadcastDataSets.get(value);
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //  PCollections methods
-  // --------------------------------------------------------------------------------------------
-  public PValue getInput() {
-    return Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform));
-  }
-
-  public Map<TupleTag<?>, PCollection<?>> getInputs() {
-    return currentTransform.getInputs();
-  }
-
-  public PValue getOutput() {
-    return Iterables.getOnlyElement(currentTransform.getOutputs().values());
-  }
-
-  public Map<TupleTag<?>, PCollection<?>> getOutputs() {
-    return currentTransform.getOutputs();
-  }
-
-  @SuppressWarnings("unchecked")
-  public Map<TupleTag<?>, 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<WindowedValue> dataset) {
-    // cannot use dataset.show because dataset schema is binary so it will print binary
-    // code.
-    List<WindowedValue> windowedValues = dataset.collectAsList();
-    for (WindowedValue windowedValue : windowedValues) {
-      LOG.debug("**** dataset content {} ****", windowedValue.toString());
-    }
-  }
-
-  private static class NoOpForeachWriter<T> extends ForeachWriter<T> {
-
-    @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 0f851d9588d..00000000000
--- 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 <T extends PTransform<?, ?>> 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<T> typedTransformTranslator = (TransformTranslator<T>) 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 d8430f5f130..00000000000
--- 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<String> 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 996f60cb747..00000000000
--- 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<PTransformOverride> getDefaultOverrides(boolean streaming) {
-    ImmutableList.Builder<PTransformOverride> 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 61580aed219..00000000000
--- 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<TransformT extends PTransform> 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 ece050e5cc9..00000000000
--- 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 d0f46ea807c..00000000000
--- 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<WindowedValue<AccumT>> because an {@code InputT} can be in multiple windows. So, when accumulating {@code InputT} values, we create one accumulator per input window.
- * */
-class AggregatorCombiner<K, InputT, AccumT, OutputT, W extends BoundedWindow>
-    extends Aggregator<
-        WindowedValue<KV<K, InputT>>,
-        Iterable<WindowedValue<AccumT>>,
-        Iterable<WindowedValue<OutputT>>> {
-
-  private final Combine.CombineFn<InputT, AccumT, OutputT> combineFn;
-  private WindowingStrategy<InputT, W> windowingStrategy;
-  private TimestampCombiner timestampCombiner;
-  private Coder<AccumT> accumulatorCoder;
-  private IterableCoder<WindowedValue<AccumT>> bufferEncoder;
-  private IterableCoder<WindowedValue<OutputT>> outputCoder;
-
-  public AggregatorCombiner(
-      Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Coder<AccumT> accumulatorCoder,
-      Coder<OutputT> outputCoder) {
-    this.combineFn = combineFn;
-    this.windowingStrategy = (WindowingStrategy<InputT, W>) 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<WindowedValue<AccumT>> zero() {
-    return new ArrayList<>();
-  }
-
-  private Iterable<WindowedValue<AccumT>> createAccumulator(WindowedValue<KV<K, InputT>> 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<WindowedValue<AccumT>> reduce(
-      Iterable<WindowedValue<AccumT>> accumulators, WindowedValue<KV<K, InputT>> inputWv) {
-    return merge(accumulators, createAccumulator(inputWv));
-  }
-
-  @Override
-  public Iterable<WindowedValue<AccumT>> merge(
-      Iterable<WindowedValue<AccumT>> accumulators1,
-      Iterable<WindowedValue<AccumT>> accumulators2) {
-
-    // merge the windows of all the accumulators
-    Iterable<WindowedValue<AccumT>> accumulators = Iterables.concat(accumulators1, accumulators2);
-    Set<W> accumulatorsWindows = collectAccumulatorsWindows(accumulators);
-    Map<W, W> 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<W, List<Tuple2<AccumT, Instant>>> mergedWindowToAccumulators = new HashMap<>();
-    for (WindowedValue<AccumT> 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<AccumT, Instant> 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<WindowedValue<AccumT>> result = new ArrayList<>();
-    for (Map.Entry<W, List<Tuple2<AccumT, Instant>>> entry :
-        mergedWindowToAccumulators.entrySet()) {
-      W mergedWindow = entry.getKey();
-      List<Tuple2<AccumT, Instant>> accumsAndInstantsForMergedWindow = entry.getValue();
-
-      // we need to create the first accumulator because combineFn.mergerAccumulators can modify the
-      // first accumulator
-      AccumT first = combineFn.createAccumulator();
-      Iterable<AccumT> 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<WindowedValue<OutputT>> finish(Iterable<WindowedValue<AccumT>> reduction) {
-    List<WindowedValue<OutputT>> result = new ArrayList<>();
-    for (WindowedValue<AccumT> windowedValue : reduction) {
-      result.add(windowedValue.withValue(combineFn.extractOutput(windowedValue.getValue())));
-    }
-    return result;
-  }
-
-  @Override
-  public Encoder<Iterable<WindowedValue<AccumT>>> bufferEncoder() {
-    return EncoderHelpers.fromBeamCoder(bufferEncoder);
-  }
-
-  @Override
-  public Encoder<Iterable<WindowedValue<OutputT>>> outputEncoder() {
-    return EncoderHelpers.fromBeamCoder(outputCoder);
-  }
-
-  private Set<W> collectAccumulatorsWindows(Iterable<WindowedValue<AccumT>> accumulators) {
-    Set<W> 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<W, W> mergeWindows(WindowingStrategy<InputT, W> windowingStrategy, Set<W> windows)
-      throws Exception {
-    WindowFn<InputT, W> windowFn = windowingStrategy.getWindowFn();
-
-    if (!windowingStrategy.needsMerge()) {
-      // Return an empty map, indicating that every window is not merged.
-      return Collections.emptyMap();
-    }
-
-    Map<W, W> windowToMergeResult = new HashMap<>();
-    windowFn.mergeWindows(new MergeContextImpl(windowFn, windows, windowToMergeResult));
-    return windowToMergeResult;
-  }
-
-  private class MergeContextImpl extends WindowFn<InputT, W>.MergeContext {
-
-    private Set<W> windows;
-    private Map<W, W> windowToMergeResult;
-
-    MergeContextImpl(WindowFn<InputT, W> windowFn, Set<W> windows, Map<W, W> windowToMergeResult) {
-      windowFn.super();
-      this.windows = windows;
-      this.windowToMergeResult = windowToMergeResult;
-    }
-
-    @Override
-    public Collection<W> windows() {
-      return windows;
-    }
-
-    @Override
-    public void merge(Collection<W> 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 2b0cf8be995..00000000000
--- 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<K, InputT, AccumT, OutputT>
-    implements TransformTranslator<
-        PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, OutputT>>>> {
-
-  @Override
-  public void translateTransform(
-      PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, OutputT>>> transform,
-      AbstractTranslationContext context) {
-
-    Combine.PerKey combineTransform = (Combine.PerKey) transform;
-    @SuppressWarnings("unchecked")
-    final PCollection<KV<K, InputT>> input = (PCollection<KV<K, InputT>>) context.getInput();
-    @SuppressWarnings("unchecked")
-    final PCollection<KV<K, OutputT>> output = (PCollection<KV<K, OutputT>>) context.getOutput();
-    @SuppressWarnings("unchecked")
-    final Combine.CombineFn<InputT, AccumT, OutputT> combineFn =
-        (Combine.CombineFn<InputT, AccumT, OutputT>) combineTransform.getFn();
-    WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-
-    Dataset<WindowedValue<KV<K, InputT>>> inputDataset = context.getDataset(input);
-
-    KvCoder<K, InputT> inputCoder = (KvCoder<K, InputT>) input.getCoder();
-    Coder<K> keyCoder = inputCoder.getKeyCoder();
-    KvCoder<K, OutputT> outputKVCoder = (KvCoder<K, OutputT>) output.getCoder();
-    Coder<OutputT> outputCoder = outputKVCoder.getValueCoder();
-
-    KeyValueGroupedDataset<K, WindowedValue<KV<K, InputT>>> groupedDataset =
-        inputDataset.groupByKey(KVHelpers.extractKey(), EncoderHelpers.fromBeamCoder(keyCoder));
-
-    Coder<AccumT> accumulatorCoder = null;
-    try {
-      accumulatorCoder =
-          combineFn.getAccumulatorCoder(
-              input.getPipeline().getCoderRegistry(), inputCoder.getValueCoder());
-    } catch (CannotProvideCoderException e) {
-      throw new RuntimeException(e);
-    }
-
-    Dataset<Tuple2<K, Iterable<WindowedValue<OutputT>>>> combinedDataset =
-        groupedDataset.agg(
-            new AggregatorCombiner<K, InputT, AccumT, OutputT, BoundedWindow>(
-                    combineFn, windowingStrategy, accumulatorCoder, outputCoder)
-                .toColumn());
-
-    // expand the list into separate elements and put the key back into the elements
-    WindowedValue.WindowedValueCoder<KV<K, OutputT>> wvCoder =
-        WindowedValue.FullWindowedValueCoder.of(
-            outputKVCoder, input.getWindowingStrategy().getWindowFn().windowCoder());
-    Dataset<WindowedValue<KV<K, OutputT>>> outputDataset =
-        combinedDataset.flatMap(
-            (FlatMapFunction<
-                    Tuple2<K, Iterable<WindowedValue<OutputT>>>, WindowedValue<KV<K, OutputT>>>)
-                tuple2 -> {
-                  K key = tuple2._1();
-                  Iterable<WindowedValue<OutputT>> windowedValues = tuple2._2();
-                  List<WindowedValue<KV<K, OutputT>>> result = new ArrayList<>();
-                  for (WindowedValue<OutputT> windowedValue : windowedValues) {
-                    KV<K, OutputT> 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 ae1eeced328..00000000000
--- 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<ElemT, ViewT>
-    implements TransformTranslator<PTransform<PCollection<ElemT>, PCollection<ElemT>>> {
-
-  @Override
-  public void translateTransform(
-      PTransform<PCollection<ElemT>, PCollection<ElemT>> transform,
-      AbstractTranslationContext context) {
-
-    Dataset<WindowedValue<ElemT>> inputDataSet = context.getDataset(context.getInput());
-
-    @SuppressWarnings("unchecked")
-    AppliedPTransform<
-            PCollection<ElemT>,
-            PCollection<ElemT>,
-            PTransform<PCollection<ElemT>, PCollection<ElemT>>>
-        application =
-            (AppliedPTransform<
-                    PCollection<ElemT>,
-                    PCollection<ElemT>,
-                    PTransform<PCollection<ElemT>, PCollection<ElemT>>>)
-                context.getCurrentTransform();
-    PCollectionView<ViewT> 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 766d143d6d7..00000000000
--- 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<T> implements DataSourceReader, Serializable {
-
-    private int numPartitions;
-    private BoundedSource<T> 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<InputPartition<InternalRow>> planInputPartitions() {
-      PipelineOptions options = serializablePipelineOptions.get();
-      List<InputPartition<InternalRow>> result = new ArrayList<>();
-      long desiredSizeBytes;
-      try {
-        desiredSizeBytes = source.getEstimatedSizeBytes(options) / numPartitions;
-        List<? extends BoundedSource<T>> splits = source.split(desiredSizeBytes, options);
-        for (BoundedSource<T> split : splits) {
-          result.add(
-              (InputPartition<InternalRow>)
-                  () -> 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<T> implements InputPartitionReader<InternalRow> {
-    private boolean started;
-    private boolean closed;
-    private final BoundedSource<T> source;
-    private BoundedReader<T> reader;
-
-    DatasetPartitionReader(
-        BoundedSource<T> 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<T> 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 42a809fdd97..00000000000
--- 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}.
- *
- * <p>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<InputT, OutputT>
-    implements MapPartitionsFunction<WindowedValue<InputT>, Tuple2<TupleTag<?>, WindowedValue<?>>> {
-
-  private final MetricsContainerStepMapAccumulator metricsAccum;
-  private final String stepName;
-  private final DoFn<InputT, OutputT> doFn;
-  private transient boolean wasSetupCalled;
-  private final WindowingStrategy<?, ?> windowingStrategy;
-  private final Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs;
-  private final SerializablePipelineOptions serializableOptions;
-  private final List<TupleTag<?>> additionalOutputTags;
-  private final TupleTag<OutputT> mainOutputTag;
-  private final Coder<InputT> inputCoder;
-  private final Map<TupleTag<?>, Coder<?>> outputCoderMap;
-  private final SideInputBroadcast broadcastStateData;
-  private DoFnSchemaInformation doFnSchemaInformation;
-  private Map<String, PCollectionView<?>> sideInputMapping;
-
-  public DoFnFunction(
-      MetricsContainerStepMapAccumulator metricsAccum,
-      String stepName,
-      DoFn<InputT, OutputT> doFn,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs,
-      SerializablePipelineOptions serializableOptions,
-      List<TupleTag<?>> additionalOutputTags,
-      TupleTag<OutputT> mainOutputTag,
-      Coder<InputT> inputCoder,
-      Map<TupleTag<?>, Coder<?>> outputCoderMap,
-      SideInputBroadcast broadcastStateData,
-      DoFnSchemaInformation doFnSchemaInformation,
-      Map<String, PCollectionView<?>> 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<Tuple2<TupleTag<?>, WindowedValue<?>>> call(Iterator<WindowedValue<InputT>> iter)
-      throws Exception {
-    if (!wasSetupCalled && iter.hasNext()) {
-      DoFnInvokers.tryInvokeSetupFor(doFn, serializableOptions.get());
-      wasSetupCalled = true;
-    }
-
-    DoFnOutputManager outputManager = new DoFnOutputManager();
-
-    DoFnRunner<InputT, OutputT> doFnRunner =
-        DoFnRunners.simpleRunner(
-            serializableOptions.get(),
-            doFn,
-            CachedSideInputReader.of(new SparkSideInputReader(sideInputs, broadcastStateData)),
-            outputManager,
-            mainOutputTag,
-            additionalOutputTags,
-            new NoOpStepContext(),
-            inputCoder,
-            outputCoderMap,
-            windowingStrategy,
-            doFnSchemaInformation,
-            sideInputMapping);
-
-    DoFnRunnerWithMetrics<InputT, OutputT> doFnRunnerWithMetrics =
-        new DoFnRunnerWithMetrics<>(stepName, doFnRunner, metricsAccum);
-
-    return new ProcessContext<>(
-            doFn, doFnRunnerWithMetrics, outputManager, Collections.emptyIterator())
-        .processPartition(iter)
-        .iterator();
-  }
-
-  private class DoFnOutputManager
-      implements ProcessContext.ProcessOutputManager<Tuple2<TupleTag<?>, WindowedValue<?>>> {
-
-    private final Multimap<TupleTag<?>, WindowedValue<?>> outputs = LinkedListMultimap.create();
-
-    @Override
-    public void clear() {
-      outputs.clear();
-    }
-
-    @Override
-    public Iterator<Tuple2<TupleTag<?>, WindowedValue<?>>> iterator() {
-      Iterator<Map.Entry<TupleTag<?>, WindowedValue<?>>> entryIter = outputs.entries().iterator();
-      return Iterators.transform(entryIter, this.entryToTupleFn());
-    }
-
-    private <K, V> Function<Map.Entry<K, V>, Tuple2<K, V>> entryToTupleFn() {
-      return en -> new Tuple2<>(en.getKey(), en.getValue());
-    }
-
-    @Override
-    public synchronized <T> void output(TupleTag<T> tag, WindowedValue<T> 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 b80ec87d3c5..00000000000
--- 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<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
-  private final DoFnRunner<InputT, OutputT> delegate;
-  private final String stepName;
-  private final MetricsContainerStepMapAccumulator metricsAccum;
-
-  DoFnRunnerWithMetrics(
-      String stepName,
-      DoFnRunner<InputT, OutputT> delegate,
-      MetricsContainerStepMapAccumulator metricsAccum) {
-    this.delegate = delegate;
-    this.stepName = stepName;
-    this.metricsAccum = metricsAccum;
-  }
-
-  @Override
-  public DoFn<InputT, OutputT> 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<InputT> elem) {
-    try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer(metricsContainer())) {
-      delegate.processElement(elem);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public <KeyT> 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 <KeyT> 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 db361f7753e..00000000000
--- 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<T>
-    implements TransformTranslator<PTransform<PCollectionList<T>, PCollection<T>>> {
-
-  @Override
-  public void translateTransform(
-      PTransform<PCollectionList<T>, PCollection<T>> transform,
-      AbstractTranslationContext context) {
-    Collection<PCollection<?>> pcollectionList = context.getInputs().values();
-    Dataset<WindowedValue<T>> 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<T> pCollection = (PCollection<T>) pValue;
-        Dataset<WindowedValue<T>> 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 6391ba4600c..00000000000
--- 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<K, V>
-    implements TransformTranslator<
-        PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>>> {
-
-  @Override
-  public void translateTransform(
-      PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> transform,
-      AbstractTranslationContext context) {
-
-    @SuppressWarnings("unchecked")
-    final PCollection<KV<K, V>> inputPCollection = (PCollection<KV<K, V>>) context.getInput();
-    Dataset<WindowedValue<KV<K, V>>> input = context.getDataset(inputPCollection);
-    WindowingStrategy<?, ?> windowingStrategy = inputPCollection.getWindowingStrategy();
-    KvCoder<K, V> kvCoder = (KvCoder<K, V>) inputPCollection.getCoder();
-    Coder<V> valueCoder = kvCoder.getValueCoder();
-
-    // group by key only
-    Coder<K> keyCoder = kvCoder.getKeyCoder();
-    KeyValueGroupedDataset<K, WindowedValue<KV<K, V>>> groupByKeyOnly =
-        input.groupByKey(KVHelpers.extractKey(), EncoderHelpers.fromBeamCoder(keyCoder));
-
-    // group also by windows
-    WindowedValue.FullWindowedValueCoder<KV<K, Iterable<V>>> outputCoder =
-        WindowedValue.FullWindowedValueCoder.of(
-            KvCoder.of(keyCoder, IterableCoder.of(valueCoder)),
-            windowingStrategy.getWindowFn().windowCoder());
-    Dataset<WindowedValue<KV<K, Iterable<V>>>> 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 <K> State key type.
-   */
-  static class InMemoryStateInternalsFactory<K> implements StateInternalsFactory<K>, 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 65f496c772b..00000000000
--- 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<PTransform<PBegin, PCollection<byte[]>>> {
-
-  @Override
-  public void translateTransform(
-      PTransform<PBegin, PCollection<byte[]>> transform, AbstractTranslationContext context) {
-    Coder<WindowedValue<byte[]>> windowedValueCoder =
-        WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE);
-    Dataset<WindowedValue<byte[]>> 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 52c2d5ae642..00000000000
--- 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 <InputT>
- * @param <OutputT>
- */
-@SuppressWarnings({
-  "rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-class ParDoTranslatorBatch<InputT, OutputT>
-    implements TransformTranslator<PTransform<PCollection<InputT>, PCollectionTuple>> {
-
-  @Override
-  public void translateTransform(
-      PTransform<PCollection<InputT>, PCollectionTuple> transform,
-      AbstractTranslationContext context) {
-    String stepName = context.getCurrentTransform().getFullName();
-
-    // Check for not supported advanced features
-    // TODO: add support of Splittable DoFn
-    DoFn<InputT, OutputT> 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<WindowedValue<InputT>> inputDataSet = context.getDataset(input);
-    Map<TupleTag<?>, PCollection<?>> outputs = context.getOutputs();
-    TupleTag<?> mainOutputTag = getTupleTag(context);
-    List<TupleTag<?>> outputTags = new ArrayList<>(outputs.keySet());
-    WindowingStrategy<?, ?> windowingStrategy =
-        ((PCollection<InputT>) input).getWindowingStrategy();
-    Coder<InputT> inputCoder = ((PCollection<InputT>) input).getCoder();
-    Coder<? extends BoundedWindow> 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<PCollectionView<?>> sideInputs = getSideInputs(context);
-    Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>();
-    for (PCollectionView<?> sideInput : sideInputs) {
-      sideInputStrategies.put(sideInput, sideInput.getPCollection().getWindowingStrategy());
-    }
-
-    SideInputBroadcast broadcastStateData = createBroadcastSideInputs(sideInputs, context);
-
-    Map<TupleTag<?>, Coder<?>> outputCoderMap = context.getOutputCoders();
-    MetricsContainerStepMapAccumulator metricsAccum = MetricsAccumulator.getInstance();
-
-    List<TupleTag<?>> additionalOutputTags = new ArrayList<>();
-    for (TupleTag<?> tag : outputTags) {
-      if (!tag.equals(mainOutputTag)) {
-        additionalOutputTags.add(tag);
-      }
-    }
-
-    Map<String, PCollectionView<?>> sideInputMapping =
-        ParDoTranslation.getSideInputMapping(context.getCurrentTransform());
-    @SuppressWarnings("unchecked")
-    DoFnFunction<InputT, OutputT> 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<Tuple2<TupleTag<?>, WindowedValue<?>>> allOutputs =
-        inputDataSet.mapPartitions(doFnWrapper, EncoderHelpers.fromBeamCoder(multipleOutputCoder));
-    if (outputs.entrySet().size() > 1) {
-      allOutputs.persist();
-      for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
-        pruneOutputFilteredByTag(context, allOutputs, output, windowCoder);
-      }
-    } else {
-      Coder<OutputT> outputCoder = ((PCollection<OutputT>) outputs.get(mainOutputTag)).getCoder();
-      Coder<WindowedValue<?>> windowedValueCoder =
-          (Coder<WindowedValue<?>>) (Coder<?>) WindowedValue.getFullCoder(outputCoder, windowCoder);
-      Dataset<WindowedValue<?>> outputDataset =
-          allOutputs.map(
-              (MapFunction<Tuple2<TupleTag<?>, WindowedValue<?>>, WindowedValue<?>>)
-                  value -> value._2,
-              EncoderHelpers.fromBeamCoder(windowedValueCoder));
-      context.putDatasetWildcard(outputs.entrySet().iterator().next().getValue(), outputDataset);
-    }
-  }
-
-  private static SideInputBroadcast createBroadcastSideInputs(
-      List<PCollectionView<?>> sideInputs, AbstractTranslationContext context) {
-    JavaSparkContext jsc =
-        JavaSparkContext.fromSparkContext(context.getSparkSession().sparkContext());
-
-    SideInputBroadcast sideInputBroadcast = new SideInputBroadcast();
-    for (PCollectionView<?> sideInput : sideInputs) {
-      Coder<? extends BoundedWindow> windowCoder =
-          sideInput.getPCollection().getWindowingStrategy().getWindowFn().windowCoder();
-
-      Coder<WindowedValue<?>> windowedValueCoder =
-          (Coder<WindowedValue<?>>)
-              (Coder<?>)
-                  WindowedValue.getFullCoder(sideInput.getPCollection().getCoder(), windowCoder);
-      Dataset<WindowedValue<?>> broadcastSet = context.getSideInputDataSet(sideInput);
-      List<WindowedValue<?>> valuesList = broadcastSet.collectAsList();
-      List<byte[]> 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<PCollectionView<?>> getSideInputs(AbstractTranslationContext context) {
-    List<PCollectionView<?>> 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<InputT, OutputT> getDoFn(AbstractTranslationContext context) {
-    DoFn<InputT, OutputT> doFn;
-    try {
-      doFn = (DoFn<InputT, OutputT>) ParDoTranslation.getDoFn(context.getCurrentTransform());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return doFn;
-  }
-
-  private void pruneOutputFilteredByTag(
-      AbstractTranslationContext context,
-      Dataset<Tuple2<TupleTag<?>, WindowedValue<?>>> allOutputs,
-      Map.Entry<TupleTag<?>, PCollection<?>> output,
-      Coder<? extends BoundedWindow> windowCoder) {
-    Dataset<Tuple2<TupleTag<?>, WindowedValue<?>>> filteredDataset =
-        allOutputs.filter(new DoFnFilterFunction(output.getKey()));
-    Coder<WindowedValue<?>> windowedValueCoder =
-        (Coder<WindowedValue<?>>)
-            (Coder<?>)
-                WindowedValue.getFullCoder(
-                    ((PCollection<OutputT>) output.getValue()).getCoder(), windowCoder);
-    Dataset<WindowedValue<?>> outputDataset =
-        filteredDataset.map(
-            (MapFunction<Tuple2<TupleTag<?>, WindowedValue<?>>, WindowedValue<?>>)
-                value -> value._2,
-            EncoderHelpers.fromBeamCoder(windowedValueCoder));
-    context.putDatasetWildcard(output.getValue(), outputDataset);
-  }
-
-  static class DoFnFilterFunction implements FilterFunction<Tuple2<TupleTag<?>, WindowedValue<?>>> {
-
-    private final TupleTag<?> key;
-
-    DoFnFilterFunction(TupleTag<?> key) {
-      this.key = key;
-    }
-
-    @Override
-    public boolean call(Tuple2<TupleTag<?>, 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 5789db6cd30..00000000000
--- 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<Class<? extends PTransform>, 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 db64bfd19f3..00000000000
--- 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<FnInputT, FnOutputT, OutputT> {
-
-  private final DoFn<FnInputT, FnOutputT> doFn;
-  private final DoFnRunner<FnInputT, FnOutputT> doFnRunner;
-  private final ProcessOutputManager<OutputT> outputManager;
-  private final Iterator<TimerInternals.TimerData> timerDataIterator;
-
-  ProcessContext(
-      DoFn<FnInputT, FnOutputT> doFn,
-      DoFnRunner<FnInputT, FnOutputT> doFnRunner,
-      ProcessOutputManager<OutputT> outputManager,
-      Iterator<TimerInternals.TimerData> timerDataIterator) {
-
-    this.doFn = doFn;
-    this.doFnRunner = doFnRunner;
-    this.outputManager = outputManager;
-    this.timerDataIterator = timerDataIterator;
-  }
-
-  Iterable<OutputT> processPartition(Iterator<WindowedValue<FnInputT>> 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<OutputT> getOutputIterator() {
-    return outputManager.iterator();
-  }
-
-  private Iterable<OutputT> getOutputIterable(
-      final Iterator<WindowedValue<FnInputT>> iter,
-      final DoFnRunner<FnInputT, FnOutputT> doFnRunner) {
-    return () -> new ProcCtxtIterator(iter, doFnRunner);
-  }
-
-  interface ProcessOutputManager<T> extends OutputManager, Iterable<T> {
-    void clear();
-  }
-
-  private class ProcCtxtIterator extends AbstractIterator<OutputT> {
-
-    private final Iterator<WindowedValue<FnInputT>> inputIterator;
-    private final DoFnRunner<FnInputT, FnOutputT> doFnRunner;
-    private Iterator<OutputT> outputIterator;
-    private boolean isBundleStarted;
-    private boolean isBundleFinished;
-
-    ProcCtxtIterator(
-        Iterator<WindowedValue<FnInputT>> iterator, DoFnRunner<FnInputT, FnOutputT> 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 ebeb8a96eda..00000000000
--- 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<T>
-    implements TransformTranslator<PTransform<PBegin, PCollection<T>>> {
-
-  private static final String sourceProviderClass = DatasetSourceBatch.class.getCanonicalName();
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void translateTransform(
-      PTransform<PBegin, PCollection<T>> transform, AbstractTranslationContext context) {
-    AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>> rootTransform =
-        (AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>>)
-            context.getCurrentTransform();
-
-    BoundedSource<T> source;
-    try {
-      source = ReadTranslation.boundedSourceFromTransform(rootTransform);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    SparkSession sparkSession = context.getSparkSession();
-
-    String serializedSource = Base64Serializer.serializeUnchecked(source);
-    Dataset<Row> 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<T> windowedValueCoder =
-        WindowedValue.FullWindowedValueCoder.of(
-            source.getOutputCoder(), GlobalWindow.Coder.INSTANCE);
-
-    Dataset<WindowedValue<T>> dataset =
-        rowDataset.map(
-            RowHelpers.extractWindowedValueFromRowMapFunction(windowedValueCoder),
-            EncoderHelpers.fromBeamCoder(windowedValueCoder));
-
-    PCollection<T> output = (PCollection<T>) 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 a88d5454667..00000000000
--- 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<K, InputT> implements TransformTranslator<Reshuffle<K, InputT>> {
-
-  @Override
-  public void translateTransform(
-      Reshuffle<K, InputT> 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 875a983b401..00000000000
--- 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<T>
-    implements TransformTranslator<PTransform<PCollection<T>, PCollection<T>>> {
-
-  @Override
-  public void translateTransform(
-      PTransform<PCollection<T>, PCollection<T>> transform, AbstractTranslationContext context) {
-
-    Window.Assign<T> assignTransform = (Window.Assign<T>) transform;
-    @SuppressWarnings("unchecked")
-    final PCollection<T> input = (PCollection<T>) context.getInput();
-    @SuppressWarnings("unchecked")
-    final PCollection<T> output = (PCollection<T>) context.getOutput();
-
-    Dataset<WindowedValue<T>> inputDataset = context.getDataset(input);
-    if (WindowingHelpers.skipAssignWindows(assignTransform, context)) {
-      context.putDataset(output, inputDataset);
-    } else {
-      WindowFn<T, ?> windowFn = assignTransform.getWindowFn();
-      WindowedValue.FullWindowedValueCoder<T> windowedValueCoder =
-          WindowedValue.FullWindowedValueCoder.of(input.getCoder(), windowFn.windowCoder());
-      Dataset<WindowedValue<T>> 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 77ec5c1269a..00000000000
--- 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<K, InputT, W extends BoundedWindow>
-    implements FlatMapGroupsFunction<
-        K, WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, Iterable<InputT>>>> {
-
-  private final WindowingStrategy<?, W> windowingStrategy;
-  private final StateInternalsFactory<K> stateInternalsFactory;
-  private final SystemReduceFn<K, InputT, Iterable<InputT>, Iterable<InputT>, W> reduceFn;
-  private final SerializablePipelineOptions options;
-
-  public GroupAlsoByWindowViaOutputBufferFn(
-      WindowingStrategy<?, W> windowingStrategy,
-      StateInternalsFactory<K> stateInternalsFactory,
-      SystemReduceFn<K, InputT, Iterable<InputT>, Iterable<InputT>, W> reduceFn,
-      SerializablePipelineOptions options) {
-    this.windowingStrategy = windowingStrategy;
-    this.stateInternalsFactory = stateInternalsFactory;
-    this.reduceFn = reduceFn;
-    this.options = options;
-  }
-
-  @Override
-  public Iterator<WindowedValue<KV<K, Iterable<InputT>>>> call(
-      K key, Iterator<WindowedValue<KV<K, InputT>>> 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<WindowedValue<InputT>> values = new ArrayList<>();
-    while (iterator.hasNext()) {
-      WindowedValue<KV<K, InputT>> 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<K, InputT> outputter = new GABWOutputWindowedValue<>();
-
-    ReduceFnRunner<K, InputT, Iterable<InputT>, 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<K, InputT, Iterable<InputT>, W> reduceFnRunner)
-      throws Exception {
-    List<TimerInternals.TimerData> 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<K, V>
-      implements OutputWindowedValue<KV<K, Iterable<V>>> {
-    private final List<WindowedValue<KV<K, Iterable<V>>>> outputs = new ArrayList<>();
-
-    @Override
-    public void outputWindowedValue(
-        KV<K, Iterable<V>> output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      outputs.add(WindowedValue.of(output, timestamp, windows, pane));
-    }
-
-    @Override
-    public <AdditionalOutputT> void outputWindowedValue(
-        TupleTag<AdditionalOutputT> tag,
-        AdditionalOutputT output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      throw new UnsupportedOperationException("GroupAlsoByWindow should not use tagged outputs.");
-    }
-
-    Iterable<WindowedValue<KV<K, Iterable<V>>>> 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 25e6f112a3f..00000000000
--- 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 a9897d71091..00000000000
--- 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<String> SUPPORTED_MATERIALIZATIONS =
-      ImmutableSet.of(
-          Materializations.ITERABLE_MATERIALIZATION_URN,
-          Materializations.MULTIMAP_MATERIALIZATION_URN);
-
-  private final Map<TupleTag<?>, WindowingStrategy<?, ?>> sideInputs;
-  private final SideInputBroadcast broadcastStateData;
-
-  public SparkSideInputReader(
-      Map<PCollectionView<?>, 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<PCollectionView<?>, WindowingStrategy<?, ?>> entry : indexByView.entrySet()) {
-      sideInputs.put(entry.getKey().getTagInternal(), entry.getValue());
-    }
-    this.broadcastStateData = broadcastStateData;
-  }
-
-  @Override
-  public <T> @Nullable T get(PCollectionView<T> 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<byte[]> sideInputsValues =
-        (List<byte[]>) broadcastStateData.getBroadcastValue(tag.getId()).getValue();
-    Coder<?> coder = broadcastStateData.getCoder(tag.getId());
-
-    List<WindowedValue<?>> decodedValues = new ArrayList<>();
-    for (byte[] value : sideInputsValues) {
-      decodedValues.add((WindowedValue<?>) CoderHelpers.fromByteArray(value, coder));
-    }
-
-    Map<BoundedWindow, T> sideInputs = initializeBroadcastVariable(decodedValues, view);
-    T result = sideInputs.get(window);
-    if (result == null) {
-      switch (view.getViewFn().getMaterialization().getUrn()) {
-        case Materializations.ITERABLE_MATERIALIZATION_URN:
-          {
-            ViewFn<IterableView, T> viewFn = (ViewFn<IterableView, T>) view.getViewFn();
-            return viewFn.apply(() -> Collections.emptyList());
-          }
-        case Materializations.MULTIMAP_MATERIALIZATION_URN:
-          {
-            ViewFn<MultimapView, T> viewFn = (ViewFn<MultimapView, T>) 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 <T> boolean contains(PCollectionView<T> view) {
-    return sideInputs.containsKey(view.getTagInternal());
-  }
-
-  @Override
-  public boolean isEmpty() {
-    return sideInputs.isEmpty();
-  }
-
-  private <T> Map<BoundedWindow, T> initializeBroadcastVariable(
-      Iterable<WindowedValue<?>> inputValues, PCollectionView<T> view) {
-
-    // first partition into windows
-    Map<BoundedWindow, List<WindowedValue<?>>> partitionedElements = new HashMap<>();
-    for (WindowedValue<?> value : inputValues) {
-      for (BoundedWindow window : value.getWindows()) {
-        List<WindowedValue<?>> windowedValues =
-            partitionedElements.computeIfAbsent(window, k -> new ArrayList<>());
-        windowedValues.add(value);
-      }
-    }
-
-    Map<BoundedWindow, T> resultMap = new HashMap<>();
-
-    for (Map.Entry<BoundedWindow, List<WindowedValue<?>>> elements :
-        partitionedElements.entrySet()) {
-
-      switch (view.getViewFn().getMaterialization().getUrn()) {
-        case Materializations.ITERABLE_MATERIALIZATION_URN:
-          {
-            ViewFn<IterableView, T> viewFn = (ViewFn<IterableView, T>) view.getViewFn();
-            resultMap.put(
-                elements.getKey(),
-                viewFn.apply(
-                    () ->
-                        elements.getValue().stream()
-                            .map(WindowedValue::getValue)
-                            .collect(Collectors.toList())));
-          }
-          break;
-        case Materializations.MULTIMAP_MATERIALIZATION_URN:
-          {
-            ViewFn<MultimapView, T> viewFn = (ViewFn<MultimapView, T>) 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 1f03bac2124..00000000000
--- 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 6d3ce5aa723..00000000000
--- 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 fe3f39ef51e..00000000000
--- 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 <T> type of value that is serialized
-   * @return Byte array representing serialized object.
-   */
-  public static <T> byte[] toByteArray(T value, Coder<T> 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 <T> Type of object to be returned.
-   * @return Deserialized object.
-   */
-  public static <T> T fromByteArray(byte[] serialized, Coder<T> 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 2b86ec839c9..00000000000
--- 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 <T> Encoder<T> create(
-      Expression serializer, Expression deserializer, Class<? super T> 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 68738cf0308..00000000000
--- 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 <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> 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 <T> Expression beamSerializer(Expression obj, Coder<T> 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 <T> Expression beamDeserializer(Expression bytes, Coder<T> 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 <T extends @NonNull Object> Literal lit(T obj, Class<? extends T> 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 2406c0f49ab..00000000000
--- 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 <K, V> MapFunction<WindowedValue<KV<K, V>>, 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 f77fcea6796..00000000000
--- 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<TupleTag<T>, WindowedValue<T>{/@code} to be used
- * in spark encoders while applying {@link org.apache.beam.sdk.transforms.DoFn}.
- *
- * @param <T> 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<T> extends CustomCoder<Tuple2<TupleTag<T>, WindowedValue<T>>> {
-  Coder<TupleTag> tupleTagCoder;
-  Map<TupleTag<?>, Coder<?>> coderMap;
-  Coder<? extends BoundedWindow> windowCoder;
-
-  public static MultiOutputCoder of(
-      Coder<TupleTag> tupleTagCoder,
-      Map<TupleTag<?>, Coder<?>> coderMap,
-      Coder<? extends BoundedWindow> windowCoder) {
-    return new MultiOutputCoder(tupleTagCoder, coderMap, windowCoder);
-  }
-
-  private MultiOutputCoder(
-      Coder<TupleTag> tupleTagCoder,
-      Map<TupleTag<?>, Coder<?>> coderMap,
-      Coder<? extends BoundedWindow> windowCoder) {
-    this.tupleTagCoder = tupleTagCoder;
-    this.coderMap = coderMap;
-    this.windowCoder = windowCoder;
-  }
-
-  @Override
-  public void encode(Tuple2<TupleTag<T>, WindowedValue<T>> tuple2, OutputStream outStream)
-      throws IOException {
-    TupleTag<T> tupleTag = tuple2._1();
-    tupleTagCoder.encode(tupleTag, outStream);
-    Coder<T> valueCoder = (Coder<T>) coderMap.get(tupleTag);
-    WindowedValue.FullWindowedValueCoder<T> wvCoder =
-        WindowedValue.FullWindowedValueCoder.of(valueCoder, windowCoder);
-    wvCoder.encode(tuple2._2(), outStream);
-  }
-
-  @Override
-  public Tuple2<TupleTag<T>, WindowedValue<T>> decode(InputStream inStream)
-      throws CoderException, IOException {
-    TupleTag<T> tupleTag = (TupleTag<T>) tupleTagCoder.decode(inStream);
-    Coder<T> valueCoder = (Coder<T>) coderMap.get(tupleTag);
-    WindowedValue.FullWindowedValueCoder<T> wvCoder =
-        WindowedValue.FullWindowedValueCoder.of(valueCoder, windowCoder);
-    WindowedValue<T> 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 9b5d5da2b2c..00000000000
--- 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 <T> The type of the object.
-   * @return A {@link MapFunction} that accepts a {@link Row} and returns its {@link WindowedValue}.
-   */
-  public static <T> MapFunction<Row, WindowedValue<T>> extractWindowedValueFromRowMapFunction(
-      WindowedValue.WindowedValueCoder<T> windowedValueCoder) {
-    return (MapFunction<Row, WindowedValue<T>>)
-        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 <T> InternalRow storeWindowedValueInRow(
-      WindowedValue<T> windowedValue, Coder<T> coder) {
-    List<Object> list = new ArrayList<>();
-    // serialize the windowedValue to bytes array to comply with dataset binary schema
-    WindowedValue.FullWindowedValueCoder<T> 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 71dca5264dd..00000000000
--- 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 9a397f98cea..00000000000
--- 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<String, Broadcast<?>> bcast = new HashMap<>();
-  private final Map<String, Coder<?>> 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 5085eb9f796..00000000000
--- 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.
-   *
-   * <p>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 <T, W extends BoundedWindow> boolean skipAssignWindows(
-      Window.Assign<T> transform, AbstractTranslationContext context) {
-    WindowFn<? super T, W> windowFnToApply = (WindowFn<? super T, W>) transform.getWindowFn();
-    PCollection<T> input = (PCollection<T>) context.getInput();
-    WindowFn<?, ?> windowFnOfInput = input.getWindowingStrategy().getWindowFn();
-    return windowFnToApply == null
-        || (windowFnOfInput instanceof GlobalWindows && windowFnToApply instanceof GlobalWindows);
-  }
-
-  public static <T, W extends BoundedWindow>
-      MapFunction<WindowedValue<T>, WindowedValue<T>> assignWindowsMapFunction(
-          WindowFn<T, W> windowFn) {
-    return (MapFunction<WindowedValue<T>, WindowedValue<T>>)
-        windowedValue -> {
-          final BoundedWindow boundedWindow = Iterables.getOnlyElement(windowedValue.getWindows());
-          final T element = windowedValue.getValue();
-          final Instant timestamp = windowedValue.getTimestamp();
-          Collection<W> 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 7079eadfbe2..00000000000
--- 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 2754ac50003..00000000000
--- 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 c91ab8a1af6..00000000000
--- 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}.
- *
- * <p>As Continuous streaming is tagged experimental in spark (no aggregation support + no exactly
- * once guaranty), this class does no implement {@link ContinuousReadSupport}.
- *
- * <p>Spark {@link Offset}s are ignored because:
- *
- * <ul>
- *   <li>resuming from checkpoint is supported by the Beam framework through {@link CheckpointMark}
- *   <li>{@link DatasetSourceStreaming} is a generic wrapper that could wrap a Beam {@link
- *       UnboundedSource} that cannot specify offset ranges
- * </ul>
- *
- * 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<StructType> 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<T, CheckpointMarkT> source;
-    private SerializablePipelineOptions serializablePipelineOptions;
-
-    private final List<DatasetPartitionReader> 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<Offset> start, Optional<Offset> 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<InputPartition<InternalRow>> planInputPartitions() {
-      PipelineOptions options = serializablePipelineOptions.get();
-      List<InputPartition<InternalRow>> result = new ArrayList<>();
-      try {
-        List<? extends UnboundedSource<T, CheckpointMarkT>> splits =
-            source.split(numPartitions, options);
-        for (UnboundedSource<T, CheckpointMarkT> split : splits) {
-          result.add(
-              new InputPartition<InternalRow>() {
-
-                @Override
-                public InputPartitionReader<InternalRow> createPartitionReader() {
-                  DatasetPartitionReader<T, CheckpointMarkT> 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<InternalRow> {
-    private boolean started;
-    private boolean closed;
-    private final UnboundedSource<T, CheckpointMarkT> source;
-    private UnboundedSource.UnboundedReader<T> reader;
-
-    DatasetPartitionReader(
-        UnboundedSource<T, CheckpointMarkT> 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<T> 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 73d99efa463..00000000000
--- 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<Class<? extends PTransform>, 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 8abc8771a4e..00000000000
--- 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<T>
-    implements TransformTranslator<PTransform<PBegin, PCollection<T>>> {
-
-  private static final String sourceProviderClass = DatasetSourceStreaming.class.getCanonicalName();
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void translateTransform(
-      PTransform<PBegin, PCollection<T>> transform, AbstractTranslationContext context) {
-    AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>> rootTransform =
-        (AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>>)
-            context.getCurrentTransform();
-
-    UnboundedSource<T, UnboundedSource.CheckpointMark> source;
-    try {
-      source = ReadTranslation.unboundedSourceFromTransform(rootTransform);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    SparkSession sparkSession = context.getSparkSession();
-
-    String serializedSource = Base64Serializer.serializeUnchecked(source);
-    Dataset<Row> 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<T> windowedValueCoder =
-        WindowedValue.FullWindowedValueCoder.of(
-            source.getOutputCoder(), GlobalWindow.Coder.INSTANCE);
-    Dataset<WindowedValue<T>> dataset =
-        rowDataset.map(
-            RowHelpers.extractWindowedValueFromRowMapFunction(windowedValueCoder),
-            EncoderHelpers.fromBeamCoder(windowedValueCoder));
-
-    PCollection<T> output = (PCollection<T>) 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 67f3613e056..00000000000
--- 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 d2e4751c811..00000000000
--- 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 <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
-    @SuppressWarnings("unchecked")
-    final Cache<Key<T>, Value<T>> materializedCasted =
-        (Cache) SideInputStorage.getMaterializedSideInputs();
-
-    Key<T> sideInputKey = new Key<>(view, window);
-
-    try {
-      Value<T> 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 <T> boolean contains(PCollectionView<T> 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 c5bc71af602..00000000000
--- 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 <T> Seq<T> seqOf(T... t) {
-    return new WrappedArray.ofRef<>(t);
-  }
-
-  public static <T> Seq<T> listOf(T t) {
-    return emptyList().$colon$colon(t);
-  }
-
-  public static <T> List<T> emptyList() {
-    return (List<T>) 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 4febddd4f88..00000000000
--- 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<Key<?>, Value<?>> materializedSideInputs =
-      CacheBuilder.newBuilder().expireAfterAccess(5, TimeUnit.MINUTES).build();
-
-  static Cache<Key<?>, Value<?>> getMaterializedSideInputs() {
-    return materializedSideInputs;
-  }
-
-  /**
-   * Composite key of {@link PCollectionView} and {@link BoundedWindow} used to identify
-   * materialized results.
-   *
-   * @param <T> type of result
-   */
-  public static class Key<T> {
-
-    private final PCollectionView<T> view;
-    private final BoundedWindow window;
-
-    Key(PCollectionView<T> 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<T> {
-    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 470bef88fb4..00000000000
--- 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 33eef26dddd..00000000000
--- 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<String, String> sparkConfig) {
-    builder = SparkSessionFactory.sessionBuilder(sparkMaster);
-    sparkConfig.forEach(builder::config);
-  }
-
-  public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local[2]", sparkConfig);
-  }
-
-  public SparkSessionRule(String sparkMaster, KV<String, String>... 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 30d8297809b..00000000000
--- 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 b44df7bf101..00000000000
--- 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<String, String> printParDo(final String prefix) {
-    return ParDo.of(
-        new DoFn<String, String>() {
-
-          @ProcessElement
-          public void processElement(final ProcessContext c) {
-            System.out.println(prefix + " " + c.element());
-          }
-        });
-  }
-
-  private PTransform<PBegin, PCollection<String>> 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<String, String>() {
-
-                    @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 f994f7712b3..00000000000
--- 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> 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<Gauge> 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 f1b996eaf21..00000000000
--- 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 2f02656dc37..00000000000
--- 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<String> WORDS =
-      ImmutableList.of("hi there", "hi", "hi sue bob", "hi sue", "", "bob hi");
-  private static final ImmutableSet<String> 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<String> 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 fd0aa35e5c8..00000000000
--- 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<Object> 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<Object> 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 52e60a3db54..00000000000
--- 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<Integer> 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<Integer> 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<KV<Integer, Integer>> 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<KV<Integer, Integer>> 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<KV<Integer, Integer>> 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<KV<Integer, Integer>> 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<Integer> 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<Integer, Integer, Integer>)
-                                (integer1, integer2) -> integer1 > integer2 ? integer1 : integer2))
-                    .withoutDefaults());
-    PAssert.that(input).containsInAnyOrder(1, 3, 5, 5, 5);
-    pipeline.run();
-  }
-
-  @Test
-  public void testCountPerElementWithSlidingWindows() {
-    PCollection<String> 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<KV<String, Long>> 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 0175d03f875..00000000000
--- 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<String> 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<String> input = pipeline.apply(TextIO.read().from(file.getPath()));
-    PAssert.that(input).containsInAnyOrder(lines);
-    pipeline.run();
-  }
-
-  private static File createFile(List<String> 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<String> createLines(int size) {
-    List<String> 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 e126d06e685..00000000000
--- 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<Integer> input1 = pipeline.apply(Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
-    PCollection<Integer> input2 = pipeline.apply(Create.of(11, 12, 13, 14, 15, 16, 17, 18, 19, 20));
-    PCollectionList<Integer> pcs = PCollectionList.of(input1).and(input2);
-    PCollection<Integer> 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 07850232853..00000000000
--- 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<Integer, Iterable<Integer>>, KV<Integer, Iterable<Integer>>>() {
-
-                  @ProcessElement
-                  public void processElement(ProcessContext context) {
-                    KV<Integer, Iterable<Integer>> 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<KV<Integer, Integer>> 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<KV<Integer, Iterable<Integer>>> 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 16d9a8b7fa8..00000000000
--- 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<Integer> 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<Integer> 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<List<Integer>> sideInputView =
-        pipeline.apply("Create sideInput", Create.of(1, 2, 3)).apply(View.asList());
-    PCollection<Integer> input =
-        pipeline
-            .apply("Create input", Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
-            .apply(
-                ParDo.of(
-                        new DoFn<Integer, Integer>() {
-                          @ProcessElement
-                          public void processElement(ProcessContext c) {
-                            List<Integer> 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<Integer> sideInputView =
-        pipeline.apply("Create sideInput", Create.of(1)).apply(View.asSingleton());
-
-    PCollection<Integer> input =
-        pipeline
-            .apply("Create input", Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
-            .apply(
-                ParDo.of(
-                        new DoFn<Integer, Integer>() {
-                          @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<Map<String, Integer>> sideInputView =
-        pipeline
-            .apply("Create sideInput", Create.of(KV.of("key1", 1), KV.of("key2", 2)))
-            .apply(View.asMap());
-    PCollection<Integer> input =
-        pipeline
-            .apply("Create input", Create.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
-            .apply(
-                ParDo.of(
-                        new DoFn<Integer, Integer>() {
-                          @ProcessElement
-                          public void processElement(ProcessContext c) {
-                            Map<String, Integer> 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<Integer, Integer> PLUS_ONE_DOFN =
-      new DoFn<Integer, Integer>() {
-        @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 70cdca630b9..00000000000
--- 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<Integer> 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 b8b41010a24..00000000000
--- 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<Integer> 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 c8a8fba8d28..00000000000
--- 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 <T> Dataset<T> createDataset(List<T> data, Encoder<T> encoder) {
-    Dataset<T> ds = sessionRule.getSession().createDataset(data, encoder);
-    ds.printSchema();
-    return ds;
-  }
-
-  @Test
-  public void beamCoderToSparkEncoderTest() {
-    List<Integer> data = Arrays.asList(1, 2, 3);
-    Dataset<Integer> 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<PrivateString> data = asList(new PrivateString("1"), new PrivateString("2"));
-    Dataset<PrivateString> dataset = createDataset(data, fromBeamCoder(PrivateString.CODER));
-    assertThat(dataset.collect(), equalTo(data.toArray()));
-  }
-
-  private static class PrivateString {
-    private static final Coder<PrivateString> CODER =
-        DelegateCoder.of(
-            StringUtf8Coder.of(),
-            str -> str.string,
-            PrivateString::new,
-            new TypeDescriptor<PrivateString>() {});
-
-    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 a06d2cec1e9..00000000000
--- 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 b384b9b9d35..00000000000
--- 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<Object> 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<Object> 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 3d7da111a9c..00000000000
--- 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 b3cbb9f6dac..7d820c28f83 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 4b714b65581..9f9465ccde8 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 @@ public final class SparkContextFactory {
     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 08b418fa366..657ad0c42b6 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.metrics.MetricsAccumulator;
 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.storage.StorageLevel;
 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 @@ public class SparkStreamingPortablePipelineTranslator
         }
       }
       // 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 4cd73585e46..7fd0f9155d3 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.MetricsAccumulator;
 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.Partitioner;
 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 final class TransformTranslator {
                 vaCoder,
                 windowingStrategy);
 
+        FlatMapFunction<
+                SparkCombineFn.WindowedAccumulator<KV<K, InputT>, InputT, AccumT, ?>,
+                WindowedValue<OutputT>>
+            flatMapFunction =
+                windowedAccumulator ->
+                    sparkCombineFn.extractOutputStream(windowedAccumulator).iterator();
+
         JavaPairRDD<K, WindowedValue<OutputT>> kwvs =
-            SparkCompat.extractOutput(accumulatePerKey, sparkCombineFn);
+            accumulatePerKey.flatMapValues(flatMapFunction);
         JavaRDD<WindowedValue<KV<K, OutputT>>> 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 55048f0f6fd..2abeeb7c33f 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.TransformEvaluator;
 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.JavaPairDStream;
 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 final class StreamingTransformTranslator {
         }
         // start by unifying streams into a single stream.
         JavaDStream<WindowedValue<T>> 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 17bafd5ff52..00000000000
--- 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.
-   *
-   * <p>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 <T> JavaDStream<WindowedValue<T>> joinStreams(
-      JavaStreamingContext streamingContext, List<JavaDStream<WindowedValue<T>>> 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<WindowedValue<T>>) 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<WindowedValue<T>>) result;
-    } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
-      throw new RuntimeException("Error invoking Spark union", e);
-    }
-  }
-
-  /**
-   * Extracts the output for a given collection of WindowedAccumulators.
-   *
-   * <p>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 <K, InputT, AccumT, OutputT> JavaPairRDD<K, WindowedValue<OutputT>> extractOutput(
-      JavaPairRDD<K, SparkCombineFn.WindowedAccumulator<KV<K, InputT>, InputT, AccumT, ?>>
-          accumulatePerKey,
-      SparkCombineFn<KV<K, InputT>, InputT, AccumT, OutputT> sparkCombineFn) {
-    try {
-      if (accumulatePerKey.context().version().startsWith("3")) {
-        FlatMapFunction<
-                SparkCombineFn.WindowedAccumulator<KV<K, InputT>, InputT, AccumT, ?>,
-                WindowedValue<OutputT>>
-            flatMapFunction =
-                (FlatMapFunction<
-                        SparkCombineFn.WindowedAccumulator<KV<K, InputT>, InputT, AccumT, ?>,
-                        WindowedValue<OutputT>>)
-                    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<K, WindowedValue<OutputT>>) result;
-      }
-
-      Function<
-              SparkCombineFn.WindowedAccumulator<KV<K, InputT>, InputT, AccumT, ?>,
-              Iterable<WindowedValue<OutputT>>>
-          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<K, WindowedValue<OutputT>>) 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 0a1b137c5b7..c6c2a50279c 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 1bcce558384..37bdb106038 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -1535,9 +1535,8 @@ class SparkRunnerOptions(PipelineOptions):
     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 b4c46c0dac0..c06d1b9c1a2 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 @@ class SparkJarJobServer(job_server.JavaJarJobServer):
       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 97fa6b629ce..f754b4c330a 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 @@ class SparkUberJarJobServer(abstract_job_service.AbstractJobServiceServicer):
       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 b8879bfa3f3..3c19893774e 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 15cf6cf5ac7..3b166c077e0 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: