Skip to content

Commit

Permalink
[BEAM-14170] - Create a test that runs sickbayed tests (#17471)
Browse files Browse the repository at this point in the history
  • Loading branch information
fernando-wizeline committed May 29, 2022
1 parent 0fb6886 commit 9a6f769
Show file tree
Hide file tree
Showing 9 changed files with 225 additions and 47 deletions.
43 changes: 43 additions & 0 deletions .test-infra/jenkins/job_PostCommit_Java_Sickbay.groovy
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

import CommonJobProperties as commonJobProperties
import PostcommitJobBuilder

// This job runs the Java sickbay tests.
PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Sickbay',
'Run Java Sickbay', 'Java Sickbay Tests', this) {

description('Run Java Sickbay Tests')

// Set common parameters.
commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120)

publishers {
archiveJunit('**/build/test-results/**/*.xml')
}

// Execute shell command to run sickbay tests.
steps {
gradle {
rootBuildScriptDir(commonJobProperties.checkoutDir)
tasks(':javaPostCommitSickbay')
commonJobProperties.setGradleSwitches(delegate)
}
}
}
8 changes: 8 additions & 0 deletions build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -207,6 +207,14 @@ tasks.register("javaPostCommit") {
dependsOn(":sdks:java:io:neo4j:integrationTest")
}

tasks.register("javaPostCommitSickbay") {
dependsOn(":runners:samza:validatesRunnerSickbay")
dependsOn(":runners:flink:validatesRunnerSickbay")
dependsOn(":runners:spark:validatesRunnerSickbay")
dependsOn(":runners:direct-java:validatesRunnerSickbay")
dependsOn(":runners:portability:java:validatesRunnerSickbay")
}

tasks.register("javaHadoopVersionsTest") {
dependsOn(":sdks:java:io:hadoop-common:hadoopVersionsTest")
dependsOn(":sdks:java:io:hadoop-file-system:hadoopVersionsTest")
Expand Down
38 changes: 36 additions & 2 deletions runners/direct-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,15 @@ static def pipelineOptionsStringCrossPlatformHandling(String[] options) {
}
}

def sickbayTests = [
// https://issues.apache.org/jira/browse/BEAM-2791
'org.apache.beam.sdk.testing.UsesLoopingTimer',
// https://issues.apache.org/jira/browse/BEAM-8035
'org.apache.beam.sdk.transforms.WatchTest.testMultiplePollsWithManyResults',
// https://issues.apache.org/jira/browse/BEAM-6354
'org.apache.beam.sdk.io.BoundedReadFromUnboundedSourceTest.testTimeBound',
]

task needsRunnerTests(type: Test) {
group = "Verification"
description = "Runs tests that require a runner to validate that piplines/transforms work correctly"
Expand All @@ -131,6 +140,13 @@ task needsRunnerTests(type: Test) {
testClassesDirs += files(project(":runners:core-java").sourceSets.test.output.classesDirs)
testClassesDirs += files(project(":sdks:java:core").sourceSets.test.output.classesDirs)
useJUnit {

filter {
for (String test : sickbayTests) {
excludeTestsMatching test
}
}

includeCategories "org.apache.beam.sdk.testing.NeedsRunner"
// Should be run only in a properly configured SDK harness environment
excludeCategories 'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment'
Expand All @@ -140,8 +156,6 @@ task needsRunnerTests(type: Test) {
excludeCategories "org.apache.beam.sdk.testing.UsesJavaExpansionService"
excludeCategories "org.apache.beam.sdk.testing.UsesPythonExpansionService"
excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
// https://issues.apache.org/jira/browse/BEAM-2791
excludeCategories 'org.apache.beam.sdk.testing.UsesLoopingTimer'
}
testLogging {
outputs.upToDateWhen {false}
Expand Down Expand Up @@ -178,6 +192,26 @@ task validatesRunner(type: Test) {
}
}

tasks.register("validatesRunnerSickbay", Test) {
group = "Verification"
description "Validates Direct runner (Sickbay Tests)"
systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
"--runner=DirectRunner",
])

classpath = configurations.needsRunner
testClassesDirs = files(project(":sdks:java:core").sourceSets.test.output.classesDirs)

filter {
for (String test : sickbayTests) {
includeTestsMatching test
}

// https://issues.apache.org/jira/browse/BEAM-4191
includeTestsMatching 'org.apache.beam.runners.direct.WatermarkManagerTest.updateWatermarkWithDifferentWindowedValueInstances'
}
}

def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing'
def gcsBucket = project.findProperty('gcsBucket') ?: 'temp-storage-for-release-validation-tests/nightly-snapshot-validation'
def bqDataset = project.findProperty('bqDataset') ?: 'beam_postrelease_mobile_gaming'
Expand Down
35 changes: 30 additions & 5 deletions runners/flink/flink_runner.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -220,8 +220,14 @@ class ValidatesRunnerConfig {
String name
boolean streaming
boolean checkpointing
ArrayList<String> sickbayTests
}

def sickbayTests = [
// TODO(BEAM-13573)
'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testOnWindowTimestampSkew',
]

def createValidatesRunnerTask(Map m) {
def config = m as ValidatesRunnerConfig
tasks.register(config.name, Test) {
Expand Down Expand Up @@ -279,27 +285,29 @@ def createValidatesRunnerTask(Map m) {
excludeCategories 'org.apache.beam.sdk.testing.UsesPerKeyOrderInBundle'
}
filter {
for (String test : config.sickbayTests) {
excludeTestsMatching test
}

// https://issues.apache.org/jira/browse/BEAM-12039
excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testDiscardingMode'
// https://issues.apache.org/jira/browse/BEAM-12037
excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testFirstElementLate'
// https://issues.apache.org/jira/browse/BEAM-12038
excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testLateDataAccumulating'
// TODO(BEAM-13573)
excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testOnWindowTimestampSkew'
}
}
}
}

