-
-See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
-
-
-GitHub Actions Tests Status (on master branch)
-------------------------------------------------------------------------------------------------
-[](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
-[](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
-[](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
-
-See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
diff --git a/.test-infra/validate-runner/src/main/resources/configuration.yaml b/.test-infra/validate-runner/src/main/resources/configuration.yaml
index 31934d2146d7..4eb8cd018f99 100644
--- a/.test-infra/validate-runner/src/main/resources/configuration.yaml
+++ b/.test-infra/validate-runner/src/main/resources/configuration.yaml
@@ -19,6 +19,6 @@ batch:
- dataflow: beam_PostCommit_Java_VR_Dataflow_V2
stream:
- flink: beam_PostCommit_Java_PVR_Flink_Streaming
- - samza: beam_PostCommit_Java_PVR_Samza
+ - spark: beam_PostCommit_Java_PVR_Spark3_Streaming
server: https://ci-beam.apache.org/
jsonapi: testReport/api/json
diff --git a/CHANGES.md b/CHANGES.md
index bdcbd3451c7b..dc2a94f6dd02 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -82,6 +82,7 @@
## Deprecations
* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)).
+* Removed Samza Runner support ([#35448](https://github.com/apache/beam/issues/35448)).
## Bugfixes
diff --git a/build.gradle.kts b/build.gradle.kts
index 2ccab281b49c..f6e97d19487e 100644
--- a/build.gradle.kts
+++ b/build.gradle.kts
@@ -283,8 +283,6 @@ tasks.register("javaPreCommit") {
dependsOn(":runners:local-java:build")
dependsOn(":runners:portability:java:build")
dependsOn(":runners:prism:java:build")
- dependsOn(":runners:samza:build")
- dependsOn(":runners:samza:job-server:build")
dependsOn(":runners:spark:3:build")
dependsOn(":runners:spark:3:job-server:build")
dependsOn(":runners:twister2:build")
@@ -412,7 +410,6 @@ tasks.register("javaPostCommit") {
}
tasks.register("javaPostCommitSickbay") {
- dependsOn(":runners:samza:validatesRunnerSickbay")
for (version in project.ext.get("allFlinkVersions") as Array<*>) {
dependsOn(":runners:flink:${version}:validatesRunnerSickbay")
}
diff --git a/runners/samza/build.gradle b/runners/samza/build.gradle
deleted file mode 100644
index 626588b79a5d..000000000000
--- a/runners/samza/build.gradle
+++ /dev/null
@@ -1,194 +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 groovy.json.JsonOutput
-
-plugins { id 'org.apache.beam.module' }
-applyJavaNature(
- exportJavadoc: false,
- classesTriggerCheckerBugs: [
- 'GroupWithoutRepartition': 'https://github.com/typetools/checker-framework/issues/3791',
- ],
- automaticModuleName: 'org.apache.beam.runners.samza',
-)
-
-description = "Apache Beam :: Runners :: Samza"
-
-/*
- * We need to rely on manually specifying these evaluationDependsOn to ensure that
- * the following projects are evaluated before we evaluate this project. This is because
- * we are attempting to reference the "sourceSets.test.output" directly.
- */
-evaluationDependsOn(":sdks:java:core")
-
-configurations {
- validatesRunner
-}
-
-def samza_version = "1.6.0"
-
-dependencies {
- implementation library.java.vendored_guava_32_1_2_jre
- implementation project(path: ":sdks:java:core", configuration: "shadow")
- implementation project(":runners:core-java")
- implementation project(":runners:java-fn-execution")
- implementation project(":runners:java-job-service")
- implementation library.java.jackson_annotations
- implementation library.java.slf4j_api
- implementation library.java.joda_time
- implementation library.java.args4j
- implementation library.java.commons_io
- implementation library.java.commons_collections
- runtimeOnly "org.rocksdb:rocksdbjni:6.15.2"
- runtimeOnly "org.scala-lang:scala-library:2.11.8"
- implementation "org.apache.samza:samza-api:$samza_version"
- implementation "org.apache.samza:samza-core_2.11:$samza_version"
- runtimeOnly "org.apache.samza:samza-kafka_2.11:$samza_version"
- runtimeOnly "org.apache.samza:samza-kv_2.11:$samza_version"
- implementation "org.apache.samza:samza-kv-rocksdb_2.11:$samza_version"
- implementation "org.apache.samza:samza-kv-inmemory_2.11:$samza_version"
- implementation "org.apache.samza:samza-yarn_2.11:$samza_version"
- compileOnly library.java.error_prone_annotations
- runtimeOnly "org.apache.kafka:kafka-clients:2.0.1"
- implementation library.java.vendored_grpc_1_69_0
- implementation project(path: ":model:fn-execution", configuration: "shadow")
- implementation project(path: ":model:job-management", configuration: "shadow")
- implementation project(path: ":model:pipeline", configuration: "shadow")
- testImplementation project(path: ":sdks:java:core", configuration: "shadowTest")
- testImplementation project(path: ":runners:core-java", configuration: "testRuntimeMigration")
- testImplementation library.java.hamcrest
- testImplementation library.java.junit
- testImplementation library.java.mockito_core
- testImplementation library.java.jackson_dataformat_yaml
- testImplementation library.java.google_code_gson
- validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest")
- validatesRunner project(path: ":runners:core-java", configuration: "testRuntimeMigration")
- validatesRunner project(project.path)
-}
-
-configurations.all {
- exclude group: "org.slf4j", module: "slf4j-jdk14"
-}
-
-def sickbayTests = [
- // TODO(https://github.com/apache/beam/issues/21033)
- 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInGlobalWindowBatchSizeByteSizeFn',
- 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInStreamingMode',
- 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testWithShardedKeyInGlobalWindow',
- // TODO(https://github.com/apache/beam/issues/21036)
- 'org.apache.beam.sdk.transforms.MapElementsTest.testMapSimpleFunction',
- // TODO(https://github.com/apache/beam/issues/21035)
- 'org.apache.beam.sdk.transforms.ViewTest.testEmptySingletonSideInput',
- 'org.apache.beam.sdk.transforms.ViewTest.testNonSingletonSideInput',
- // TODO(https://github.com/apache/beam/issues/21037)
- 'org.apache.beam.sdk.transforms.WithTimestampsTest.withTimestampsBackwardsInTimeShouldThrow',
- 'org.apache.beam.sdk.transforms.WithTimestampsTest.withTimestampsWithNullTimestampShouldThrow',
- // TODO(https://github.com/apache/beam/issues/21039)
- 'org.apache.beam.sdk.io.FileIOTest*',
- // TODO(https://github.com/apache/beam/issues/21038)
- 'org.apache.beam.sdk.io.AvroIOTest*',
- // TODO(https://github.com/apache/beam/issues/21040)
- 'org.apache.beam.sdk.PipelineTest.testEmptyPipeline',
- // TODO(https://github.com/apache/beam/issues/21041)
- 'org.apache.beam.sdk.coders.PCollectionCustomCoderTest.testEncodingNPException',
- 'org.apache.beam.sdk.coders.PCollectionCustomCoderTest.testEncodingIOException',
- 'org.apache.beam.sdk.coders.PCollectionCustomCoderTest.testDecodingNPException',
- 'org.apache.beam.sdk.coders.PCollectionCustomCoderTest.testDecodingIOException',
- // https://github.com/apache/beam/issues/19344
- 'org.apache.beam.sdk.io.BoundedReadFromUnboundedSourceTest.testTimeBound',
- // https://github.com/apache/beam/issues/31725
- 'org.apache.beam.sdk.io.TextIOWriteTest.testWriteUnboundedWithCustomBatchParameters',
-]
-tasks.register("validatesRunner", Test) {
- group = "Verification"
- description "Validates Samza runner"
- systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
- "--runner=TestSamzaRunner",
- ])
-
- classpath = configurations.validatesRunner
- testClassesDirs = files(project(":sdks:java:core").sourceSets.test.output.classesDirs)
- useJUnit {
- includeCategories 'org.apache.beam.sdk.testing.NeedsRunner'
- includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
- excludeCategories 'org.apache.beam.sdk.testing.UsesExternalService'
- // Should be run only in a properly configured SDK harness environment
- excludeCategories 'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment'
- excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
- excludeCategories 'org.apache.beam.sdk.testing.UsesSchema'
- excludeCategories 'org.apache.beam.sdk.testing.LargeKeys$Above100MB'
- excludeCategories 'org.apache.beam.sdk.testing.UsesAttemptedMetrics'
- excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
- excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime'
- excludeCategories 'org.apache.beam.sdk.testing.UsesMetricsPusher'
- excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
- excludeCategories 'org.apache.beam.sdk.testing.UsesProcessingTimeTimers'
- excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering'
- excludeCategories 'org.apache.beam.sdk.testing.UsesOnWindowExpiration'
- excludeCategories 'org.apache.beam.sdk.testing.UsesOrderedListState'
- excludeCategories 'org.apache.beam.sdk.testing.UsesMultimapState'
- excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
- excludeCategories 'org.apache.beam.sdk.testing.UsesLoopingTimer'
- excludeCategories 'org.apache.beam.sdk.testing.UsesTriggeredSideInputs'
- }
- filter {
- for (String test : sickbayTests) {
- excludeTestsMatching test
- }
- // TODO(BEAM-10025)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestampDefaultUnbounded'
- // TODO(https://github.com/apache/beam/issues/20703)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestamp'
- // TODO(https://github.com/apache/beam/issues/20703)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testRelativeTimerWithOutputTimestamp'
- // TODO(https://github.com/apache/beam/issues/20847)
- excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testFirstElementLate'
- // TODO(https://github.com/apache/beam/issues/20846)
- excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testLateDataAccumulating'
-
- // These tests fail since there is no support for side inputs in Samza's unbounded splittable DoFn integration
- excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testWindowedSideInputWithCheckpointsUnbounded'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testSideInputUnbounded'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testWindowedSideInputUnbounded'
- // These tests produce the output but either the pipeline doesn't shutdown or PAssert fails
- excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testAdditionalOutputUnbounded'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testPairWithIndexBasicUnbounded'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testPairWithIndexWindowedTimestampedUnbounded'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.SplittableDoFnTest.testOutputAfterCheckpointUnbounded'
- }
-}
-
-tasks.register("validatesRunnerSickbay", Test) {
- group = "Verification"
- description "Validates Samza runner (Sickbay Tests)"
- systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
- "--runner=TestSamzaRunner",
- ])
-
- classpath = configurations.validatesRunner
- testClassesDirs = files(project(":sdks:java:core").sourceSets.test.output.classesDirs)
-
- filter {
- for (String test : sickbayTests) {
- includeTestsMatching test
- }
- }
-}
-
-// Generates :runners:samza:runQuickstartJavaSamza
-createJavaExamplesArchetypeValidationTask(type: 'Quickstart', runner:'Samza')
diff --git a/runners/samza/job-server/build.gradle b/runners/samza/job-server/build.gradle
deleted file mode 100644
index 7ffb2becd6d0..000000000000
--- a/runners/samza/job-server/build.gradle
+++ /dev/null
@@ -1,253 +0,0 @@
-import org.apache.beam.gradle.BeamModulePlugin
-
-/*
- * 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.
- */
-
-apply plugin: 'org.apache.beam.module'
-apply plugin: 'application'
-// we need to set mainClassName before applying shadow plugin
-mainClassName = "org.apache.beam.runners.samza.SamzaJobServerDriver"
-
-applyJavaNature(
- automaticModuleName: 'org.apache.beam.runners.samza.jobserver',
- archivesBaseName: project.hasProperty('archives_base_name') ? archives_base_name : archivesBaseName,
- validateShadowJar: false,
- exportJavadoc: false,
- shadowClosure: {
- append "reference.conf"
- },
-)
-
-def samzaRunnerProject = project.parent.path
-
-description = "Apache Beam :: Runners :: Samza :: Job Server"
-
-configurations {
- validatesPortableRunner
-}
-
-dependencies {
- implementation project(samzaRunnerProject)
- permitUnusedDeclared project(samzaRunnerProject)
- runtimeOnly group: "org.slf4j", name: "jcl-over-slf4j", version: dependencies.create(project.library.java.slf4j_api).getVersion()
- validatesPortableRunner project(path: samzaRunnerProject, configuration: "testRuntimeMigration")
- validatesPortableRunner project(path: ":sdks:java:core", configuration: "shadowTest")
- validatesPortableRunner project(path: ":runners:core-java", configuration: "testRuntimeMigration")
- validatesPortableRunner project(path: ":runners:portability:java", configuration: "testRuntimeMigration")
- runtimeOnly library.java.slf4j_simple
-}
-
-runShadow {
- args = []
-}
-
-def portableValidatesRunnerTask(String name, boolean docker) {
- def tempDir = File.createTempDir()
- def pipelineOptions = [
- "--configOverride={\"job.non-logged.store.base.dir\":\"" + tempDir + "\"}"
- ]
- createPortableValidatesRunnerTask(
- name: "validatesPortableRunner${name}",
- jobServerDriver: "org.apache.beam.runners.samza.SamzaJobServerDriver",
- jobServerConfig: "--job-host=localhost,--job-port=0,--artifact-port=0,--expansion-port=0",
- testClasspathConfiguration: configurations.validatesPortableRunner,
- numParallelTests: 1,
- pipelineOpts: pipelineOptions,
- environment: docker ? BeamModulePlugin.PortableValidatesRunnerConfiguration.Environment.DOCKER : BeamModulePlugin.PortableValidatesRunnerConfiguration.Environment.EMBEDDED,
- testCategories: {
- if (docker) {
- includeCategories 'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment'
- return
- }
- // TODO(https://github.com/apache/beam/issues/22657)
- // includeCategories 'org.apache.beam.sdk.testing.NeedsRunner'
- includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
- // Should be run only in a properly configured SDK harness environment
- excludeCategories 'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment'
- // TODO: BEAM-12350
- excludeCategories 'org.apache.beam.sdk.testing.UsesAttemptedMetrics'
- // TODO: BEAM-12681
- excludeCategories 'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
- // Larger keys are possible, but they require more memory.
- excludeCategories 'org.apache.beam.sdk.testing.LargeKeys$Above10MB'
- excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
- excludeCategories 'org.apache.beam.sdk.testing.UsesExternalService'
- excludeCategories 'org.apache.beam.sdk.testing.UsesCustomWindowMerging'
- excludeCategories 'org.apache.beam.sdk.testing.UsesFailureMessage'
- excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
- excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
- excludeCategories 'org.apache.beam.sdk.testing.UsesMultimapState'
- excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
- excludeCategories 'org.apache.beam.sdk.testing.UsesOrderedListState'
- excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering'
- excludeCategories 'org.apache.beam.sdk.testing.UsesOnWindowExpiration'
- excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
- excludeCategories 'org.apache.beam.sdk.testing.UsesOrderedListState'
- excludeCategories 'org.apache.beam.sdk.testing.UsesBoundedSplittableParDo'
- excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime'
- // TODO(https://github.com/apache/beam/issues/21023)
- excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithMultipleStages'
- excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
- excludeCategories 'org.apache.beam.sdk.testing.UsesLoopingTimer'
- excludeCategories 'org.apache.beam.sdk.testing.UsesTriggeredSideInputs'
- },
- testFilter: {
- // TODO(https://github.com/apache/beam/issues/21042)
- excludeTestsMatching "org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2"
- excludeTestsMatching "org.apache.beam.sdk.transforms.FlattenTest.testEmptyFlattenAsSideInput"
- excludeTestsMatching "org.apache.beam.sdk.transforms.FlattenTest.testFlattenPCollectionsEmptyThenParDo"
- excludeTestsMatching "org.apache.beam.sdk.transforms.FlattenTest.testFlattenPCollectionsEmpty"
- // TODO(BEAM-10025)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestampDefaultUnbounded'
- // TODO(https://github.com/apache/beam/issues/20703)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestamp'
- // TODO(https://github.com/apache/beam/issues/20847)
- excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testFirstElementLate'
- // TODO(https://github.com/apache/beam/issues/20846)
- excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testLateDataAccumulating'
- // TODO(https://github.com/apache/beam/issues/21142)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupByKeyTest$WindowTests.testWindowFnPostMerging'
- // TODO(https://github.com/apache/beam/issues/21143)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testParDoShiftTimestampInvalid'
- // TODO(https://github.com/apache/beam/issues/21144)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testParDoShiftTimestampInvalidZeroAllowed'
- // TODO(https://github.com/apache/beam/issues/32520)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoLifecycleTest.testTeardownCalledAfterExceptionIn*Stateful'
- // TODO(https://github.com/apache/beam/issues/21145)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.DeduplicateTest.testEventTime'
- // TODO(https://github.com/apache/beam/issues/21146)
- excludeTestsMatching 'org.apache.beam.sdk.io.TFRecordIOTest.testReadInvalidRecord'
- // TODO(https://github.com/apache/beam/issues/21147)
- excludeTestsMatching 'org.apache.beam.sdk.io.TFRecordIOTest.testReadInvalidDataMask'
- // TODO(https://github.com/apache/beam/issues/21148)
- excludeTestsMatching 'org.apache.beam.sdk.io.TFRecordIOTest.testReadInvalidLengthMask'
- // TODO(https://github.com/apache/beam/issues/21149)
- excludeTestsMatching 'org.apache.beam.sdk.io.TextIOReadTest$CompressedReadTest.testCompressedReadWithoutExtension'
- // TODO(https://github.com/apache/beam/issues/21150)
- excludeTestsMatching 'org.apache.beam.sdk.io.WriteFilesTest.testWithRunnerDeterminedShardingUnbounded'
- // TODO(https://github.com/apache/beam/issues/211505)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$MultipleInputsAndOutputTests.testParDoWritingToUndeclaredTag'
- // TODO(https://github.com/apache/beam/issues/21152)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$MultipleInputsAndOutputTests.testParDoReadingFromUnknownSideInput'
- // TODO(https://github.com/apache/beam/issues/21153)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testMapSideInputWithNullValuesCatchesDuplicates'
-
- // TODO(https://github.com/apache/beam/issues/21041)
- excludeTestsMatching 'org.apache.beam.sdk.coders.PCollectionCustomCoderTest.testEncodingNPException'
- excludeTestsMatching 'org.apache.beam.sdk.coders.PCollectionCustomCoderTest.testEncodingIOException'
- excludeTestsMatching 'org.apache.beam.sdk.coders.PCollectionCustomCoderTest.testDecodingNPException'
- excludeTestsMatching 'org.apache.beam.sdk.coders.PCollectionCustomCoderTest.testDecodingIOException'
- // TODO(https://github.com/apache/beam/issues/21040)
- excludeTestsMatching 'org.apache.beam.sdk.PipelineTest.testEmptyPipeline'
- // TODO(https://github.com/apache/beam/issues/21038)
- excludeTestsMatching 'org.apache.beam.sdk.io.AvroIOTest*'
- // TODO(https://github.com/apache/beam/issues/21039)
- excludeTestsMatching 'org.apache.beam.sdk.io.FileIOTest*'
- // TODO(https://github.com/apache/beam/issues/21037)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.WithTimestampsTest.withTimestampsBackwardsInTimeShouldThrow'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.WithTimestampsTest.withTimestampsWithNullTimestampShouldThrow'
- // TODO(https://github.com/apache/beam/issues/21035)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptySingletonSideInput'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testNonSingletonSideInput'
- // TODO(https://github.com/apache/beam/issues/21036)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.MapElementsTest.testMapSimpleFunction'
- // TODO(https://github.com/apache/beam/issues/21033)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInGlobalWindowBatchSizeByteSizeFn'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInStreamingMode'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testWithShardedKeyInGlobalWindow'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testWithUnevenBatches'
- excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInGlobalWindowBatchSizeByteSize'
- // TODO(BEAM-10025)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestampDefaultUnbounded'
- // TODO(https://github.com/apache/beam/issues/20703)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestamp'
- // TODO(https://github.com/apache/beam/issues/20703)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testRelativeTimerWithOutputTimestamp'
- // TODO(BEAM-13498)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testProcessElementSkew'
- // TODO(https://github.com/apache/beam/issues/22650)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupByKeyTest$BasicTests.testAfterProcessingTimeContinuationTriggerUsingState'
- // TODO(https://github.com/apache/beam/issues/29973)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.ReshuffleTest.testReshufflePreservesMetadata'
- // TODO(https://github.com/apache/beam/issues/31231)
- excludeTestsMatching 'org.apache.beam.sdk.transforms.RedistributeTest.testRedistributePreservesMetadata'
- }
- )
-}
-
-project.ext.validatesPortableRunnerDocker = portableValidatesRunnerTask("Docker", true)
-project.ext.validatesPortableRunnerEmbedded = portableValidatesRunnerTask("Embedded", false)
-
-tasks.register("validatesPortableRunner") {
- dependsOn validatesPortableRunnerDocker
- dependsOn validatesPortableRunnerEmbedded
-}
-
-def testJavaVersion = project.findProperty('testJavaVersion')
-String testJavaHome = null
-if (testJavaVersion) {
- testJavaHome = project.findProperty("java${testJavaVersion}Home")
-}
-
-def jobPort = BeamModulePlugin.getRandomPort()
-def artifactPort = BeamModulePlugin.getRandomPort()
-
-def setupTask = project.tasks.register("samzaJobServerSetup", Exec) {
- dependsOn shadowJar
- def pythonDir = project.project(":sdks:python").projectDir
- def samzaJobServerJar = shadowJar.archivePath
- if (testJavaHome) {
- environment "JAVA_HOME", testJavaHome
- }
- executable 'sh'
- args '-c', "$pythonDir/scripts/run_job_server.sh stop --group_id ${project.name} && $pythonDir/scripts/run_job_server.sh start --group_id ${project.name} --job_port ${jobPort} --artifact_port ${artifactPort} --job_server_jar ${samzaJobServerJar}"
-}
-
-def cleanupTask = project.tasks.register("samzaJobServerCleanup", Exec) {
- def pythonDir = project.project(":sdks:python").projectDir
- if (testJavaHome) {
- environment "JAVA_HOME", testJavaHome
- }
- executable 'sh'
- args '-c', "$pythonDir/scripts/run_job_server.sh stop --group_id ${project.name}"
-}
-
-createCrossLanguageValidatesRunnerTask(
- startJobServer: setupTask,
- cleanupJobServer: cleanupTask,
- classpath: configurations.validatesPortableRunner,
- numParallelTests: 1,
- pythonPipelineOptions: [
- "--runner=PortableRunner",
- "--job_endpoint=localhost:${jobPort}",
- "--environment_cache_millis=10000",
- "--experiments=beam_fn_api",
- ],
- javaPipelineOptions: [
- "--runner=PortableRunner",
- "--jobEndpoint=localhost:${jobPort}",
- "--environmentCacheMillis=10000",
- "--experiments=beam_fn_api",
- "--customBeamRequirement=${project.project(":sdks:python").projectDir}/build/apache-beam.tar.gz",
- ],
- goScriptOptions: [
- "--runner samza",
- "--tests \"./test/integration/xlang ./test/integration/io/xlang/...\"",
- "--endpoint localhost:${jobPort}",
- ],
-)
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionContext.java
deleted file mode 100644
index 18bb098bf2cc..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionContext.java
+++ /dev/null
@@ -1,72 +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.samza;
-
-import org.apache.beam.runners.samza.metrics.SamzaMetricsContainer;
-import org.apache.samza.context.ApplicationContainerContext;
-import org.apache.samza.context.ApplicationContainerContextFactory;
-import org.apache.samza.context.ContainerContext;
-import org.apache.samza.context.ExternalContext;
-import org.apache.samza.context.JobContext;
-import org.apache.samza.metrics.MetricsRegistryMap;
-
-/** Runtime context for the Samza runner. */
-@SuppressWarnings({
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class SamzaExecutionContext implements ApplicationContainerContext {
-
- private final SamzaPipelineOptions options;
- private SamzaMetricsContainer metricsContainer;
-
- public SamzaExecutionContext(SamzaPipelineOptions options) {
- this.options = options;
- }
-
- public SamzaPipelineOptions getPipelineOptions() {
- return options;
- }
-
- public SamzaMetricsContainer getMetricsContainer() {
- return this.metricsContainer;
- }
-
- void setMetricsContainer(SamzaMetricsContainer metricsContainer) {
- this.metricsContainer = metricsContainer;
- }
-
- @Override
- public void start() {}
-
- @Override
- public void stop() {}
-
- /** The factory to return this {@link SamzaExecutionContext}. */
- public class Factory implements ApplicationContainerContextFactory {
-
- @Override
- public SamzaExecutionContext create(
- ExternalContext externalContext, JobContext jobContext, ContainerContext containerContext) {
-
- final MetricsRegistryMap metricsRegistry =
- (MetricsRegistryMap) containerContext.getContainerMetricsRegistry();
- SamzaExecutionContext.this.setMetricsContainer(new SamzaMetricsContainer(metricsRegistry));
- return SamzaExecutionContext.this;
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionEnvironment.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionEnvironment.java
deleted file mode 100644
index 02c31edfd1dd..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionEnvironment.java
+++ /dev/null
@@ -1,44 +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.samza;
-
-/** Different Samza execution environments that defines how the Samza job will be deployed. */
-public enum SamzaExecutionEnvironment {
- /**
- * Runs the Samza job on the local machine with only one container. There is no coordination
- * required since there is only one container deployed in a single JVM. This setting is generally
- * used for development and testing.
- */
- LOCAL,
-
- /**
- * Submits and runs the Samza job on YARN, a remote clustered resource manager. Samza works with
- * the YARN to provision and coordinate resources for your application and run it across a cluster
- * of machines. It also handles failures of individual instances and automatically restarts them.
- */
- YARN,
-
- /**
- * Runs Samza job as a stand alone embedded library mode which can be imported into your Java
- * application. You can increase your application's capacity by spinning up multiple instances.
- * These instances will then dynamically coordinate with each other and distribute work among
- * themselves. If an instance fails, the tasks running on it will be re-assigned to the remaining
- * ones. By default, Samza uses Zookeeper for coordination across individual instances.
- */
- STANDALONE
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java
deleted file mode 100644
index 6638b35f377d..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.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.samza;
-
-import java.util.UUID;
-import javax.annotation.Nullable;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
-import org.apache.beam.runners.jobsubmission.JobInvocation;
-import org.apache.beam.runners.jobsubmission.JobInvoker;
-import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator;
-import org.apache.beam.runners.jobsubmission.PortablePipelineRunner;
-import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation;
-import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.Struct;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ListeningExecutorService;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings({
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class SamzaJobInvoker extends JobInvoker {
-
- private static final Logger LOG = LoggerFactory.getLogger(SamzaJobInvoker.class);
-
- public static SamzaJobInvoker create(
- SamzaJobServerDriver.SamzaServerConfiguration configuration) {
- return new SamzaJobInvoker();
- }
-
- private SamzaJobInvoker() {
- this("samza-runner-job-invoker-%d");
- }
-
- protected SamzaJobInvoker(String name) {
- super(name);
- }
-
- @Override
- protected JobInvocation invokeWithExecutor(
- RunnerApi.Pipeline pipeline,
- Struct options,
- @Nullable String retrievalToken,
- ListeningExecutorService executorService) {
- LOG.trace("Parsing pipeline options");
- final SamzaPortablePipelineOptions samzaOptions =
- PipelineOptionsTranslation.fromProto(options).as(SamzaPortablePipelineOptions.class);
-
- final PortablePipelineRunner pipelineRunner;
- if (Strings.isNullOrEmpty(samzaOptions.getOutputExecutablePath())) {
- pipelineRunner = new SamzaPipelineRunner(samzaOptions);
- } else {
- /*
- * To support --output_executable_path where bundles the input pipeline along with all
- * artifacts, etc. required to run the pipeline into a jar that can be executed later.
- */
- pipelineRunner = new PortablePipelineJarCreator(SamzaPipelineRunner.class);
- }
-
- final String invocationId =
- String.format("%s_%s", samzaOptions.getJobName(), UUID.randomUUID().toString());
- final JobInfo jobInfo =
- JobInfo.create(invocationId, samzaOptions.getJobName(), retrievalToken, options);
- return new JobInvocation(jobInfo, executorService, pipeline, pipelineRunner);
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobServerDriver.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobServerDriver.java
deleted file mode 100644
index f8139c0d26f2..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobServerDriver.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.samza;
-
-import org.apache.beam.runners.jobsubmission.JobServerDriver;
-import org.apache.beam.sdk.fn.server.ServerFactory;
-import org.apache.beam.sdk.io.FileSystems;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.kohsuke.args4j.CmdLineException;
-import org.kohsuke.args4j.CmdLineParser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Driver program that starts a job server for the Samza runner. */
-public class SamzaJobServerDriver extends JobServerDriver {
-
- private static final Logger LOG = LoggerFactory.getLogger(SamzaJobServerDriver.class);
-
- /** Samza runner-specific Configuration for the jobServer. */
- public static class SamzaServerConfiguration extends ServerConfiguration {}
-
- public static void main(String[] args) {
- // TODO: Expose the fileSystem related options.
- PipelineOptions options = PipelineOptionsFactory.create();
- // Register standard file systems.
- FileSystems.setDefaultPipelineOptions(options);
- fromParams(args).run();
- }
-
- private static SamzaJobServerDriver fromParams(String[] args) {
- return fromConfig(parseArgs(args));
- }
-
- private static void printUsage(CmdLineParser parser) {
- System.err.printf("Usage: java %s arguments...%n", SamzaJobServerDriver.class.getSimpleName());
- parser.printUsage(System.err);
- System.err.println();
- }
-
- private static SamzaJobServerDriver fromConfig(SamzaServerConfiguration configuration) {
- return create(
- configuration,
- createJobServerFactory(configuration),
- createArtifactServerFactory(configuration));
- }
-
- public static SamzaServerConfiguration parseArgs(String[] args) {
- SamzaServerConfiguration configuration = new SamzaServerConfiguration();
- CmdLineParser parser = new CmdLineParser(configuration);
- try {
- parser.parseArgument(args);
- } catch (CmdLineException e) {
- LOG.error("Unable to parse command line arguments.", e);
- printUsage(parser);
- throw new IllegalArgumentException("Unable to parse command line arguments.", e);
- }
- return configuration;
- }
-
- private static SamzaJobServerDriver create(
- SamzaServerConfiguration configuration,
- ServerFactory jobServerFactory,
- ServerFactory artifactServerFactory) {
- return new SamzaJobServerDriver(configuration, jobServerFactory, artifactServerFactory);
- }
-
- private SamzaJobServerDriver(
- SamzaServerConfiguration configuration,
- ServerFactory jobServerFactory,
- ServerFactory artifactServerFactory) {
- this(
- configuration,
- jobServerFactory,
- artifactServerFactory,
- () -> SamzaJobInvoker.create(configuration));
- }
-
- protected SamzaJobServerDriver(
- ServerConfiguration configuration,
- ServerFactory jobServerFactory,
- ServerFactory artifactServerFactory,
- JobInvokerFactory jobInvokerFactory) {
- super(configuration, jobServerFactory, artifactServerFactory, jobInvokerFactory);
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineExceptionContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineExceptionContext.java
deleted file mode 100644
index 5bd02b3fc737..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineExceptionContext.java
+++ /dev/null
@@ -1,37 +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.samza;
-
-/** Helper that is used to metadata associated with an exception thrown by Samza Runner. */
-public class SamzaPipelineExceptionContext {
- private final String transformFullName;
- private final Exception exception;
-
- public SamzaPipelineExceptionContext(String transformFullName, Exception exception) {
- this.transformFullName = transformFullName;
- this.exception = exception;
- }
-
- public String getTransformFullName() {
- return transformFullName;
- }
-
- public Exception getException() {
- return exception;
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineLifeCycleListener.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineLifeCycleListener.java
deleted file mode 100644
index 47f36a229ac1..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineLifeCycleListener.java
+++ /dev/null
@@ -1,44 +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.samza;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.context.ExternalContext;
-
-/** Life cycle listener for a Samza pipeline during runtime. */
-public interface SamzaPipelineLifeCycleListener {
- /** Callback when the pipeline options is created. */
- void onInit(Config config, SamzaPipelineOptions options);
-
- /** Callback when the pipeline is started. */
- ExternalContext onStart();
-
- /**
- * Callback after the pipeline is submmitted. This will be invoked only for Samza jobs submitted
- * to a cluster.
- */
- void onSubmit();
-
- /** Callback after the pipeline is finished. */
- void onFinish();
-
- /** A registrar for {@link SamzaPipelineLifeCycleListener}. */
- interface Registrar {
- SamzaPipelineLifeCycleListener getLifeCycleListener();
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptions.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptions.java
deleted file mode 100644
index a34303d92552..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptions.java
+++ /dev/null
@@ -1,182 +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.samza;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.Hidden;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.samza.config.ConfigLoaderFactory;
-import org.apache.samza.config.loaders.PropertiesConfigLoaderFactory;
-import org.apache.samza.metrics.MetricsReporter;
-
-/** Options which can be used to configure a Samza PortablePipelineRunner. */
-public interface SamzaPipelineOptions extends PipelineOptions {
-
- @Description(
- "The config file for Samza. It is *optional*. By default Samza supports properties config."
- + "Without a config file, Samza uses a default config for local execution.")
- String getConfigFilePath();
-
- void setConfigFilePath(String filePath);
-
- @Description("The factory to read config file from config file path.")
- @Default.Class(PropertiesConfigLoaderFactory.class)
- Class extends ConfigLoaderFactory> getConfigLoaderFactory();
-
- void setConfigLoaderFactory(Class extends ConfigLoaderFactory> configLoaderFactory);
-
- @Description(
- "The config override to set programmatically. It will be applied on "
- + "top of config file if it exits, otherwise used directly as the config.")
- Map getConfigOverride();
-
- void setConfigOverride(Map configs);
-
- @Description("The instance name of the job")
- @Default.String("1")
- String getJobInstance();
-
- void setJobInstance(String instance);
-
- @Description(
- "Samza application execution environment."
- + "See {@link org.apache.beam.runners.samza.SamzaExecutionEnvironment} for detailed environment descriptions.")
- @Default.Enum("LOCAL")
- SamzaExecutionEnvironment getSamzaExecutionEnvironment();
-
- void setSamzaExecutionEnvironment(SamzaExecutionEnvironment environment);
-
- @Description("The interval to check for watermarks in milliseconds.")
- @Default.Long(1000)
- long getWatermarkInterval();
-
- void setWatermarkInterval(long interval);
-
- @Description("The maximum number of messages to buffer for a given system.")
- @Default.Integer(5000)
- int getSystemBufferSize();
-
- void setSystemBufferSize(int consumerBufferSize);
-
- @Description("The maximum number of event-time timers to buffer in memory for a PTransform")
- @Default.Integer(50000)
- int getEventTimerBufferSize();
-
- void setEventTimerBufferSize(int eventTimerBufferSize);
-
- @Description("The maximum number of ready timers to process at once per watermark.")
- @Default.Integer(Integer.MAX_VALUE)
- int getMaxReadyTimersToProcessOnce();
-
- void setMaxReadyTimersToProcessOnce(int maxReadyTimersToProcessOnce);
-
- @Description("The maximum parallelism allowed for any data source.")
- @Default.Integer(1)
- int getMaxSourceParallelism();
-
- void setMaxSourceParallelism(int maxSourceParallelism);
-
- @Description("The batch get size limit for the state store.")
- @Default.Integer(10000)
- int getStoreBatchGetSize();
-
- void setStoreBatchGetSize(int storeBatchGetSize);
-
- @Description("Enable/disable Beam metrics in Samza Runner")
- @Default.Boolean(true)
- Boolean getEnableMetrics();
-
- void setEnableMetrics(Boolean enableMetrics);
-
- @Description("Enable/disable Beam Transform throughput, latency metrics in Samza Runner")
- @Default.Boolean(false)
- Boolean getEnableTransformMetrics();
-
- void setEnableTransformMetrics(Boolean enableMetrics);
-
- @Description("The config for state to be durable")
- @Default.Boolean(false)
- Boolean getStateDurable();
-
- void setStateDurable(Boolean stateDurable);
-
- @JsonIgnore
- @Description("The metrics reporters that will be used to emit metrics.")
- List getMetricsReporters();
-
- void setMetricsReporters(List reporters);
-
- @Description("The maximum number of elements in a bundle.")
- @Default.Long(1)
- long getMaxBundleSize();
-
- void setMaxBundleSize(long maxBundleSize);
-
- @Description("The maximum time to wait before finalising a bundle (in milliseconds).")
- @Default.Long(1000)
- long getMaxBundleTimeMs();
-
- void setMaxBundleTimeMs(long maxBundleTimeMs);
-
- @Description(
- "Wait if necessary for completing a remote bundle processing for at most the given time (in milliseconds). if the value of timeout is negative, wait forever until the bundle processing is completed. Used only in portable mode for now.")
- @Default.Long(-1)
- long getBundleProcessingTimeout();
-
- void setBundleProcessingTimeout(long timeoutMs);
-
- @Description(
- "The number of threads to run DoFn.processElements in parallel within a bundle. Used only in non-portable mode.")
- @Default.Integer(1)
- int getNumThreadsForProcessElement();
-
- void setNumThreadsForProcessElement(int numThreads);
-
- @JsonIgnore
- @Description(
- "The ExecutorService instance to run DoFN.processElements in parallel within a bundle. Used only in non-portable mode.")
- @Default.InstanceFactory(ProcessElementExecutorServiceFactory.class)
- @Hidden
- ExecutorService getExecutorServiceForProcessElement();
-
- void setExecutorServiceForProcessElement(ExecutorService executorService);
-
- class ProcessElementExecutorServiceFactory implements DefaultValueFactory {
-
- @Override
- public ExecutorService create(PipelineOptions options) {
- return Executors.newFixedThreadPool(
- options.as(SamzaPipelineOptions.class).getNumThreadsForProcessElement(),
- new ThreadFactoryBuilder().setNameFormat("Process Element Thread-%d").build());
- }
- }
-
- @Description("Enable/disable late data dropping in GroupByKey/Combine transforms")
- @Default.Boolean(false)
- boolean getDropLateData();
-
- void setDropLateData(boolean dropLateData);
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptionsValidator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptionsValidator.java
deleted file mode 100644
index 1db0974b5d30..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptionsValidator.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.samza;
-
-import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.samza.config.JobConfig.JOB_CONTAINER_THREAD_POOL_SIZE;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.samza.config.JobConfig;
-import org.apache.samza.config.MapConfig;
-
-/** Validates that the {@link SamzaPipelineOptions} conforms to all the criteria. */
-public class SamzaPipelineOptionsValidator {
- public static void validate(SamzaPipelineOptions opts) {
- checkArgument(opts.getMaxSourceParallelism() >= 1);
- validateBundlingRelatedOptions(opts);
- }
-
- /*
- * Perform some bundling related validation for pipeline option.
- * Visible for testing.
- */
- static void validateBundlingRelatedOptions(SamzaPipelineOptions pipelineOptions) {
- if (pipelineOptions.getMaxBundleSize() > 1) {
- final Map configs =
- pipelineOptions.getConfigOverride() == null
- ? new HashMap<>()
- : pipelineOptions.getConfigOverride();
- final JobConfig jobConfig = new JobConfig(new MapConfig(configs));
-
- // Validate that the threadPoolSize is not override in the code
- checkArgument(
- jobConfig.getThreadPoolSize() <= 1,
- JOB_CONTAINER_THREAD_POOL_SIZE
- + " config should be replaced with SamzaPipelineOptions.numThreadsForProcessElement");
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java
deleted file mode 100644
index e84cf086edc9..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java
+++ /dev/null
@@ -1,161 +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.samza;
-
-import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults;
-import static org.apache.samza.config.TaskConfig.TASK_SHUTDOWN_MS;
-
-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.samza.config.Config;
-import org.apache.samza.job.ApplicationStatus;
-import org.apache.samza.runtime.ApplicationRunner;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import org.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** The result from executing a Samza Pipeline. */
-@SuppressWarnings({
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class SamzaPipelineResult implements PipelineResult {
- private static final Logger LOG = LoggerFactory.getLogger(SamzaPipelineResult.class);
- // allow some buffer on top of samza's own shutdown timeout
- private static final long SHUTDOWN_TIMEOUT_BUFFER = 5000L;
- private static final long DEFAULT_TASK_SHUTDOWN_MS = 30000L;
-
- private final SamzaExecutionContext executionContext;
- private final ApplicationRunner runner;
- private final SamzaPipelineLifeCycleListener listener;
- private final long shutdownTiemoutMs;
-
- public SamzaPipelineResult(
- ApplicationRunner runner,
- SamzaExecutionContext executionContext,
- SamzaPipelineLifeCycleListener listener,
- Config config) {
- this.executionContext = executionContext;
- this.runner = runner;
- this.listener = listener;
- this.shutdownTiemoutMs =
- config.getLong(TASK_SHUTDOWN_MS, DEFAULT_TASK_SHUTDOWN_MS) + SHUTDOWN_TIMEOUT_BUFFER;
- }
-
- @Override
- public State getState() {
- return getStateInfo().state;
- }
-
- @Override
- public State cancel() {
- LOG.info("Start to cancel samza pipeline...");
- runner.kill();
- LOG.info("Start awaiting finish for {} ms.", shutdownTiemoutMs);
- return waitUntilFinish(Duration.millis(shutdownTiemoutMs));
- }
-
- @Override
- public State waitUntilFinish(@Nullable Duration duration) {
- try {
- if (duration == null) {
- runner.waitForFinish();
- } else {
- runner.waitForFinish(java.time.Duration.ofMillis(duration.getMillis()));
- }
- } catch (Exception e) {
- throw new Pipeline.PipelineExecutionException(e);
- }
-
- final StateInfo stateInfo = getStateInfo();
-
- if (listener != null && (stateInfo.state == State.DONE || stateInfo.state == State.FAILED)) {
- listener.onFinish();
- }
-
- if (stateInfo.state == State.FAILED) {
- throw stateInfo.error;
- }
-
- LOG.info("Pipeline finished. Final state: {}", stateInfo.state);
- return stateInfo.state;
- }
-
- @Override
- public State waitUntilFinish() {
- return waitUntilFinish(null);
- }
-
- @Override
- public MetricResults metrics() {
- return asAttemptedOnlyMetricResults(executionContext.getMetricsContainer().getContainers());
- }
-
- @SuppressWarnings("Slf4jDoNotLogMessageOfExceptionExplicitly")
- private StateInfo getStateInfo() {
- final ApplicationStatus status = runner.status();
- switch (status.getStatusCode()) {
- case New:
- return new StateInfo(State.STOPPED);
- case Running:
- return new StateInfo(State.RUNNING);
- case SuccessfulFinish:
- return new StateInfo(State.DONE);
- case UnsuccessfulFinish:
- LOG.error("Pipeline execution failed", status.getThrowable());
- return new StateInfo(
- State.FAILED,
- new Pipeline.PipelineExecutionException(getUserCodeException(status.getThrowable())));
- default:
- return new StateInfo(State.UNKNOWN);
- }
- }
-
- private static class StateInfo {
- private final State state;
- private final Pipeline.PipelineExecutionException error;
-
- private StateInfo(State state) {
- this(state, null);
- }
-
- private StateInfo(State state, Pipeline.PipelineExecutionException error) {
- this.state = state;
- this.error = error;
- }
- }
-
- /**
- * Some of the Beam unit tests relying on the exception message to do assertion. This function
- * will find the original UserCodeException so the message will be exposed directly.
- */
- private static Throwable getUserCodeException(Throwable throwable) {
- Throwable t = throwable;
- while (t != null) {
- if (t instanceof UserCodeException) {
- return t;
- }
-
- t = t.getCause();
- }
-
- return throwable;
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.java
deleted file mode 100644
index 897b78cf9e47..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineRunner.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.samza;
-
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
-import org.apache.beam.runners.jobsubmission.PortablePipelineResult;
-import org.apache.beam.runners.jobsubmission.PortablePipelineRunner;
-import org.apache.beam.runners.samza.translation.SamzaPortablePipelineTranslator;
-import org.apache.beam.sdk.util.construction.PTransformTranslation;
-import org.apache.beam.sdk.util.construction.graph.ExecutableStage;
-import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser;
-import org.apache.beam.sdk.util.construction.graph.ProtoOverrides;
-import org.apache.beam.sdk.util.construction.graph.SplittableParDoExpander;
-import org.apache.beam.sdk.util.construction.graph.TrivialNativeTransformExpander;
-import org.apache.beam.sdk.util.construction.renderer.PipelineDotRenderer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Runs a Samza job via {@link SamzaRunner}. */
-public class SamzaPipelineRunner implements PortablePipelineRunner {
-
- private static final Logger LOG = LoggerFactory.getLogger(SamzaPipelineRunner.class);
-
- private final SamzaPipelineOptions options;
-
- @Override
- public PortablePipelineResult run(final RunnerApi.Pipeline pipeline, JobInfo jobInfo) {
- // Expand any splittable DoFns within the graph to enable sizing and splitting of bundles.
- RunnerApi.Pipeline pipelineWithSdfExpanded =
- ProtoOverrides.updateTransform(
- PTransformTranslation.PAR_DO_TRANSFORM_URN,
- pipeline,
- SplittableParDoExpander.createSizedReplacement());
-
- // Don't let the fuser fuse any subcomponents of native transforms.
- RunnerApi.Pipeline trimmedPipeline =
- TrivialNativeTransformExpander.forKnownUrns(
- pipelineWithSdfExpanded, SamzaPortablePipelineTranslator.knownUrns());
-
- // Fused pipeline proto.
- // TODO: Consider supporting partially-fused graphs.
- RunnerApi.Pipeline fusedPipeline =
- trimmedPipeline.getComponents().getTransformsMap().values().stream()
- .anyMatch(proto -> ExecutableStage.URN.equals(proto.getSpec().getUrn()))
- ? trimmedPipeline
- : GreedyPipelineFuser.fuse(trimmedPipeline).toPipeline();
-
- LOG.info("Portable pipeline to run:");
- LOG.info("{}", PipelineDotRenderer.toDotString(fusedPipeline));
- // the pipeline option coming from sdk will set the sdk specific runner which will break
- // serialization
- // so we need to reset the runner here to a valid Java runner
- options.setRunner(SamzaRunner.class);
- try {
- final SamzaRunner runner = SamzaRunner.fromOptions(options);
- final PortablePipelineResult result = runner.runPortablePipeline(fusedPipeline, jobInfo);
-
- final SamzaExecutionEnvironment exeEnv = options.getSamzaExecutionEnvironment();
- if (exeEnv == SamzaExecutionEnvironment.LOCAL
- || exeEnv == SamzaExecutionEnvironment.STANDALONE) {
- // Make run() sync for local mode
- result.waitUntilFinish();
- }
- return result;
- } catch (Exception e) {
- throw new RuntimeException("Failed to invoke samza job", e);
- }
- }
-
- public SamzaPipelineRunner(SamzaPipelineOptions options) {
- this.options = options;
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineOptions.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineOptions.java
deleted file mode 100644
index aa8e7ceb71d7..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineOptions.java
+++ /dev/null
@@ -1,32 +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.samza;
-
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PortablePipelineOptions;
-
-/** Samza pipeline option that contains portability specific logic. For internal usage only. */
-public interface SamzaPortablePipelineOptions
- extends SamzaPipelineOptions, PortablePipelineOptions {
- @Description(
- "The file path for the local file system token. If not set (by default), then the runner would"
- + " not use secure server factory.")
- String getFsTokenPath();
-
- void setFsTokenPath(String path);
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineResult.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineResult.java
deleted file mode 100644
index a3452097f511..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineResult.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.samza;
-
-import org.apache.beam.model.jobmanagement.v1.JobApi;
-import org.apache.beam.runners.jobsubmission.PortablePipelineResult;
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.runtime.ApplicationRunner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** The result from executing a Samza Portable Pipeline. */
-public class SamzaPortablePipelineResult extends SamzaPipelineResult
- implements PortablePipelineResult {
-
- private static final Logger LOG = LoggerFactory.getLogger(SamzaPortablePipelineResult.class);
-
- SamzaPortablePipelineResult(
- StreamApplication app,
- ApplicationRunner runner,
- SamzaExecutionContext executionContext,
- SamzaPipelineLifeCycleListener listener,
- Config config) {
- super(runner, executionContext, listener, config);
- }
-
- @Override
- public JobApi.MetricResults portableMetrics() throws UnsupportedOperationException {
- LOG.warn("Collecting monitoring infos is not implemented yet in Samza portable runner.");
- return JobApi.MetricResults.newBuilder().build();
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java
deleted file mode 100644
index eb16faa41ac0..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java
+++ /dev/null
@@ -1,213 +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.samza;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.ServiceLoader;
-import java.util.Set;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
-import org.apache.beam.runners.jobsubmission.PortablePipelineResult;
-import org.apache.beam.runners.samza.translation.ConfigBuilder;
-import org.apache.beam.runners.samza.translation.ConfigContext;
-import org.apache.beam.runners.samza.translation.PViewToIdMapper;
-import org.apache.beam.runners.samza.translation.PortableTranslationContext;
-import org.apache.beam.runners.samza.translation.SamzaPipelineTranslator;
-import org.apache.beam.runners.samza.translation.SamzaPortablePipelineTranslator;
-import org.apache.beam.runners.samza.translation.SamzaTransformOverrides;
-import org.apache.beam.runners.samza.translation.StateIdParser;
-import org.apache.beam.runners.samza.translation.TranslationContext;
-import org.apache.beam.runners.samza.util.PipelineJsonRenderer;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineRunner;
-import org.apache.beam.sdk.metrics.MetricsEnvironment;
-import org.apache.beam.sdk.options.ExperimentalOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.util.construction.SplittableParDo;
-import org.apache.beam.sdk.util.construction.renderer.PipelineDotRenderer;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators;
-import org.apache.samza.application.StreamApplication;
-import org.apache.samza.config.Config;
-import org.apache.samza.context.ExternalContext;
-import org.apache.samza.metrics.MetricsReporter;
-import org.apache.samza.metrics.MetricsReporterFactory;
-import org.apache.samza.runtime.ApplicationRunner;
-import org.apache.samza.runtime.ApplicationRunners;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A {@link PipelineRunner} that executes the operations in the {@link Pipeline} into an equivalent
- * Samza plan.
- *
- * @deprecated The support for Samza is scheduled for removal in Beam 3.0.
- */
-@SuppressWarnings({
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-@Deprecated
-public class SamzaRunner extends PipelineRunner {
- private static final Logger LOG = LoggerFactory.getLogger(SamzaRunner.class);
- private static final String BEAM_DOT_GRAPH = "beamDotGraph";
- public static final String BEAM_JSON_GRAPH = "beamJsonGraph";
-
- public static SamzaRunner fromOptions(PipelineOptions opts) {
- final SamzaPipelineOptions samzaOptions =
- PipelineOptionsValidator.validate(SamzaPipelineOptions.class, opts);
- return new SamzaRunner(samzaOptions);
- }
-
- private final SamzaPipelineOptions options;
- private final SamzaPipelineLifeCycleListener listener;
-
- private SamzaRunner(SamzaPipelineOptions options) {
- this.options = options;
- final Iterator listenerReg =
- ServiceLoader.load(SamzaPipelineLifeCycleListener.Registrar.class).iterator();
- this.listener =
- listenerReg.hasNext() ? Iterators.getOnlyElement(listenerReg).getLifeCycleListener() : null;
- }
-
- public PortablePipelineResult runPortablePipeline(RunnerApi.Pipeline pipeline, JobInfo jobInfo) {
- final String dotGraph = PipelineDotRenderer.toDotString(pipeline);
- LOG.info("Portable pipeline to run DOT graph:\n{}", dotGraph);
-
- final ConfigBuilder configBuilder = new ConfigBuilder(options);
- SamzaPortablePipelineTranslator.createConfig(pipeline, configBuilder, options);
- configBuilder.put(BEAM_DOT_GRAPH, dotGraph);
-
- final Config config = configBuilder.build();
- options.setConfigOverride(config);
-
- if (listener != null) {
- listener.onInit(config, options);
- }
-
- final SamzaExecutionContext executionContext = new SamzaExecutionContext(options);
- final Map reporterFactories = getMetricsReporters();
- final StreamApplication app =
- appDescriptor -> {
- appDescriptor
- .withApplicationContainerContextFactory(executionContext.new Factory())
- .withMetricsReporterFactories(reporterFactories);
- SamzaPortablePipelineTranslator.translate(
- pipeline, new PortableTranslationContext(appDescriptor, options, jobInfo));
- };
-
- ApplicationRunner runner = runSamzaApp(app, config);
- return new SamzaPortablePipelineResult(app, runner, executionContext, listener, config);
- }
-
- @Override
- public SamzaPipelineResult run(Pipeline pipeline) {
- // TODO(https://github.com/apache/beam/issues/20530): Use SDF read as default for non-portable
- // execution when we address performance issue.
- if (!ExperimentalOptions.hasExperiment(pipeline.getOptions(), "beam_fn_api")) {
- SplittableParDo.convertReadBasedSplittableDoFnsToPrimitiveReadsIfNecessary(pipeline);
- }
-
- MetricsEnvironment.setMetricsSupported(true);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Pre-processed Beam pipeline in dot format:\n{}",
- PipelineDotRenderer.toDotString(pipeline));
- }
-
- pipeline.replaceAll(SamzaTransformOverrides.getDefaultOverrides());
- final Map idMap = PViewToIdMapper.buildIdMap(pipeline);
- final Set nonUniqueStateIds = StateIdParser.scan(pipeline);
- final ConfigContext configCtx = new ConfigContext(idMap, nonUniqueStateIds, options);
-
- final String dotGraph = PipelineDotRenderer.toDotString(pipeline);
- LOG.info("Beam pipeline DOT graph:\n{}", dotGraph);
-
- final String jsonGraph = PipelineJsonRenderer.toJsonString(pipeline, configCtx);
- LOG.info("Beam pipeline JSON graph:\n{}", jsonGraph);
-
- final ConfigBuilder configBuilder = new ConfigBuilder(options);
- SamzaPipelineTranslator.createConfig(pipeline, configCtx, configBuilder);
- configBuilder.put(BEAM_DOT_GRAPH, dotGraph);
- configBuilder.put(BEAM_JSON_GRAPH, jsonGraph);
-
- final Config config = configBuilder.build();
- options.setConfigOverride(config);
-
- if (listener != null) {
- listener.onInit(config, options);
- }
-
- final SamzaExecutionContext executionContext = new SamzaExecutionContext(options);
- final Map reporterFactories = getMetricsReporters();
-
- final StreamApplication app =
- appDescriptor -> {
- appDescriptor.withApplicationContainerContextFactory(executionContext.new Factory());
- appDescriptor.withMetricsReporterFactories(reporterFactories);
-
- SamzaPipelineTranslator.translate(
- pipeline, new TranslationContext(appDescriptor, idMap, nonUniqueStateIds, options));
- };
-
- // perform a final round of validation for the pipeline options now that all configs are
- // generated
- SamzaPipelineOptionsValidator.validate(options);
- ApplicationRunner runner = runSamzaApp(app, config);
- return new SamzaPipelineResult(runner, executionContext, listener, config);
- }
-
- private Map getMetricsReporters() {
- if (options.getMetricsReporters() != null) {
- final Map reporters = new HashMap<>();
- for (int i = 0; i < options.getMetricsReporters().size(); i++) {
- final String name = "beam-metrics-reporter-" + i;
- final MetricsReporter reporter = options.getMetricsReporters().get(i);
-
- reporters.put(name, (MetricsReporterFactory) (nm, processorId, config) -> reporter);
- LOG.info("{}: {}", name, reporter.getClass().getName());
- }
- return reporters;
- } else {
- return Collections.emptyMap();
- }
- }
-
- private ApplicationRunner runSamzaApp(StreamApplication app, Config config) {
-
- final ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config);
-
- ExternalContext externalContext = null;
- if (listener != null) {
- externalContext = listener.onStart();
- }
-
- runner.run(externalContext);
-
- if (listener != null
- && options.getSamzaExecutionEnvironment() == SamzaExecutionEnvironment.YARN) {
- listener.onSubmit();
- }
-
- return runner;
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerOverrideConfigs.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerOverrideConfigs.java
deleted file mode 100644
index 4c5fa432ca82..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerOverrideConfigs.java
+++ /dev/null
@@ -1,85 +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.samza;
-
-import java.time.Duration;
-
-// TODO: can we get rid of this class? Right now the SamzaPipelineOptionsValidator would force
-// the pipeline option to be the type SamzaPipelineOption. Ideally, we should be able to keep
-// passing SamzaPortablePipelineOption. Alternative, we could merge portable and non-portable
-// pipeline option.
-/** A helper class for holding all the beam runner specific samza configs. */
-@SuppressWarnings({
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class SamzaRunnerOverrideConfigs {
- public static final String BEAM_RUNNER_CONFIG_PREFIX = "beam.override.";
- // whether the job is in portable mode
- public static final String IS_PORTABLE_MODE = BEAM_RUNNER_CONFIG_PREFIX + "portable";
- // for portable mode only: port number for fn control api
- public static final String FN_CONTROL_PORT = BEAM_RUNNER_CONFIG_PREFIX + "control.port";
- // timeout for waiting for control client to connect
- public static final String CONTROL_CLIENT_MAX_WAIT_TIME_MS = "controL.wait.time.ms";
- public static final long DEFAULT_CONTROL_CLIENT_MAX_WAIT_TIME_MS =
- Duration.ofMinutes(2).toMillis();
- public static final String FS_TOKEN_PATH = BEAM_RUNNER_CONFIG_PREFIX + "fs.token.path";
- public static final String DEFAULT_FS_TOKEN_PATH = null;
-
- private static boolean containsKey(SamzaPipelineOptions options, String configKey) {
- if (options == null || options.getConfigOverride() == null) {
- return false;
- }
- return options.getConfigOverride().containsKey(configKey);
- }
-
- /** Whether the job is in portable mode based on the config override in the pipeline options. */
- public static boolean isPortableMode(SamzaPipelineOptions options) {
- if (containsKey(options, IS_PORTABLE_MODE)) {
- return options.getConfigOverride().get(IS_PORTABLE_MODE).equals(String.valueOf(true));
- } else {
- return false;
- }
- }
-
- /** Get fn control port number based on the config override in the pipeline options. */
- public static int getFnControlPort(SamzaPipelineOptions options) {
- if (containsKey(options, FN_CONTROL_PORT)) {
- return Integer.parseInt(options.getConfigOverride().get(FN_CONTROL_PORT));
- } else {
- return -1;
- }
- }
-
- /** Get max wait time for control client connection. */
- public static long getControlClientWaitTimeoutMs(SamzaPipelineOptions options) {
- if (containsKey(options, CONTROL_CLIENT_MAX_WAIT_TIME_MS)) {
- return Long.parseLong(options.getConfigOverride().get(CONTROL_CLIENT_MAX_WAIT_TIME_MS));
- } else {
- return DEFAULT_CONTROL_CLIENT_MAX_WAIT_TIME_MS;
- }
- }
-
- /** Get fs token path for portable mode. */
- public static String getFsTokenPath(SamzaPipelineOptions options) {
- if (containsKey(options, FS_TOKEN_PATH)) {
- return options.getConfigOverride().get(FS_TOKEN_PATH);
- } else {
- return DEFAULT_FS_TOKEN_PATH;
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerRegistrar.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerRegistrar.java
deleted file mode 100644
index 102838975d13..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerRegistrar.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.samza;
-
-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.v32_1_2_jre.com.google.common.collect.ImmutableList;
-
-/**
- * AutoService registrar - will register SamzaRunner and SamzaOptions as possible pipeline runner
- * services.
- *
- *
It ends up in META-INF/services and gets picked up by Beam.
- */
-public class SamzaRunnerRegistrar {
- private SamzaRunnerRegistrar() {}
-
- /** Pipeline runner registrar. */
- @AutoService(PipelineRunnerRegistrar.class)
- public static class Runner implements PipelineRunnerRegistrar {
- @Override
- public Iterable>> getPipelineRunners() {
- return ImmutableList.of(SamzaRunner.class, TestSamzaRunner.class);
- }
- }
-
- /** Pipeline options registrar. */
- @AutoService(PipelineOptionsRegistrar.class)
- public static class Options implements PipelineOptionsRegistrar {
- @Override
- public Iterable> getPipelineOptions() {
- return ImmutableList.of(SamzaPipelineOptions.class);
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/TestSamzaRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/TestSamzaRunner.java
deleted file mode 100644
index 810fc0c983f7..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/TestSamzaRunner.java
+++ /dev/null
@@ -1,109 +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.samza;
-
-import static org.apache.samza.config.JobConfig.JOB_JMX_ENABLED;
-import static org.apache.samza.config.JobConfig.JOB_LOGGED_STORE_BASE_DIR;
-import static org.apache.samza.config.JobConfig.JOB_NON_LOGGED_STORE_BASE_DIR;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.io.File;
-import java.nio.file.Files;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.beam.runners.samza.translation.ConfigBuilder;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.PipelineRunner;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.commons.io.FileUtils;
-
-/** Test {@link SamzaRunner}. */
-public class TestSamzaRunner extends PipelineRunner {
-
- private final SamzaRunner delegate;
- private final File storeDir;
-
- public static TestSamzaRunner fromOptions(PipelineOptions options) {
- return new TestSamzaRunner(options);
- }
-
- public static SamzaPipelineOptions createSamzaPipelineOptions(
- PipelineOptions options, File storeDir) {
- try {
- final SamzaPipelineOptions samzaOptions =
- PipelineOptionsValidator.validate(SamzaPipelineOptions.class, options);
- final Map config = new HashMap<>(ConfigBuilder.localRunConfig());
- config.put(JOB_LOGGED_STORE_BASE_DIR, storeDir.getAbsolutePath());
- config.put(JOB_NON_LOGGED_STORE_BASE_DIR, storeDir.getAbsolutePath());
- config.put(JOB_JMX_ENABLED, "false");
-
- if (samzaOptions.getConfigOverride() != null) {
- config.putAll(samzaOptions.getConfigOverride());
- }
- samzaOptions.setConfigOverride(config);
- return samzaOptions;
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- private static File createStoreDir() {
- try {
- return Files.createTempDirectory("beam-samza-test").toFile();
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- public TestSamzaRunner(PipelineOptions options) {
- this.storeDir = createStoreDir();
- this.delegate = SamzaRunner.fromOptions(createSamzaPipelineOptions(options, storeDir));
- }
-
- @Override
- @SuppressFBWarnings(value = "DE_MIGHT_IGNORE")
- public PipelineResult run(Pipeline pipeline) {
- try {
- final PipelineResult result = delegate.run(pipeline);
- result.waitUntilFinish();
-
- return result;
- } catch (Throwable t) {
- // Search for AssertionError. If present use it as the cause of the pipeline failure.
- Throwable current = t;
-
- while (current != null) {
- if (current instanceof AssertionError) {
- throw (AssertionError) current;
- }
- current = current.getCause();
- }
-
- throw t;
- } finally {
- try {
- // delete the store folder
- FileUtils.deleteDirectory(storeDir);
- } catch (Exception ignore) {
- // Ignore
- }
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java
deleted file mode 100644
index 92c9eea4293a..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java
+++ /dev/null
@@ -1,449 +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.samza.adapter;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
-import org.apache.beam.runners.core.serialization.Base64Serializer;
-import org.apache.beam.runners.samza.SamzaPipelineOptions;
-import org.apache.beam.runners.samza.metrics.FnWithMetricsWrapper;
-import org.apache.beam.runners.samza.metrics.SamzaMetricsContainer;
-import org.apache.beam.runners.samza.runtime.OpMessage;
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.WindowedValue;
-import org.apache.beam.sdk.values.WindowedValues;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
-import org.apache.samza.Partition;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.metrics.MetricsRegistry;
-import org.apache.samza.metrics.MetricsRegistryMap;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemAdmin;
-import org.apache.samza.system.SystemConsumer;
-import org.apache.samza.system.SystemFactory;
-import org.apache.samza.system.SystemProducer;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata;
-import org.apache.samza.system.SystemStreamPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A Samza system that supports reading from a Beam {@link BoundedSource}. The source is treated as
- * though it has a single partition and does not support checkpointing via a changelog stream. If
- * the job is restarted the bounded source will be consumed from the beginning.
- */
-// TODO: instrumentation for the consumer
-@SuppressWarnings({
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class BoundedSourceSystem {
- private static final Logger LOG = LoggerFactory.getLogger(BoundedSourceSystem.class);
-
- private static List> split(
- BoundedSource source, SamzaPipelineOptions pipelineOptions) throws Exception {
- final int numSplits = pipelineOptions.getMaxSourceParallelism();
- if (numSplits > 1) {
- final long estimatedSize = source.getEstimatedSizeBytes(pipelineOptions);
- // calculate the size of each split, rounded up to the ceiling.
- final long bundleSize = (estimatedSize + numSplits - 1) / numSplits;
- @SuppressWarnings("unchecked")
- final List> splits =
- (List>) source.split(bundleSize, pipelineOptions);
- // Need the empty check here because Samza doesn't handle empty partition well
- if (!splits.isEmpty()) {
- return splits;
- }
- }
- return Collections.singletonList(source);
- }
-
- /** A {@link SystemAdmin} for {@link BoundedSourceSystem}. */
- public static class Admin implements SystemAdmin {
- private final BoundedSource source;
- private final SamzaPipelineOptions pipelineOptions;
-
- public Admin(BoundedSource source, SamzaPipelineOptions pipelineOptions) {
- this.source = source;
- this.pipelineOptions = pipelineOptions;
- }
-
- @Override
- public Map getOffsetsAfter(
- Map offsets) {
- // BEAM checkpoints the next offset so here we just need to return the map itself
- return offsets;
- }
-
- @Override
- public Map getSystemStreamMetadata(Set streamNames) {
- return streamNames.stream()
- .collect(
- Collectors.toMap(
- Function.identity(),
- streamName -> {
- try {
- List> splits = split(source, pipelineOptions);
- final Map partitionMetaData =
- new HashMap<>();
- // we assume that the generated splits are stable,
- // this is necessary so that the mapping of partition to source is correct
- // in each container.
- for (int i = 0; i < splits.size(); i++) {
- partitionMetaData.put(
- new Partition(i), new SystemStreamPartitionMetadata(null, null, null));
- }
- return new SystemStreamMetadata(streamName, partitionMetaData);
- } catch (Exception e) {
- throw new SamzaException("Fail to read stream metadata", e);
- }
- }));
- }
-
- @Override
- public Integer offsetComparator(String offset1, String offset2) {
- if (offset1 == null) {
- return offset2 == null ? 0 : -1;
- }
-
- if (offset2 == null) {
- return 1;
- }
-
- return Long.valueOf(offset1).compareTo(Long.valueOf(offset2));
- }
- }
-
- /**
- * A {@link SystemConsumer} for a {@link BoundedSource}. See {@link BoundedSourceSystem} for more
- * details.
- */
- public static class Consumer implements SystemConsumer {
- private static final Logger LOG = LoggerFactory.getLogger(Consumer.class);
- private static final AtomicInteger NEXT_ID = new AtomicInteger();
-
- private final List> splits;
- private final SamzaPipelineOptions pipelineOptions;
- private final Map, SystemStreamPartition> readerToSsp = new HashMap<>();
- private final SamzaMetricsContainer metricsContainer;
- private final String stepName;
-
- private ReaderTask readerTask;
-
- Consumer(
- BoundedSource source,
- SamzaPipelineOptions pipelineOptions,
- SamzaMetricsContainer metricsContainer,
- String stepName) {
- try {
- splits = split(source, pipelineOptions);
- } catch (Exception e) {
- throw new SamzaException("Fail to split source", e);
- }
- this.pipelineOptions = pipelineOptions;
- this.metricsContainer = metricsContainer;
- this.stepName = stepName;
- }
-
- @Override
- public void start() {
- if (this.readerToSsp.isEmpty()) {
- throw new IllegalArgumentException(
- "Attempted to call start without assigned system stream partitions");
- }
-
- final int capacity = pipelineOptions.getSystemBufferSize();
- final FnWithMetricsWrapper metricsWrapper =
- pipelineOptions.getEnableMetrics()
- ? new FnWithMetricsWrapper(metricsContainer, stepName)
- : null;
- readerTask = new ReaderTask<>(readerToSsp, capacity, metricsWrapper);
- final Thread thread =
- new Thread(readerTask, "bounded-source-system-consumer-" + NEXT_ID.getAndIncrement());
- thread.start();
- }
-
- @Override
- public void stop() {
- // NOTE: this is not a blocking shutdown
- if (readerTask != null) {
- readerTask.stop();
- }
- }
-
- @Override
- public void register(SystemStreamPartition ssp, String offset) {
- final int partitionId = ssp.getPartition().getPartitionId();
- try {
- final BoundedReader reader = splits.get(partitionId).createReader(pipelineOptions);
- readerToSsp.put(reader, ssp);
- } catch (Exception e) {
- throw new SamzaException("Error while creating source reader for ssp: " + ssp, e);
- }
- }
-
- @Override
- public Map> poll(
- Set systemStreamPartitions, long timeout)
- throws InterruptedException {
- assert !readerToSsp.isEmpty(); // start should be called before poll
-
- final Map> envelopes = new HashMap<>();
- for (SystemStreamPartition ssp : systemStreamPartitions) {
- envelopes.put(ssp, readerTask.getNextMessages(ssp, timeout));
- }
- return envelopes;
- }
-
- private static class ReaderTask implements Runnable {
- private final Map, SystemStreamPartition> readerToSsp;
- private final Map> queues;
- private final Semaphore available;
- private final FnWithMetricsWrapper metricsWrapper;
-
- // NOTE: we do not support recovery with a bounded source (we restart from the beginning),
- // so we do not need to have a way to tie an offset to a position in the bounded source.
- private long offset;
- private volatile Thread readerThread;
- private volatile boolean stopInvoked = false;
- private volatile Exception lastException;
-
- private ReaderTask(
- Map, SystemStreamPartition> readerToSsp,
- int capacity,
- FnWithMetricsWrapper metricsWrapper) {
- this.readerToSsp = readerToSsp;
- this.available = new Semaphore(capacity);
- this.metricsWrapper = metricsWrapper;
-
- final Map> qs =
- new HashMap<>();
- readerToSsp.values().forEach(ssp -> qs.put(ssp, new LinkedBlockingQueue<>()));
- this.queues = ImmutableMap.copyOf(qs);
- }
-
- @Override
- public void run() {
- readerThread = Thread.currentThread();
-
- final Set> availableReaders = new HashSet<>(readerToSsp.keySet());
- try {
- for (BoundedReader reader : readerToSsp.keySet()) {
- boolean hasData = invoke(reader::start);
- if (hasData) {
- enqueueMessage(reader);
- } else {
- enqueueMaxWatermarkAndEndOfStream(reader);
- reader.close();
- availableReaders.remove(reader);
- }
- }
-
- while (!stopInvoked && !availableReaders.isEmpty()) {
- final Iterator> iter = availableReaders.iterator();
- while (iter.hasNext()) {
- final BoundedReader reader = iter.next();
- final boolean hasData = invoke(reader::advance);
- if (hasData) {
- enqueueMessage(reader);
- } else {
- enqueueMaxWatermarkAndEndOfStream(reader);
- reader.close();
- iter.remove();
- }
- }
- }
- } catch (InterruptedException e) {
- // We use an interrupt to wake the reader from a blocking read under normal termination,
- // so ignore it here.
- } catch (Exception e) {
- setError(e);
- } finally {
- availableReaders.forEach(
- reader -> {
- try {
- reader.close();
- } catch (IOException e) {
- LOG.error(
- "Reader task failed to close reader for ssp {}", readerToSsp.get(reader), e);
- }
- });
- }
- }
-
- private X invoke(FnWithMetricsWrapper.SupplierWithException fn) throws Exception {
- if (metricsWrapper != null) {
- return metricsWrapper.wrap(fn, true);
- } else {
- return fn.get();
- }
- }
-
- private void enqueueMessage(BoundedReader reader) throws InterruptedException {
- final T value = reader.getCurrent();
- final WindowedValue windowedValue =
- WindowedValues.timestampedValueInGlobalWindow(value, reader.getCurrentTimestamp());
- final SystemStreamPartition ssp = readerToSsp.get(reader);
- final IncomingMessageEnvelope envelope =
- new IncomingMessageEnvelope(
- ssp, Long.toString(offset++), null, OpMessage.ofElement(windowedValue));
-
- available.acquire();
- queues.get(ssp).put(envelope);
- }
-
- private void enqueueMaxWatermarkAndEndOfStream(BoundedReader reader) {
- final SystemStreamPartition ssp = readerToSsp.get(reader);
- // Send the max watermark to force completion of any open windows.
- final IncomingMessageEnvelope watermarkEnvelope =
- IncomingMessageEnvelope.buildWatermarkEnvelope(
- ssp, BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
- enqueueUninterruptibly(watermarkEnvelope);
-
- final IncomingMessageEnvelope endOfStreamEnvelope =
- IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp);
- enqueueUninterruptibly(endOfStreamEnvelope);
- }
-
- private void stop() {
- stopInvoked = true;
-
- final Thread readerThread = this.readerThread;
- if (readerThread != null) {
- readerThread.interrupt();
- }
- }
-
- private List getNextMessages(
- SystemStreamPartition ssp, long timeoutMillis) throws InterruptedException {
- if (lastException != null) {
- throw new RuntimeException(lastException);
- }
-
- final List envelopes = new ArrayList<>();
- final BlockingQueue queue = queues.get(ssp);
- final IncomingMessageEnvelope envelope = queue.poll(timeoutMillis, TimeUnit.MILLISECONDS);
-
- if (envelope != null) {
- envelopes.add(envelope);
- queue.drainTo(envelopes);
- }
-
- available.release(envelopes.size());
-
- if (lastException != null) {
- throw new RuntimeException(lastException);
- }
-
- return envelopes;
- }
-
- private void setError(Exception exception) {
- this.lastException = exception;
- // A dummy message used to force the consumer to wake up immediately and check the
- // lastException field, which will be populated.
- readerToSsp
- .values()
- .forEach(
- ssp -> {
- final IncomingMessageEnvelope checkLastExceptionEvelope =
- new IncomingMessageEnvelope(ssp, null, null, null);
- enqueueUninterruptibly(checkLastExceptionEvelope);
- });
- }
-
- private void enqueueUninterruptibly(IncomingMessageEnvelope envelope) {
- final BlockingQueue queue =
- queues.get(envelope.getSystemStreamPartition());
- while (true) {
- try {
- queue.put(envelope);
- return;
- } catch (InterruptedException e) {
- // Some events require that we post an envelope to the queue even if the interrupt
- // flag was set (i.e. during a call to stop) to ensure that the consumer properly
- // shuts down. Consequently, if we receive an interrupt here we ignore it and retry
- // the put operation.
- }
- }
- }
- }
- }
-
- /**
- * A {@link SystemFactory} that produces a {@link BoundedSourceSystem} for a particular {@link
- * BoundedSource} registered in {@link Config}.
- */
- public static class Factory implements SystemFactory {
- @Override
- public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) {
- final String streamPrefix = "systems." + systemName;
- final Config scopedConfig = config.subset(streamPrefix + ".", true);
-
- return new Consumer(
- getBoundedSource(scopedConfig),
- getPipelineOptions(config),
- new SamzaMetricsContainer((MetricsRegistryMap) registry),
- scopedConfig.get("stepName"));
- }
-
- @Override
- public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) {
- LOG.info("System {} does not have producer.", systemName);
- return null;
- }
-
- @Override
- public SystemAdmin getAdmin(String systemName, Config config) {
- final Config scopedConfig = config.subset("systems." + systemName + ".", true);
- return new Admin(getBoundedSource(scopedConfig), getPipelineOptions(config));
- }
-
- private static BoundedSource getBoundedSource(Config config) {
- @SuppressWarnings("unchecked")
- final BoundedSource source =
- Base64Serializer.deserializeUnchecked(config.get("source"), BoundedSource.class);
- return source;
- }
-
- private static SamzaPipelineOptions getPipelineOptions(Config config) {
- return Base64Serializer.deserializeUnchecked(
- config.get("beamPipelineOptions"), SerializablePipelineOptions.class)
- .get()
- .as(SamzaPipelineOptions.class);
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java
deleted file mode 100644
index ffab2ff59ce5..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java
+++ /dev/null
@@ -1,533 +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.samza.adapter;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Base64;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils;
-import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
-import org.apache.beam.runners.core.serialization.Base64Serializer;
-import org.apache.beam.runners.samza.SamzaPipelineOptions;
-import org.apache.beam.runners.samza.metrics.FnWithMetricsWrapper;
-import org.apache.beam.runners.samza.metrics.SamzaMetricsContainer;
-import org.apache.beam.runners.samza.runtime.OpMessage;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
-import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.WindowedValue;
-import org.apache.beam.sdk.values.WindowedValues;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
-import org.apache.samza.Partition;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.metrics.MetricsRegistry;
-import org.apache.samza.metrics.MetricsRegistryMap;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemAdmin;
-import org.apache.samza.system.SystemConsumer;
-import org.apache.samza.system.SystemFactory;
-import org.apache.samza.system.SystemProducer;
-import org.apache.samza.system.SystemStreamMetadata;
-import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata;
-import org.apache.samza.system.SystemStreamPartition;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A Samza system that supports reading from a Beam {@link UnboundedSource}. The source is split
- * into partitions. Samza creates the job model by assigning partitions to Samza tasks.
- */
-@SuppressWarnings({
- "rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class UnboundedSourceSystem {
- private static final Logger LOG = LoggerFactory.getLogger(UnboundedSourceSystem.class);
-
- // A dummy message used to force the consumer to wake up immediately and check the
- // lastException field, which will be populated.
- private static final IncomingMessageEnvelope CHECK_LAST_EXCEPTION_ENVELOPE =
- new IncomingMessageEnvelope(null, null, null, null);
-
- /**
- * For better parallelism in Samza, we need to configure a large split number for {@link
- * UnboundedSource} like Kafka. This will most likely make each split contain a single partition,
- * and be assigned to a Samza task. A large split number is safe since the actual split is bounded
- * by the number of source partitions.
- */
- private static
- List> split(
- UnboundedSource source, SamzaPipelineOptions pipelineOptions)
- throws Exception {
- final int numSplits = pipelineOptions.getMaxSourceParallelism();
- if (numSplits > 1) {
- @SuppressWarnings("unchecked")
- final List> splits =
- (List>) source.split(numSplits, pipelineOptions);
- // Need the empty check here because Samza doesn't handle empty partition well
- if (!splits.isEmpty()) {
- return splits;
- }
- }
- return Collections.singletonList(source);
- }
-
- /** A {@link SystemAdmin} for {@link UnboundedSourceSystem}. */
- public static class Admin implements SystemAdmin {
- private final UnboundedSource source;
- private final SamzaPipelineOptions pipelineOptions;
-
- public Admin(UnboundedSource source, SamzaPipelineOptions pipelineOptions) {
- this.source = source;
- this.pipelineOptions = pipelineOptions;
- }
-
- @Override
- public Map getOffsetsAfter(
- Map offsets) {
- // BEAM checkpoints the next offset so here we just need to return the map itself
- return offsets;
- }
-
- @Override
- public Map getSystemStreamMetadata(Set streamNames) {
- return streamNames.stream()
- .collect(
- Collectors.toMap(
- Function.identity(),
- streamName -> {
- try {
- final List> splits =
- split(source, pipelineOptions);
- final Map partitionMetaData =
- new HashMap<>();
- // we assume that the generated splits are stable,
- // this is necessary so that the mapping of partition to source is correct
- // in each container.
- for (int i = 0; i < splits.size(); i++) {
- partitionMetaData.put(
- new Partition(i), new SystemStreamPartitionMetadata(null, null, null));
- }
- return new SystemStreamMetadata(streamName, partitionMetaData);
- } catch (Exception e) {
- throw new SamzaException("Fail to read stream metadata", e);
- }
- }));
- }
-
- @Override
- public Integer offsetComparator(String offset1, String offset2) {
- // BEAM will fetch the exact offset. So we don't need to compare them.
- // Return null indicating it's caught up.
- return null;
- }
- }
-
- /**
- * A {@link SystemConsumer} for a {@link UnboundedSource}. See {@link UnboundedSourceSystem} for
- * more details.
- */
- public static class Consumer
- implements SystemConsumer {
- private static final Logger LOG = LoggerFactory.getLogger(Consumer.class);
-
- private static final AtomicInteger NEXT_ID = new AtomicInteger();
-
- private final Coder checkpointMarkCoder;
- private final List> splits;
- private final SamzaPipelineOptions pipelineOptions;
- private final Map readerToSsp = new HashMap<>();
- private final SamzaMetricsContainer metricsContainer;
- private final String stepName;
-
- private ReaderTask readerTask;
-
- Consumer(
- UnboundedSource source,
- SamzaPipelineOptions pipelineOptions,
- SamzaMetricsContainer metricsContainer,
- String stepName) {
- try {
- this.splits = split(source, pipelineOptions);
- } catch (Exception e) {
- throw new SamzaException("Fail to split source", e);
- }
- this.checkpointMarkCoder = source.getCheckpointMarkCoder();
- this.pipelineOptions = pipelineOptions;
- this.metricsContainer = metricsContainer;
- this.stepName = stepName;
- }
-
- @Override
- public void start() {
- if (this.readerToSsp.isEmpty()) {
- throw new IllegalArgumentException(
- "Attempted to call start without assigned system stream partitions");
- }
-
- final FnWithMetricsWrapper metricsWrapper =
- pipelineOptions.getEnableMetrics()
- ? new FnWithMetricsWrapper(metricsContainer, stepName)
- : null;
- readerTask =
- new ReaderTask<>(
- readerToSsp,
- checkpointMarkCoder,
- pipelineOptions.getSystemBufferSize(),
- pipelineOptions.getWatermarkInterval(),
- metricsWrapper);
- final Thread thread =
- new Thread(readerTask, "unbounded-source-system-consumer-" + NEXT_ID.getAndIncrement());
- thread.start();
- }
-
- @Override
- public void stop() {
- // NOTE: this is not a blocking shutdown
- readerTask.stop();
- }
-
- @Override
- public void register(SystemStreamPartition ssp, String offset) {
- CheckpointMarkT checkpoint = null;
- if (StringUtils.isNoneEmpty(offset)) {
- final byte[] offsetBytes = Base64.getDecoder().decode(offset);
- final ByteArrayInputStream bais = new ByteArrayInputStream(offsetBytes);
- try {
- checkpoint = checkpointMarkCoder.decode(bais);
- } catch (Exception e) {
- throw new SamzaException("Error in decode offset", e);
- }
- }
-
- // Create unbounded reader with checkpoint
- final int partitionId = ssp.getPartition().getPartitionId();
- try {
- final UnboundedReader reader =
- splits.get(partitionId).createReader(pipelineOptions, checkpoint);
- readerToSsp.put(reader, ssp);
- } catch (Exception e) {
- throw new SamzaException("Error while creating source reader for ssp: " + ssp, e);
- }
- }
-
- @Override
- public Map> poll(
- Set systemStreamPartitions, long timeout)
- throws InterruptedException {
- assert !readerToSsp.isEmpty(); // start should be called before poll
-
- final Map> envelopes = new HashMap<>();
- for (SystemStreamPartition ssp : systemStreamPartitions) {
- envelopes.put(ssp, readerTask.getNextMessages(ssp, timeout));
- }
- return envelopes;
- }
-
- private static class ReaderTask implements Runnable {
- private final Map readerToSsp;
- private final List readers;
- private final Coder checkpointMarkCoder;
- private final Map currentWatermarks = new HashMap<>();
- private final Map> queues;
- private final long watermarkInterval;
- private final Semaphore available;
- private final FnWithMetricsWrapper metricsWrapper;
-
- private volatile boolean running;
- private volatile Exception lastException;
- private long lastWatermarkTime = 0L;
-
- private ReaderTask(
- Map readerToSsp,
- Coder checkpointMarkCoder,
- int capacity,
- long watermarkInterval,
- FnWithMetricsWrapper metricsWrapper) {
- this.readerToSsp = readerToSsp;
- this.checkpointMarkCoder = checkpointMarkCoder;
- this.readers = ImmutableList.copyOf(readerToSsp.keySet());
- this.watermarkInterval = watermarkInterval;
- this.available = new Semaphore(capacity);
- this.metricsWrapper = metricsWrapper;
-
- final Map> qs =
- new HashMap<>();
- readerToSsp.values().forEach(ssp -> qs.put(ssp, new LinkedBlockingQueue<>()));
- this.queues = ImmutableMap.copyOf(qs);
- }
-
- @Override
- public void run() {
- this.running = true;
-
- try {
- for (UnboundedReader reader : readers) {
- final boolean hasData = invoke(reader::start);
- if (hasData) {
- available.acquire();
- enqueueMessage(reader);
- }
- }
-
- while (running) {
- boolean elementAvailable = false;
- for (UnboundedReader reader : readers) {
- final boolean hasData = invoke(reader::advance);
- if (hasData) {
- while (!available.tryAcquire(
- 1,
- Math.max(lastWatermarkTime + watermarkInterval - System.currentTimeMillis(), 1),
- TimeUnit.MILLISECONDS)) {
- updateWatermark();
- }
- enqueueMessage(reader);
- elementAvailable = true;
- }
- }
-
- updateWatermark();
-
- if (!elementAvailable) {
- // TODO: make poll interval configurable
- Thread.sleep(50);
- }
- }
- } catch (Exception e) {
- lastException = e;
- running = false;
- } finally {
- readers.forEach(
- reader -> {
- try {
- reader.close();
- } catch (IOException e) {
- LOG.error("Reader task failed to close reader", e);
- }
- });
- }
-
- if (lastException != null) {
- // Force any pollers to wake up
- queues
- .values()
- .forEach(
- queue -> {
- queue.clear();
- queue.add(CHECK_LAST_EXCEPTION_ENVELOPE);
- });
- }
- }
-
- private X invoke(FnWithMetricsWrapper.SupplierWithException fn) throws Exception {
- if (metricsWrapper != null) {
- return metricsWrapper.wrap(fn, true);
- } else {
- return fn.get();
- }
- }
-
- private void updateWatermark() throws InterruptedException {
- final long time = System.currentTimeMillis();
- if (time - lastWatermarkTime > watermarkInterval) {
- for (UnboundedReader reader : readers) {
- final SystemStreamPartition ssp = readerToSsp.get(reader);
- final Instant currentWatermark =
- currentWatermarks.containsKey(ssp)
- ? currentWatermarks.get(ssp)
- : BoundedWindow.TIMESTAMP_MIN_VALUE;
- final Instant nextWatermark = reader.getWatermark();
- if (currentWatermark.isBefore(nextWatermark)) {
- currentWatermarks.put(ssp, nextWatermark);
- if (BoundedWindow.TIMESTAMP_MAX_VALUE.isAfter(nextWatermark)) {
- enqueueWatermark(reader);
- } else {
- // Max watermark has been reached for this reader.
- enqueueMaxWatermarkAndEndOfStream(reader);
- running = false;
- }
- }
- }
-
- lastWatermarkTime = time;
- }
- }
-
- private void enqueueWatermark(UnboundedReader reader) throws InterruptedException {
- final SystemStreamPartition ssp = readerToSsp.get(reader);
- final IncomingMessageEnvelope envelope =
- IncomingMessageEnvelope.buildWatermarkEnvelope(ssp, reader.getWatermark().getMillis());
-
- queues.get(ssp).put(envelope);
- }
-
- private void enqueueMessage(UnboundedReader reader) throws InterruptedException {
- @SuppressWarnings("unchecked")
- final T value = (T) reader.getCurrent();
- final Instant time = reader.getCurrentTimestamp();
- final SystemStreamPartition ssp = readerToSsp.get(reader);
- final WindowedValue windowedValue =
- WindowedValues.timestampedValueInGlobalWindow(value, time);
-
- final OpMessage opMessage = OpMessage.ofElement(windowedValue);
- final IncomingMessageEnvelope envelope =
- new IncomingMessageEnvelope(ssp, getOffset(reader), null, opMessage);
-
- queues.get(ssp).put(envelope);
- }
-
- // Send an max watermark message and an end of stream message to the corresponding ssp to
- // close windows and finish the task.
- private void enqueueMaxWatermarkAndEndOfStream(UnboundedReader reader) {
- final SystemStreamPartition ssp = readerToSsp.get(reader);
- // Send the max watermark to force completion of any open windows.
- final IncomingMessageEnvelope watermarkEnvelope =
- IncomingMessageEnvelope.buildWatermarkEnvelope(
- ssp, BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
- enqueueUninterruptibly(watermarkEnvelope);
-
- final IncomingMessageEnvelope endOfStreamEnvelope =
- IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp);
- enqueueUninterruptibly(endOfStreamEnvelope);
- }
-
- private void enqueueUninterruptibly(IncomingMessageEnvelope envelope) {
- final BlockingQueue queue =
- queues.get(envelope.getSystemStreamPartition());
- while (true) {
- try {
- queue.put(envelope);
- return;
- } catch (InterruptedException e) {
- // Some events require that we post an envelope to the queue even if the interrupt
- // flag was set (i.e. during a call to stop) to ensure that the consumer properly
- // shuts down. Consequently, if we receive an interrupt here we ignore it and retry
- // the put operation.
- }
- }
- }
-
- void stop() {
- running = false;
- }
-
- List getNextMessages(SystemStreamPartition ssp, long timeoutMillis)
- throws InterruptedException {
- if (lastException != null) {
- throw new RuntimeException(lastException);
- }
-
- final List envelopes = new ArrayList<>();
- final BlockingQueue queue = queues.get(ssp);
- final IncomingMessageEnvelope envelope = queue.poll(timeoutMillis, TimeUnit.MILLISECONDS);
-
- if (envelope != null) {
- envelopes.add(envelope);
- queue.drainTo(envelopes);
- }
-
- final int numElements =
- (int) envelopes.stream().filter(ev -> (ev.getMessage() instanceof OpMessage)).count();
- available.release(numElements);
-
- if (lastException != null) {
- throw new RuntimeException(lastException);
- }
-
- return envelopes;
- }
-
- private String getOffset(UnboundedReader reader) {
- try {
- final ByteArrayOutputStream baos = new ByteArrayOutputStream();
- @SuppressWarnings("unchecked")
- final CheckpointMarkT checkpointMark =
- (CheckpointMarkT) invoke(reader::getCheckpointMark);
- checkpointMarkCoder.encode(checkpointMark, baos);
- return Base64.getEncoder().encodeToString(baos.toByteArray());
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
- }
- }
-
- /**
- * A {@link SystemFactory} that produces a {@link UnboundedSourceSystem} for a particular {@link
- * UnboundedSource} registered in {@link Config}.
- */
- public static class Factory implements SystemFactory {
- @Override
- public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) {
- final String streamPrefix = "systems." + systemName;
- final Config scopedConfig = config.subset(streamPrefix + ".", true);
- return new Consumer(
- getUnboundedSource(scopedConfig),
- getPipelineOptions(config),
- new SamzaMetricsContainer((MetricsRegistryMap) registry),
- scopedConfig.get("stepName"));
- }
-
- @Override
- public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) {
- LOG.info("System {} does not have producer.", systemName);
- return null;
- }
-
- @Override
- public SystemAdmin getAdmin(String systemName, Config config) {
- final Config scopedConfig = config.subset("systems." + systemName + ".", true);
- return new Admin(
- getUnboundedSource(scopedConfig), getPipelineOptions(config));
- }
-
- private static
- UnboundedSource getUnboundedSource(Config config) {
- @SuppressWarnings("unchecked")
- final UnboundedSource source =
- Base64Serializer.deserializeUnchecked(config.get("source"), UnboundedSource.class);
- return source;
- }
-
- private static SamzaPipelineOptions getPipelineOptions(Config config) {
- return Base64Serializer.deserializeUnchecked(
- config.get("beamPipelineOptions"), SerializablePipelineOptions.class)
- .get()
- .as(SamzaPipelineOptions.class);
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/package-info.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/package-info.java
deleted file mode 100644
index 582194440c9b..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/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 Samza. */
-package org.apache.beam.runners.samza.adapter;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamContainerRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamContainerRunner.java
deleted file mode 100644
index 0f4a1c7a6905..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamContainerRunner.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.samza.container;
-
-import java.time.Duration;
-import org.apache.samza.application.SamzaApplication;
-import org.apache.samza.application.descriptors.ApplicationDescriptor;
-import org.apache.samza.application.descriptors.ApplicationDescriptorImpl;
-import org.apache.samza.application.descriptors.ApplicationDescriptorUtil;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.ShellCommandConfig;
-import org.apache.samza.context.ExternalContext;
-import org.apache.samza.job.ApplicationStatus;
-import org.apache.samza.runtime.ApplicationRunner;
-import org.apache.samza.runtime.ContainerLaunchUtil;
-import org.apache.samza.util.SamzaUncaughtExceptionHandler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Runs the beam Yarn container, using the static global job model. */
-@SuppressWarnings({
- "rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class BeamContainerRunner implements ApplicationRunner {
- private static final Logger LOG = LoggerFactory.getLogger(BeamContainerRunner.class);
-
- @SuppressWarnings("rawtypes")
- private final ApplicationDescriptorImpl extends ApplicationDescriptor> appDesc;
-
- @SuppressWarnings("rawtypes")
- public BeamContainerRunner(SamzaApplication app, Config config) {
- this.appDesc = ApplicationDescriptorUtil.getAppDescriptor(app, config);
- }
-
- @Override
- public void run(ExternalContext externalContext) {
- Thread.setDefaultUncaughtExceptionHandler(
- new SamzaUncaughtExceptionHandler(
- () -> {
- LOG.info("Exiting process now.");
- System.exit(1);
- }));
-
- ContainerLaunchUtil.run(
- appDesc, System.getenv(ShellCommandConfig.ENV_CONTAINER_ID), ContainerCfgLoader.jobModel);
- }
-
- @Override
- public void kill() {
- // Do nothing. Yarn will kill the container.
- }
-
- @Override
- public ApplicationStatus status() {
- // The container is running during the life span of this object.
- return ApplicationStatus.Running;
- }
-
- @Override
- public void waitForFinish() {
- // Container run is synchronous
- // so calling waitForFinish() after run() should return immediately
- LOG.info("Container has stopped");
- }
-
- @Override
- public boolean waitForFinish(Duration timeout) {
- // Container run is synchronous
- // so calling waitForFinish() after run() should return immediately
- LOG.info("Container has stopped");
- return true;
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamJobCoordinatorRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamJobCoordinatorRunner.java
deleted file mode 100644
index fb00a018fb29..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamJobCoordinatorRunner.java
+++ /dev/null
@@ -1,78 +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.samza.container;
-
-import java.time.Duration;
-import org.apache.samza.application.SamzaApplication;
-import org.apache.samza.application.descriptors.ApplicationDescriptor;
-import org.apache.samza.clustermanager.JobCoordinatorLaunchUtil;
-import org.apache.samza.config.Config;
-import org.apache.samza.context.ExternalContext;
-import org.apache.samza.job.ApplicationStatus;
-import org.apache.samza.runtime.ApplicationRunner;
-
-/** Runs on Yarn AM, execute planning and launches JobCoordinator. */
-public class BeamJobCoordinatorRunner implements ApplicationRunner {
-
- @SuppressWarnings("rawtypes")
- private final SamzaApplication extends ApplicationDescriptor> app;
-
- private final Config config;
-
- /**
- * Constructors a {@link BeamJobCoordinatorRunner} to run the {@code app} with the {@code config}.
- *
- * @param app application to run
- * @param config configuration for the application
- */
- @SuppressWarnings("rawtypes")
- public BeamJobCoordinatorRunner(
- SamzaApplication extends ApplicationDescriptor> app, Config config) {
- this.app = app;
- this.config = config;
- }
-
- @Override
- public void run(ExternalContext externalContext) {
- JobCoordinatorLaunchUtil.run(app, config);
- }
-
- @Override
- public void kill() {
- throw new UnsupportedOperationException(
- "BeamJobCoordinatorRunner#kill should never be invoked.");
- }
-
- @Override
- public ApplicationStatus status() {
- throw new UnsupportedOperationException(
- "BeamJobCoordinatorRunner#status should never be invoked.");
- }
-
- @Override
- public void waitForFinish() {
- throw new UnsupportedOperationException(
- "BeamJobCoordinatorRunner#waitForFinish should never be invoked.");
- }
-
- @Override
- public boolean waitForFinish(Duration timeout) {
- throw new UnsupportedOperationException(
- "BeamJobCoordinatorRunner#waitForFinish should never be invoked.");
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoader.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoader.java
deleted file mode 100644
index 9437aea56561..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoader.java
+++ /dev/null
@@ -1,62 +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.samza.container;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Random;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.ConfigLoader;
-import org.apache.samza.config.MapConfig;
-import org.apache.samza.config.ShellCommandConfig;
-import org.apache.samza.container.SamzaContainer;
-import org.apache.samza.job.model.JobModel;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Loader for the Beam yarn container to load job model. */
-@SuppressWarnings({
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class ContainerCfgLoader implements ConfigLoader {
- private static final Logger LOG = LoggerFactory.getLogger(ContainerCfgLoader.class);
-
- private static final Object LOCK = new Object();
- static volatile JobModel jobModel;
- private static final Random RANDOM = new Random();
-
- @Override
- public Config getConfig() {
- if (jobModel == null) {
- synchronized (LOCK) {
- if (jobModel == null) {
- final String containerId = System.getenv(ShellCommandConfig.ENV_CONTAINER_ID);
- LOG.info("Got container ID: {}", containerId);
- final String coordinatorUrl = System.getenv(ShellCommandConfig.ENV_COORDINATOR_URL);
- LOG.info("Got coordinator URL: {}", coordinatorUrl);
- final int delay = RANDOM.nextInt(SamzaContainer.DEFAULT_READ_JOBMODEL_DELAY_MS()) + 1;
- jobModel = SamzaContainer.readJobModel(coordinatorUrl, delay);
- }
- }
- }
-
- final Map config = new HashMap<>(jobModel.getConfig());
- config.put("app.runner.class", BeamContainerRunner.class.getName());
- return new MapConfig(config);
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoaderFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoaderFactory.java
deleted file mode 100644
index d3b090d6e20a..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgLoaderFactory.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.samza.container;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.config.ConfigLoader;
-import org.apache.samza.config.ConfigLoaderFactory;
-
-/** Factory for the Beam yarn container to get loader to load job model. */
-public class ContainerCfgLoaderFactory implements ConfigLoaderFactory {
- @Override
- public ConfigLoader getLoader(Config config) {
- return new ContainerCfgLoader();
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/package-info.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/package-info.java
deleted file mode 100644
index 58a09e6023de..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/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 Samza. */
-package org.apache.beam.runners.samza.container;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java
deleted file mode 100644
index 7bec91abb34d..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java
+++ /dev/null
@@ -1,108 +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.samza.metrics;
-
-import org.apache.beam.runners.core.DoFnRunner;
-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.values.CausedByDrain;
-import org.apache.beam.sdk.values.WindowedValue;
-import org.joda.time.Instant;
-
-/**
- * {@link DoFnRunner} wrapper with metrics. The class uses {@link SamzaMetricsContainer} to keep
- * BEAM metrics results and update Samza metrics.
- */
-public class DoFnRunnerWithMetrics implements DoFnRunner {
- private final DoFnRunner underlying;
- private final FnWithMetricsWrapper metricsWrapper;
-
- private DoFnRunnerWithMetrics(
- DoFnRunner underlying, SamzaMetricsContainer metricsContainer, String stepName) {
- this.underlying = underlying;
- this.metricsWrapper = new FnWithMetricsWrapper(metricsContainer, stepName);
- }
-
- public static DoFnRunner wrap(
- DoFnRunner doFnRunner, SamzaMetricsContainer metricsContainer, String stepName) {
- return new DoFnRunnerWithMetrics<>(doFnRunner, metricsContainer, stepName);
- }
-
- @Override
- public void startBundle() {
- withMetrics(underlying::startBundle, false);
- }
-
- @Override
- public void processElement(WindowedValue elem) {
- withMetrics(() -> underlying.processElement(elem), false);
- }
-
- @Override
- public void onTimer(
- String timerId,
- String timerFamilyId,
- KeyT key,
- BoundedWindow window,
- Instant timestamp,
- Instant outputTimestamp,
- TimeDomain timeDomain,
- CausedByDrain causedByDrain) {
- withMetrics(
- () ->
- underlying.onTimer(
- timerId,
- timerFamilyId,
- key,
- window,
- timestamp,
- outputTimestamp,
- timeDomain,
- causedByDrain),
- false);
- }
-
- @Override
- public void finishBundle() {
- withMetrics(underlying::finishBundle, true);
- }
-
- @Override
- public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {
- underlying.onWindowExpiration(window, timestamp, key);
- }
-
- @Override
- public DoFn getFn() {
- return underlying.getFn();
- }
-
- private void withMetrics(Runnable runnable, boolean shouldUpdateMetrics) {
- try {
- metricsWrapper.wrap(
- () -> {
- runnable.run();
- return (Void) null;
- },
- shouldUpdateMetrics);
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/FnWithMetricsWrapper.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/FnWithMetricsWrapper.java
deleted file mode 100644
index 8052e0d66146..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/FnWithMetricsWrapper.java
+++ /dev/null
@@ -1,50 +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.samza.metrics;
-
-import java.io.Closeable;
-import org.apache.beam.sdk.metrics.MetricsEnvironment;
-
-/** This class wraps a {@link java.util.function.Supplier} function call with BEAM metrics. */
-public class FnWithMetricsWrapper {
-
- /** Interface for functions to be wrapped with metrics. */
- public interface SupplierWithException {
- T get() throws Exception;
- }
-
- private final SamzaMetricsContainer metricsContainer;
- private final String stepName;
-
- public FnWithMetricsWrapper(SamzaMetricsContainer metricsContainer, String stepName) {
- this.metricsContainer = metricsContainer;
- this.stepName = stepName;
- }
-
- public T wrap(SupplierWithException fn, boolean shouldUpdateMetrics) throws Exception {
- try (Closeable closeable =
- MetricsEnvironment.scopedMetricsContainer(metricsContainer.getContainer(stepName))) {
- T result = fn.get();
- // Skip updating metrics if not necessary to improve performance
- if (shouldUpdateMetrics) {
- metricsContainer.updateMetrics(stepName);
- }
- return result;
- }
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java
deleted file mode 100644
index 9b6375171c23..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java
+++ /dev/null
@@ -1,194 +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.samza.metrics;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import org.apache.beam.runners.samza.runtime.KeyedTimerData;
-import org.apache.beam.runners.samza.runtime.Op;
-import org.apache.beam.runners.samza.runtime.OpEmitter;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.WindowedValue;
-import org.apache.samza.config.Config;
-import org.apache.samza.context.Context;
-import org.apache.samza.operators.Scheduler;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SamzaGBKMetricOp is a {@link Op} that emits & maintains default metrics for input or output
- * PCollection for GroupByKey.
- *
- *
For Input PCollection: It emits the input throughput and maintains avg input time for input
- * PCollection per windowId.
- *
- *
For Output PCollection: It emits the output throughput and maintains avg output time for
- * output PCollection per windowId. It is also responsible for emitting latency metric per windowId
- * once the watermark passes the end of window timestamp.
- *
- *
Assumes that {@code SamzaGBKMetricOp#processWatermark(Instant, OpEmitter)} is exclusive of
- * {@code SamzaGBKMetricOp#processElement(Instant, OpEmitter)}. Specifically, the processWatermark
- * method assumes that no calls to processElement will be made during its execution, and vice versa.
- *
- * @param The type of the elements in the input PCollection.
- */
-class SamzaGBKMetricOp implements Op {
- private static final Logger LOG = LoggerFactory.getLogger(SamzaGBKMetricOp.class);
- // Unique name of the PTransform this MetricOp is associated with
- private final String transformFullName;
- private final SamzaTransformMetricRegistry samzaTransformMetricRegistry;
- // Type of the processing operation
- private final SamzaMetricOpFactory.OpType opType;
-
- private final String pValue;
- // Counters for keeping sum of arrival time and count of elements per windowId
- @SuppressFBWarnings("SE_BAD_FIELD")
- private final ConcurrentHashMap sumOfTimestampsPerWindowId;
-
- @SuppressFBWarnings("SE_BAD_FIELD")
- private final ConcurrentHashMap sumOfCountPerWindowId;
- // Name of the task, for logging purpose
- private transient String task;
-
- @Override
- @SuppressWarnings({"rawtypes", "unchecked"})
- public void open(
- Config config,
- Context context,
- Scheduler> timerRegistry,
- OpEmitter emitter) {
- // for logging / debugging purposes
- this.task = context.getTaskContext().getTaskModel().getTaskName().getTaskName();
- // Register the transform with SamzaTransformMetricRegistry
- samzaTransformMetricRegistry.register(transformFullName, pValue, context);
- }
-
- // Some fields are initialized in open() method, which is called after the constructor.
- @SuppressWarnings("initialization.fields.uninitialized")
- public SamzaGBKMetricOp(
- String pValue,
- String transformFullName,
- SamzaMetricOpFactory.OpType opType,
- SamzaTransformMetricRegistry samzaTransformMetricRegistry) {
- this.pValue = pValue;
- this.transformFullName = transformFullName;
- this.opType = opType;
- this.samzaTransformMetricRegistry = samzaTransformMetricRegistry;
- this.sumOfTimestampsPerWindowId = new ConcurrentHashMap<>();
- this.sumOfCountPerWindowId = new ConcurrentHashMap<>();
- }
-
- @Override
- public void processElement(WindowedValue inputElement, OpEmitter emitter) {
- // one element can belong to multiple windows
- for (BoundedWindow windowId : inputElement.getWindows()) {
- // Atomic updates to counts
- sumOfCountPerWindowId.compute(
- windowId,
- (key, value) -> {
- value = value == null ? Long.valueOf(0) : value;
- return ++value;
- });
- // Atomic updates to sum of arrival timestamps
- sumOfTimestampsPerWindowId.compute(
- windowId,
- (key, value) -> {
- value = value == null ? BigInteger.ZERO : value;
- return value.add(BigInteger.valueOf(System.nanoTime()));
- });
- }
-
- switch (opType) {
- case INPUT:
- samzaTransformMetricRegistry
- .getTransformMetrics()
- .getTransformInputThroughput(transformFullName)
- .inc();
- break;
- case OUTPUT:
- samzaTransformMetricRegistry
- .getTransformMetrics()
- .getTransformOutputThroughput(transformFullName)
- .inc();
- break;
- }
- emitter.emitElement(inputElement);
- }
-
- @Override
- public void processWatermark(Instant watermark, OpEmitter emitter) {
- final List closedWindows = new ArrayList<>();
- sumOfCountPerWindowId.keySet().stream()
- .filter(windowId -> watermark.isAfter(windowId.maxTimestamp())) // window is closed
- .forEach(
- windowId -> {
- // In case if BigInteger overflows for long we only retain the last 64 bits of the sum
- long sumOfTimestamps =
- sumOfTimestampsPerWindowId.get(windowId) != null
- ? sumOfTimestampsPerWindowId.get(windowId).longValue()
- : 0L;
- long count = sumOfCountPerWindowId.get(windowId);
- closedWindows.add(windowId);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Processing {} Watermark for Transform: {}, WindowId:{}, count: {}, sumOfTimestamps: {}, task: {}",
- opType,
- transformFullName,
- windowId,
- count,
- sumOfTimestamps,
- task);
- }
-
- // if the window is closed and there is some data
- if (sumOfTimestamps > 0 && count > 0) {
- switch (opType) {
- case INPUT:
- // Update the arrival time for the window
- samzaTransformMetricRegistry.updateArrivalTimeMap(
- transformFullName, windowId, Math.floorDiv(sumOfTimestamps, count));
- break;
- case OUTPUT:
- // Compute the latency if there is some data for the window
- samzaTransformMetricRegistry.emitLatencyMetric(
- transformFullName, windowId, Math.floorDiv(sumOfTimestamps, count), task);
- break;
- }
- }
- });
-
- // remove the closed windows
- sumOfCountPerWindowId.keySet().removeAll(closedWindows);
- sumOfTimestampsPerWindowId.keySet().removeAll(closedWindows);
-
- // Update the watermark progress for the transform output
- if (opType == SamzaMetricOpFactory.OpType.OUTPUT) {
- samzaTransformMetricRegistry
- .getTransformMetrics()
- .getTransformWatermarkProgress(transformFullName)
- .set(watermark.getMillis());
- }
-
- emitter.emitWatermark(watermark);
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java
deleted file mode 100644
index 0f5334546c7c..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.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.samza.metrics;
-
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.beam.runners.samza.runtime.KeyedTimerData;
-import org.apache.beam.runners.samza.runtime.Op;
-import org.apache.beam.runners.samza.runtime.OpEmitter;
-import org.apache.beam.runners.samza.util.PipelineJsonRenderer;
-import org.apache.beam.sdk.values.WindowedValue;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
-import org.apache.samza.config.Config;
-import org.apache.samza.context.Context;
-import org.apache.samza.operators.Scheduler;
-import org.checkerframework.checker.nullness.qual.NonNull;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SamzaMetricOp is a metric Op that emits & maintains default transform metrics for inputs &
- * outputs PCollection to the non data-shuffle transform. It emits the output throughput and
- * maintains avg arrival time for input & output PCollection per watermark.
- *
- *
Assumes that {@code SamzaMetricOp#processWatermark(Instant, OpEmitter)} is exclusive of {@code
- * SamzaMetricOp#processElement(Instant, OpEmitter)}. Specifically, the processWatermark method
- * assumes that no calls to processElement will be made during its execution, and vice versa.
- *
- * @param The type of the elements in the output PCollection.
- */
-class SamzaMetricOp implements Op {
- // Unique name of the PTransform this MetricOp is associated with
- private final String transformFullName;
- private final SamzaTransformMetricRegistry samzaTransformMetricRegistry;
- // Name or identifier of the PCollection which PTransform is processing
- private final String pValue;
- // Counters for output throughput
- private final AtomicLong count;
- private final AtomicReference sumOfTimestamps;
- // Type of the PTransform input or output
- private final SamzaMetricOpFactory.OpType opType;
- // List of input PValue(s) for all PCollections processing the PTransform
- private transient List transformInputs;
- // List of output PValue(s) for all PCollections processing the PTransform
- private transient List transformOutputs;
- // Name of the task, for logging purpose
- private transient String task;
-
- private static final Logger LOG = LoggerFactory.getLogger(SamzaMetricOp.class);
-
- // Some fields are initialized in open() method, which is called after the constructor.
- @SuppressWarnings("initialization.fields.uninitialized")
- public SamzaMetricOp(
- @NonNull String pValue,
- @NonNull String transformFullName,
- SamzaMetricOpFactory.OpType opType,
- @NonNull SamzaTransformMetricRegistry samzaTransformMetricRegistry) {
- this.transformFullName = transformFullName;
- this.samzaTransformMetricRegistry = samzaTransformMetricRegistry;
- this.pValue = pValue;
- this.opType = opType;
- this.count = new AtomicLong(0L);
- this.sumOfTimestamps = new AtomicReference<>(BigInteger.ZERO);
- }
-
- @Override
- @SuppressWarnings({"rawtypes", "unchecked"})
- public void open(
- Config config,
- Context context,
- Scheduler> timerRegistry,
- OpEmitter emitter) {
- final Map.Entry, List> transformInputOutput =
- PipelineJsonRenderer.getTransformIOMap(config).get(transformFullName);
- this.transformInputs =
- transformInputOutput != null ? transformInputOutput.getKey() : new ArrayList();
- this.transformOutputs =
- transformInputOutput != null ? transformInputOutput.getValue() : new ArrayList();
- // for logging / debugging purposes
- this.task = context.getTaskContext().getTaskModel().getTaskName().getTaskName();
- // Register the transform with SamzaTransformMetricRegistry
- samzaTransformMetricRegistry.register(transformFullName, pValue, context);
- }
-
- @Override
- public void processElement(WindowedValue inputElement, OpEmitter emitter) {
- // update counters for timestamps
- count.incrementAndGet();
- sumOfTimestamps.updateAndGet(sum -> sum.add(BigInteger.valueOf(System.nanoTime())));
- switch (opType) {
- case INPUT:
- samzaTransformMetricRegistry
- .getTransformMetrics()
- .getTransformInputThroughput(transformFullName)
- .inc();
- break;
- case OUTPUT:
- samzaTransformMetricRegistry
- .getTransformMetrics()
- .getTransformOutputThroughput(transformFullName)
- .inc();
- break;
- }
- emitter.emitElement(inputElement);
- }
-
- @Override
- @SuppressWarnings({"CompareToZero"})
- public void processWatermark(Instant watermark, OpEmitter emitter) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Processing Output Watermark for Transform: {} Count: {} SumOfTimestamps: {} for Watermark: {} for Task: {}",
- transformFullName,
- count.get(),
- sumOfTimestamps.get().longValue(),
- watermark.getMillis(),
- task);
- }
-
- // if there is no input data then counters will be zero and only watermark will progress
- if (count.get() > 0) {
- // if BigInt.longValue is out of range for long then only the low-order 64 bits are retained
- long avg = Math.floorDiv(sumOfTimestamps.get().longValue(), count.get());
- // Update MetricOp Registry with avg arrival for the pValue
- samzaTransformMetricRegistry.updateArrivalTimeMap(
- transformFullName, pValue, watermark.getMillis(), avg);
- if (opType == SamzaMetricOpFactory.OpType.OUTPUT) {
- // compute & emit the latency metric if the opType is OUTPUT
- samzaTransformMetricRegistry.emitLatencyMetric(
- transformFullName, transformInputs, transformOutputs, watermark.getMillis(), task);
- }
- }
-
- if (opType == SamzaMetricOpFactory.OpType.OUTPUT) {
- // update output watermark progress metric
- samzaTransformMetricRegistry
- .getTransformMetrics()
- .getTransformWatermarkProgress(transformFullName)
- .set(watermark.getMillis());
- }
-
- // reset all counters
- count.set(0L);
- this.sumOfTimestamps.set(BigInteger.ZERO);
- emitter.emitWatermark(watermark);
- }
-
- @VisibleForTesting
- void init(List transformInputs, List transformOutputs) {
- this.transformInputs = transformInputs;
- this.transformOutputs = transformOutputs;
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOpFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOpFactory.java
deleted file mode 100644
index a4112a510459..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOpFactory.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.samza.metrics;
-
-import org.apache.beam.runners.samza.runtime.Op;
-import org.apache.beam.sdk.util.construction.PTransformTranslation;
-import org.checkerframework.checker.nullness.qual.NonNull;
-
-/**
- * Factory class to create {@link Op} for default transform metric computation.
- *
- *
Each metric Op computes and emits default throughput, latency & watermark progress metric per
- * transform for Beam Samza Runner. A metric Op can be either attached to Input PCollection or
- * Output PCollection of a PTransform.
- *
- *
Each concrete metric OP is responsible for following metrics computation: 1. Throughput: Emit
- * the number of elements processed in the PCollection 2. Watermark Progress: Emit the output
- * watermark progress of the PCollection 3. Latency: Maintain the avg arrival time per watermark
- * across elements it processes, compute & emit the latency
- */
-@SuppressWarnings({
- "rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
- "nullness"
-}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
-public class SamzaMetricOpFactory {
- public enum OpType {
- INPUT,
- OUTPUT
- }
-
- /**
- * Create a {@link Op} for default transform metric computation.
- *
- * @param urn URN of the PCollection metric Op is processing
- * @param pValue name of the PCollection metric Op is processing
- * @param transformName name of the PTransform for which metric Op is created
- * @param opType type of the metric
- * @param samzaTransformMetricRegistry metric registry
- * @param type of the message
- * @return a {@link Op} for default transform metric computation
- */
- public static @NonNull Op createMetricOp(
- @NonNull String urn,
- @NonNull String pValue,
- @NonNull String transformName,
- @NonNull OpType opType,
- @NonNull SamzaTransformMetricRegistry samzaTransformMetricRegistry) {
- if (isDataShuffleTransform(urn)) {
- return new SamzaGBKMetricOp<>(pValue, transformName, opType, samzaTransformMetricRegistry);
- }
- return new SamzaMetricOp<>(pValue, transformName, opType, samzaTransformMetricRegistry);
- }
-
- private static boolean isDataShuffleTransform(String urn) {
- return urn.equals(PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN)
- || urn.equals(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN);
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricsContainer.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricsContainer.java
deleted file mode 100644
index 1679b748b10b..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricsContainer.java
+++ /dev/null
@@ -1,120 +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.samza.metrics;
-
-import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.function.Consumer;
-import org.apache.beam.runners.core.metrics.MetricsContainerStepMap;
-import org.apache.beam.sdk.metrics.GaugeResult;
-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.sdk.metrics.MetricsContainer;
-import org.apache.beam.sdk.metrics.MetricsFilter;
-import org.apache.samza.metrics.Counter;
-import org.apache.samza.metrics.Gauge;
-import org.apache.samza.metrics.Metric;
-import org.apache.samza.metrics.MetricsRegistryMap;
-
-/**
- * This class holds the {@link MetricsContainer}s for BEAM metrics, and update the results to Samza
- * metrics.
- */
-@SuppressWarnings({
- "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public class SamzaMetricsContainer {
- private static final String BEAM_METRICS_GROUP = "BeamMetrics";
-
- private final MetricsContainerStepMap metricsContainers = new MetricsContainerStepMap();
- private final MetricsRegistryMap metricsRegistry;
-
- public SamzaMetricsContainer(MetricsRegistryMap metricsRegistry) {
- this.metricsRegistry = metricsRegistry;
- this.metricsRegistry.metrics().put(BEAM_METRICS_GROUP, new ConcurrentHashMap<>());
- }
-
- public MetricsContainer getContainer(String stepName) {
- return this.metricsContainers.getContainer(stepName);
- }
-
- public MetricsContainerStepMap getContainers() {
- return this.metricsContainers;
- }
-
- public void updateMetrics(String stepName) {
- assert metricsRegistry != null;
-
- final MetricResults metricResults = asAttemptedOnlyMetricResults(metricsContainers);
- final MetricQueryResults results =
- metricResults.queryMetrics(MetricsFilter.builder().addStep(stepName).build());
-
- final CounterUpdater updateCounter = new CounterUpdater();
- results.getCounters().forEach(updateCounter);
-
- final GaugeUpdater updateGauge = new GaugeUpdater();
- results.getGauges().forEach(updateGauge);
-
- // TODO(https://github.com/apache/beam/issues/21043): add distribution metrics to Samza
- }
-
- public void updateExecutableStageBundleMetric(String metricName, long time) {
- @SuppressWarnings("unchecked")
- Gauge gauge = (Gauge) getSamzaMetricFor(metricName);
- if (gauge == null) {
- gauge = metricsRegistry.newGauge(BEAM_METRICS_GROUP, metricName, 0L);
- }
- gauge.set(time);
- }
-
- private class CounterUpdater implements Consumer> {
- @Override
- public void accept(MetricResult metricResult) {
- final String metricName = getMetricName(metricResult);
- Counter counter = (Counter) getSamzaMetricFor(metricName);
- if (counter == null) {
- counter = metricsRegistry.newCounter(BEAM_METRICS_GROUP, metricName);
- }
- counter.dec(counter.getCount());
- counter.inc(metricResult.getAttempted());
- }
- }
-
- private class GaugeUpdater implements Consumer> {
- @Override
- public void accept(MetricResult metricResult) {
- final String metricName = getMetricName(metricResult);
- @SuppressWarnings("unchecked")
- Gauge gauge = (Gauge) getSamzaMetricFor(metricName);
- if (gauge == null) {
- gauge = metricsRegistry.newGauge(BEAM_METRICS_GROUP, metricName, 0L);
- }
- gauge.set(metricResult.getAttempted().getValue());
- }
- }
-
- private Metric getSamzaMetricFor(String metricName) {
- return metricsRegistry.getGroup(BEAM_METRICS_GROUP).get(metricName);
- }
-
- private static String getMetricName(MetricResult> metricResult) {
- return metricResult.getKey().toString();
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaTransformMetricRegistry.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaTransformMetricRegistry.java
deleted file mode 100644
index add207752f06..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaTransformMetricRegistry.java
+++ /dev/null
@@ -1,208 +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.samza.metrics;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
-import org.apache.samza.context.Context;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SamzaTransformMetricRegistry is a registry that maintains the metrics for each transform. It
- * maintains the average arrival time for each PCollection for a primitive transform.
- *
- *
For a non-data shuffling primitive transform, the average arrival time is calculated per
- * watermark, per PCollection {@link org.apache.beam.sdk.values.PValue} and updated in
- * avgArrivalTimeMap
- */
-public class SamzaTransformMetricRegistry implements Serializable {
- private static final Logger LOG = LoggerFactory.getLogger(SamzaTransformMetricRegistry.class);
-
- // TransformName -> PValue for pCollection -> Map
- private final ConcurrentHashMap>>
- avgArrivalTimeMap;
- // TransformName -> Map
- @SuppressFBWarnings("SE_BAD_FIELD")
- private final ConcurrentHashMap>
- avgArrivalTimeMapForGbk;
-
- // Per Transform Metrics for each primitive transform
- private final SamzaTransformMetrics transformMetrics;
-
- public SamzaTransformMetricRegistry() {
- this.avgArrivalTimeMap = new ConcurrentHashMap<>();
- this.avgArrivalTimeMapForGbk = new ConcurrentHashMap<>();
- this.transformMetrics = new SamzaTransformMetrics();
- }
-
- @VisibleForTesting
- SamzaTransformMetricRegistry(SamzaTransformMetrics samzaTransformMetrics) {
- this.transformMetrics = samzaTransformMetrics;
- this.avgArrivalTimeMap = new ConcurrentHashMap<>();
- this.avgArrivalTimeMapForGbk = new ConcurrentHashMap<>();
- }
-
- public void register(String transformFullName, String pValue, Context ctx) {
- transformMetrics.register(transformFullName, ctx);
- // initialize the map for the transform
- avgArrivalTimeMap.putIfAbsent(transformFullName, new ConcurrentHashMap<>());
- avgArrivalTimeMap.get(transformFullName).putIfAbsent(pValue, new ConcurrentHashMap<>());
- avgArrivalTimeMapForGbk.putIfAbsent(transformFullName, new ConcurrentHashMap<>());
- }
-
- public SamzaTransformMetrics getTransformMetrics() {
- return transformMetrics;
- }
-
- public void updateArrivalTimeMap(String transformName, String pValue, long watermark, long avg) {
- if (avgArrivalTimeMap.get(transformName) != null
- && avgArrivalTimeMap.get(transformName).get(pValue) != null) {
- ConcurrentHashMap avgArrivalTimeMapForPValue =
- avgArrivalTimeMap.get(transformName).get(pValue);
- // update the average arrival time for the latest watermark
- avgArrivalTimeMapForPValue.put(watermark, avg);
- // remove any stale entries which are lesser than the watermark
- avgArrivalTimeMapForPValue.entrySet().removeIf(entry -> entry.getKey() < watermark);
- }
- }
-
- public void updateArrivalTimeMap(String transformName, BoundedWindow windowId, long avg) {
- ConcurrentHashMap avgArrivalTimeMapForTransform =
- avgArrivalTimeMapForGbk.get(transformName);
- if (avgArrivalTimeMapForTransform != null) {
- avgArrivalTimeMapForTransform.put(windowId, avg);
- }
- }
-
- @SuppressWarnings("nullness")
- public void emitLatencyMetric(
- String transformName, BoundedWindow windowId, long avgArrivalEndTime, String taskName) {
- Long avgArrivalStartTime =
- avgArrivalTimeMapForGbk.get(transformName) != null
- ? avgArrivalTimeMapForGbk.get(transformName).remove(windowId)
- : null;
-
- if (avgArrivalStartTime == null || avgArrivalStartTime == 0 || avgArrivalEndTime == 0) {
- LOG.debug(
- "Failure to Emit Metric for Transform: {}, Start-Time: {} or End-Time: {} found is 0/null for windowId: {}, task: {}",
- transformName,
- avgArrivalStartTime,
- avgArrivalEndTime,
- windowId,
- taskName);
- return;
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Success Emit Metric for Transform: {}, window: {} for task: {}",
- transformName,
- windowId,
- taskName);
- }
- transformMetrics
- .getTransformLatencyMetric(transformName)
- .update(avgArrivalEndTime - avgArrivalStartTime);
-
- transformMetrics
- .getTransformCacheSize(transformName)
- .set((long) avgArrivalTimeMapForGbk.get(transformName).size());
- }
-
- // Checker framework bug: https://github.com/typetools/checker-framework/issues/979
- @SuppressWarnings("return")
- public void emitLatencyMetric(
- String transformName,
- List inputs,
- List outputs,
- Long watermark,
- String taskName) {
- final ConcurrentHashMap> avgArrivalTimeMapForTransform =
- avgArrivalTimeMap.get(transformName);
-
- if (avgArrivalTimeMapForTransform == null || inputs.isEmpty() || outputs.isEmpty()) {
- return;
- }
-
- // get the avg arrival times for all the input PValues
- final List inputPValuesAvgArrivalTimes =
- inputs.stream()
- .map(avgArrivalTimeMapForTransform::get)
- .map(map -> map == null ? null : map.remove(watermark))
- .filter(avgArrivalTime -> avgArrivalTime != null)
- .collect(Collectors.toList());
-
- // get the avg arrival times for all the output PValues
- final List outputPValuesAvgArrivalTimes =
- outputs.stream()
- .map(avgArrivalTimeMapForTransform::get)
- .map(map -> map == null ? null : map.remove(watermark))
- .filter(avgArrivalTime -> avgArrivalTime != null)
- .collect(Collectors.toList());
-
- if (inputPValuesAvgArrivalTimes.isEmpty() || outputPValuesAvgArrivalTimes.isEmpty()) {
- LOG.debug(
- "Failure to Emit Metric for Transform: {} inputArrivalTime: {} or outputArrivalTime: {} not found for Watermark: {} Task: {}",
- transformName,
- inputPValuesAvgArrivalTimes,
- inputPValuesAvgArrivalTimes,
- watermark,
- taskName);
- return;
- }
-
- final long startTime = Collections.min(inputPValuesAvgArrivalTimes);
- final long endTime = Collections.max(outputPValuesAvgArrivalTimes);
- final long latency = endTime - startTime;
- transformMetrics.getTransformLatencyMetric(transformName).update(latency);
-
- transformMetrics
- .getTransformCacheSize(transformName)
- .set(
- avgArrivalTimeMapForTransform.values().stream()
- .mapToLong(ConcurrentHashMap::size)
- .sum());
-
- LOG.debug(
- "Success Emit Metric Transform: {} for watermark: {} for task: {}",
- transformName,
- watermark,
- taskName);
- }
-
- @VisibleForTesting
- @Nullable
- ConcurrentHashMap> getAverageArrivalTimeMap(
- String transformName) {
- return avgArrivalTimeMap.get(transformName);
- }
-
- @VisibleForTesting
- @Nullable
- ConcurrentHashMap getAverageArrivalTimeMapForGBK(String transformName) {
- return avgArrivalTimeMapForGbk.get(transformName);
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaTransformMetrics.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaTransformMetrics.java
deleted file mode 100644
index 229b6da4e7c0..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaTransformMetrics.java
+++ /dev/null
@@ -1,142 +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.samza.metrics;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.io.Serializable;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import org.apache.samza.context.Context;
-import org.apache.samza.metrics.Counter;
-import org.apache.samza.metrics.Gauge;
-import org.apache.samza.metrics.MetricsRegistry;
-import org.apache.samza.metrics.SlidingTimeWindowReservoir;
-import org.apache.samza.metrics.Timer;
-
-/**
- * Metrics like throughput, latency and watermark progress for each Beam transform for Samza Runner.
- */
-@SuppressWarnings("return")
-public class SamzaTransformMetrics implements Serializable {
- private static final String ENABLE_TASK_METRICS = "runner.samza.transform.enable.task.metrics";
-
- private static final int DEFAULT_LOOKBACK_TIMER_WINDOW_SIZE_MS = 180000;
- private static final String GROUP = "SamzaBeamTransformMetrics";
- private static final String TRANSFORM_LATENCY_METRIC = "handle-message-ns";
- private static final String TRANSFORM_WATERMARK_PROGRESS = "output-watermark-ms";
- private static final String TRANSFORM_IP_THROUGHPUT = "num-input-messages";
- private static final String TRANSFORM_OP_THROUGHPUT = "num-output-messages";
-
- private static final String TRANSFORM_ARRIVAL_TIME_CACHE_SIZE = "in-mem-cache-size";
-
- // Transform name to metric maps
- @SuppressFBWarnings("SE_BAD_FIELD")
- private final Map transformLatency;
-
- @SuppressFBWarnings("SE_BAD_FIELD")
- private final Map> transformWatermarkProgress;
-
- @SuppressFBWarnings("SE_BAD_FIELD")
- private final Map transformInputThroughput;
-
- @SuppressFBWarnings("SE_BAD_FIELD")
- private final Map transformOutputThroughPut;
-
- @SuppressFBWarnings("SE_BAD_FIELD")
- private final Map> transformCacheSize;
-
- public SamzaTransformMetrics() {
- this.transformLatency = new ConcurrentHashMap<>();
- this.transformOutputThroughPut = new ConcurrentHashMap<>();
- this.transformWatermarkProgress = new ConcurrentHashMap<>();
- this.transformInputThroughput = new ConcurrentHashMap<>();
- this.transformCacheSize = new ConcurrentHashMap<>();
- }
-
- public void register(String transformName, Context ctx) {
- // Output Watermark metric per transform will always be per transform, per task, since per
- // container output watermark is not useful for debugging
- transformWatermarkProgress.putIfAbsent(
- transformName,
- ctx.getTaskContext()
- .getTaskMetricsRegistry()
- .newGauge(
- GROUP, getMetricNameWithPrefix(TRANSFORM_WATERMARK_PROGRESS, transformName), 0L));
-
- // Latency, throughput metrics can be per container (default) or per task
- final boolean enablePerTaskMetrics =
- ctx.getJobContext().getConfig().getBoolean(ENABLE_TASK_METRICS, false);
- final MetricsRegistry metricsRegistry =
- enablePerTaskMetrics
- ? ctx.getTaskContext().getTaskMetricsRegistry()
- : ctx.getContainerContext().getContainerMetricsRegistry();
- transformLatency.putIfAbsent(
- transformName,
- metricsRegistry.newTimer(GROUP, getTimerWithCustomizedLookBackWindow(transformName)));
- transformOutputThroughPut.putIfAbsent(
- transformName,
- metricsRegistry.newCounter(
- GROUP, getMetricNameWithPrefix(TRANSFORM_OP_THROUGHPUT, transformName)));
- transformInputThroughput.putIfAbsent(
- transformName,
- metricsRegistry.newCounter(
- GROUP, getMetricNameWithPrefix(TRANSFORM_IP_THROUGHPUT, transformName)));
- transformCacheSize.putIfAbsent(
- transformName,
- ctx.getTaskContext()
- .getTaskMetricsRegistry()
- .newGauge(
- GROUP,
- getMetricNameWithPrefix(TRANSFORM_ARRIVAL_TIME_CACHE_SIZE, transformName),
- 0L));
- }
-
- public Timer getTransformLatencyMetric(String transformName) {
- return transformLatency.get(transformName);
- }
-
- public Counter getTransformInputThroughput(String transformName) {
- return transformInputThroughput.get(transformName);
- }
-
- public Counter getTransformOutputThroughput(String transformName) {
- return transformOutputThroughPut.get(transformName);
- }
-
- public Gauge getTransformCacheSize(String transformName) {
- return transformCacheSize.get(transformName);
- }
-
- public Gauge getTransformWatermarkProgress(String transformName) {
- return transformWatermarkProgress.get(transformName);
- }
-
- // Customize in-memory window size for timer, default from samza is 5 mins which causes memory
- // pressure if a lot of timers are registered
- private static Timer getTimerWithCustomizedLookBackWindow(String transformName) {
- return new Timer(
- getMetricNameWithPrefix(TRANSFORM_LATENCY_METRIC, transformName),
- new SlidingTimeWindowReservoir(DEFAULT_LOOKBACK_TIMER_WINDOW_SIZE_MS));
- }
-
- private static String getMetricNameWithPrefix(String metricName, String transformName) {
- // Replace all non-alphanumeric characters with underscore
- final String samzaSafeMetricName = transformName.replaceAll("[^A-Za-z0-9_]", "_");
- return String.format("%s-%s", samzaSafeMetricName, metricName);
- }
-}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/package-info.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/package-info.java
deleted file mode 100644
index 97415846e310..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/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.
- */
-
-/** Internal implementation of the Beam runner for Apache Samza. */
-package org.apache.beam.runners.samza.metrics;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/package-info.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/package-info.java
deleted file mode 100644
index 549a4d81c2de..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/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 Samza. */
-package org.apache.beam.runners.samza;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java
deleted file mode 100644
index e1bc9251a304..000000000000
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java
+++ /dev/null
@@ -1,188 +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.samza.runtime;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.stream.Collectors;
-import org.apache.beam.runners.core.DoFnRunner;
-import org.apache.beam.runners.samza.SamzaPipelineOptions;
-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.values.CausedByDrain;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.WindowedValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This {@link DoFnRunner} adds the capability of executing the {@link
- * org.apache.beam.sdk.transforms.DoFn.ProcessElement} in the thread pool, and returns the future to
- * the collector for the underlying async execution.
- */
-public class AsyncDoFnRunner implements DoFnRunner {
- private static final Logger LOG = LoggerFactory.getLogger(AsyncDoFnRunner.class);
-
- // A dummy key to represent null keys
- private static final Object NULL_KEY = new Object();
-
- private final DoFnRunner underlying;
- private final ExecutorService executor;
- private final OpEmitter emitter;
- private final FutureCollector futureCollector;
- private final boolean isStateful;
-
- /**
- * This map keeps track of the last outputFutures for a certain key. When the next element of the
- * key comes in, its outputFutures will be chained from the last outputFutures in the map. When
- * all futures of a key have been complete, the key entry will be removed. The map is bounded by
- * (bundle size * 2).
- */
- private final Map