createValidatesRunnerTask(name: "validatesRunnerBatch", streaming: false)
createValidatesRunnerTask(name: "validatesRunnerStreaming", streaming: true)
createValidatesRunnerTask(name: "validatesRunnerBatch", streaming: false, sickbayTests: sickbayTests)
createValidatesRunnerTask(name: "validatesRunnerStreaming", streaming: true, sickbayTests: sickbayTests)
// We specifically have a variant which runs with checkpointing enabled for the
// tests that require it since running a checkpoint variant is significantly
// slower since we have to wait shutdownSourcesAfterIdleMs before the source
// can shutdown because of https://issues.apache.org/jira/browse/FLINK-2491
// not supporting checkpointing when an operator has been shutdown.
createValidatesRunnerTask(name: "validatesRunnerStreamingCheckpointing", streaming: true, checkpointing: true)
createValidatesRunnerTask(name: "validatesRunnerStreamingCheckpointing", streaming: true, checkpointing: true, sickbayTests: sickbayTests)

tasks.register('validatesRunner') {
group = 'Verification'
Expand All @@ -309,6 +317,23 @@ tasks.register('validatesRunner') {
dependsOn validatesRunnerStreamingCheckpointing
}

tasks.register("validatesRunnerSickbay", Test) {
group = "Verification"
description "Validates Flink runner (Sickbay Tests)"
systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
"--runner=TestFlinkRunner",
])

classpath = configurations.validatesRunner
testClassesDirs = files(project(":sdks:java:core").sourceSets.test.output.classesDirs)

filter {
for (String test : sickbayTests) {
includeTestsMatching test
}
}
}

// Generates :runners:flink:1.13:runQuickstartJavaFlinkLocal
createJavaExamplesArchetypeValidationTask(type: 'Quickstart', runner: 'FlinkLocal')

Expand Down
29 changes: 27 additions & 2 deletions runners/portability/java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,12 @@ task stopLocalJobService {

startLocalJobService.configure{finalizedBy stopLocalJobService}

def sickbayTests = [

//TODO: https://issues.apache.org/jira/browse/BEAM-10995
'org.apache.beam.sdk.transforms.windowing.WindowingTest.testWindowPreservation',
]

/**
* Runs Java ValidatesRunner tests against the Universal Local Runner (ULR) aka local_job_service_main
* with the specified environment type.
Expand Down Expand Up @@ -193,8 +199,10 @@ def createUlrValidatesRunnerTask = { name, environmentType, dockerImageTask = ""
excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testWindowedMapAsEntrySetSideInput'
excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testMultimapAsEntrySetSideInput'

// https://issues.apache.org/jira/browse/BEAM-10995
excludeTestsMatching 'org.apache.beam.sdk.transforms.windowing.WindowingTest.testWindowPreservation'
for (String test : sickbayTests) {
excludeTestsMatching test
}

}
}

Expand All @@ -213,6 +221,23 @@ def createUlrValidatesRunnerTask = { name, environmentType, dockerImageTask = ""
return vrTask
}

tasks.register("validatesRunnerSickbay", Test) {
group = "Verification"
description "Validates Universal local runner (Sickbay Tests)"
systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
"--runner=TestUniversalRunner",
])

classpath = configurations.validatesRunner
testClassesDirs = files(project(":sdks:java:core").sourceSets.test.output.classesDirs)

filter {
for (String test : sickbayTests) {
includeTestsMatching test
}
}
}

task ulrDockerValidatesRunner {
dependsOn createUlrValidatesRunnerTask("ulrDockerValidatesRunnerTests", "DOCKER", ":sdks:java:container:java8:docker")
}
Expand Down
78 changes: 50 additions & 28 deletions runners/samza/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,11 @@ 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',
exportJavadoc: false,
classesTriggerCheckerBugs: [
'GroupWithoutRepartition': 'https://github.com/typetools/checker-framework/issues/3791',
],
automaticModuleName: 'org.apache.beam.runners.samza',
)

description = "Apache Beam :: Runners :: Samza"
Expand Down Expand Up @@ -86,6 +86,31 @@ configurations.all {
exclude group: "org.slf4j", module: "slf4j-jdk14"
}

def sickbayTests = [
// TODO(BEAM-12750)
'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInGlobalWindowBatchSizeByteSizeFn',
'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInStreamingMode',
'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testWithShardedKeyInGlobalWindow',
// TODO(BEAM-12749)
'org.apache.beam.sdk.transforms.MapElementsTest.testMapSimpleFunction',
// TODO(BEAM-12748)
'org.apache.beam.sdk.transforms.ViewTest.testEmptySingletonSideInput',
'org.apache.beam.sdk.transforms.ViewTest.testNonSingletonSideInput',
// TODO(BEAM-12747)
'org.apache.beam.sdk.transforms.WithTimestampsTest.withTimestampsBackwardsInTimeShouldThrow',
'org.apache.beam.sdk.transforms.WithTimestampsTest.withTimestampsWithNullTimestampShouldThrow',
// TODO(BEAM-12746)
'org.apache.beam.sdk.io.FileIOTest*',
// TODO(BEAM-12745)
'org.apache.beam.sdk.io.AvroIOTest*',
// TODO(BEAM-12744)
'org.apache.beam.sdk.PipelineTest.testEmptyPipeline',
// TODO(BEAM-12743)
'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',
]
tasks.register("validatesRunner", Test) {
group = "Verification"
description "Validates Samza runner"
Expand Down Expand Up @@ -116,6 +141,9 @@ tasks.register("validatesRunner", Test) {
excludeCategories 'org.apache.beam.sdk.testing.UsesLoopingTimer'
}
filter {
for (String test : sickbayTests) {
excludeTestsMatching test
}
// TODO(BEAM-10025)
excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestampDefaultUnbounded'
// TODO(BEAM-11479)
Expand All @@ -126,29 +154,6 @@ tasks.register("validatesRunner", Test) {
excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testFirstElementLate'
// TODO(BEAM-12036)
excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testLateDataAccumulating'
// TODO(BEAM-12743)
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(BEAM-12744)
excludeTestsMatching 'org.apache.beam.sdk.PipelineTest.testEmptyPipeline'
// TODO(BEAM-12745)
excludeTestsMatching 'org.apache.beam.sdk.io.AvroIOTest*'
// TODO(BEAM-12746)
excludeTestsMatching 'org.apache.beam.sdk.io.FileIOTest*'
// TODO(BEAM-12747)
excludeTestsMatching 'org.apache.beam.sdk.transforms.WithTimestampsTest.withTimestampsBackwardsInTimeShouldThrow'
excludeTestsMatching 'org.apache.beam.sdk.transforms.WithTimestampsTest.withTimestampsWithNullTimestampShouldThrow'
// TODO(BEAM-12748)
excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testEmptySingletonSideInput'
excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testNonSingletonSideInput'
// TODO(BEAM-12749)
excludeTestsMatching 'org.apache.beam.sdk.transforms.MapElementsTest.testMapSimpleFunction'
// TODO(BEAM-12750)
excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInGlobalWindowBatchSizeByteSizeFn'
excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInStreamingMode'
excludeTestsMatching 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testWithShardedKeyInGlobalWindow'

// 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'
Expand All @@ -166,5 +171,22 @@ tasks.register("validatesRunner", Test) {
}
}

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')

0 comments on commit 9a6f769

Please sign in to comment.