Skip to content

Commit

Permalink
Fix XVR test exclusions (#30354)
Browse files Browse the repository at this point in the history
* Fix XVR test exclusions

* Add trigger files
  • Loading branch information
Abacn authored Feb 21, 2024
1 parent 6a8c27e commit 9d2fea4
Show file tree
Hide file tree
Showing 24 changed files with 49 additions and 18 deletions.
Empty file.
Empty file.
Empty file.
Empty file.
Empty file.
Empty file.
Empty file.
Empty file.
Empty file.
Original file line number Diff line number Diff line change
Expand Up @@ -2680,6 +2680,7 @@ class BeamModulePlugin implements Plugin<Project> {

project.evaluationDependsOn(":sdks:python")
project.evaluationDependsOn(":sdks:java:testing:expansion-service")
project.evaluationDependsOn(":sdks:java:core")
project.evaluationDependsOn(":sdks:java:extensions:python")
project.evaluationDependsOn(":sdks:go:test")

Expand Down Expand Up @@ -2744,9 +2745,11 @@ class BeamModulePlugin implements Plugin<Project> {
systemProperty "expansionPort", port
systemProperty "semiPersistDir", config.semiPersistDir
classpath = config.classpath + project.files(
project.project(":sdks:java:core").sourceSets.test.runtimeClasspath,
project.project(":sdks:java:extensions:python").sourceSets.test.runtimeClasspath
)
testClassesDirs = project.files(
project.project(":sdks:java:core").sourceSets.test.output.classesDirs,
project.project(":sdks:java:extensions:python").sourceSets.test.output.classesDirs
)
maxParallelForks config.numParallelTests
Expand Down
6 changes: 2 additions & 4 deletions runners/direct-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -142,8 +142,7 @@ task needsRunnerTests(type: Test) {
excludeCategories "org.apache.beam.sdk.testing.LargeKeys\$Above100MB"
// MetricsPusher isn't implemented in direct runner
excludeCategories "org.apache.beam.sdk.testing.UsesMetricsPusher"
excludeCategories "org.apache.beam.sdk.testing.UsesJavaExpansionService"
excludeCategories "org.apache.beam.sdk.testing.UsesPythonExpansionService"
excludeCategories "org.apache.beam.sdk.testing.UsesExternalService"
excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
}
testLogging {
Expand Down Expand Up @@ -173,8 +172,7 @@ task validatesRunner(type: Test) {
excludeCategories 'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment'
excludeCategories "org.apache.beam.sdk.testing.LargeKeys\$Above100MB"
excludeCategories 'org.apache.beam.sdk.testing.UsesMetricsPusher'
excludeCategories "org.apache.beam.sdk.testing.UsesJavaExpansionService"
excludeCategories "org.apache.beam.sdk.testing.UsesPythonExpansionService"
excludeCategories "org.apache.beam.sdk.testing.UsesExternalService"
// https://github.com/apache/beam/issues/18499
excludeCategories 'org.apache.beam.sdk.testing.UsesLoopingTimer'
}
Expand Down
1 change: 1 addition & 0 deletions runners/flink/flink_runner.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,7 @@ def createValidatesRunnerTask(Map m) {
excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
} else {
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.UsesBundleFinalizer'
Expand Down
3 changes: 1 addition & 2 deletions runners/flink/job-server/flink_job_server.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -167,8 +167,7 @@ def portableValidatesRunnerTask(String name, boolean streaming, boolean checkpoi
// 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.UsesJavaExpansionService'
excludeCategories 'org.apache.beam.sdk.testing.UsesPythonExpansionService'
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'
Expand Down
4 changes: 2 additions & 2 deletions runners/google-cloud-dataflow-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -180,8 +180,7 @@ def commonLegacyExcludeCategories = [
'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment',
'org.apache.beam.sdk.testing.LargeKeys$Above10MB',
'org.apache.beam.sdk.testing.UsesAttemptedMetrics',
'org.apache.beam.sdk.testing.UsesJavaExpansionService',
'org.apache.beam.sdk.testing.UsesPythonExpansionService',
'org.apache.beam.sdk.testing.UsesExternalService',
'org.apache.beam.sdk.testing.UsesDistributionMetrics',
'org.apache.beam.sdk.testing.UsesGaugeMetrics',
'org.apache.beam.sdk.testing.UsesMultimapState',
Expand All @@ -192,6 +191,7 @@ def commonLegacyExcludeCategories = [
]

def commonRunnerV2ExcludeCategories = [
'org.apache.beam.sdk.testing.UsesExternalService',
'org.apache.beam.sdk.testing.UsesGaugeMetrics',
'org.apache.beam.sdk.testing.UsesSetState',
'org.apache.beam.sdk.testing.UsesMapState',
Expand Down
1 change: 1 addition & 0 deletions runners/jet/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ task validatesRunnerBatch(type: Test) {
useJUnit {
includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
// Should be run only in a properly configured SDK harness environment
excludeCategories 'org.apache.beam.sdk.testing.UsesExternalService'
excludeCategories 'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment'
excludeCategories "org.apache.beam.sdk.testing.LargeKeys\$Above100MB"
excludeCategories 'org.apache.beam.sdk.testing.UsesTimerMap'
Expand Down
1 change: 1 addition & 0 deletions runners/portability/java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,7 @@ def createUlrValidatesRunnerTask = { name, environmentType, dockerImageTask = ""
useJUnit {
includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
// Should be run only in a properly configured SDK harness environment
excludeCategories 'org.apache.beam.sdk.testing.UsesExternalService'
excludeCategories 'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment'
excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
excludeCategories 'org.apache.beam.sdk.testing.UsesOnWindowExpiration'
Expand Down
1 change: 1 addition & 0 deletions runners/samza/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,7 @@ tasks.register("validatesRunner", Test) {
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'
Expand Down
3 changes: 1 addition & 2 deletions runners/samza/job-server/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -86,8 +86,7 @@ def portableValidatesRunnerTask(String name, boolean docker) {
// 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.UsesJavaExpansionService'
excludeCategories 'org.apache.beam.sdk.testing.UsesPythonExpansionService'
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'
Expand Down
2 changes: 2 additions & 0 deletions runners/spark/job-server/spark_job_server.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@ def portableValidatesRunnerTask(String name, boolean streaming, boolean docker,

testCategories = {
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.FlattenWithHeterogeneousCoders'
Expand Down Expand Up @@ -167,6 +168,7 @@ def portableValidatesRunnerTask(String name, boolean streaming, boolean docker,
// Batch
testCategories = {
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.FlattenWithHeterogeneousCoders'
Expand Down
9 changes: 3 additions & 6 deletions runners/spark/spark_runner.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -274,8 +274,7 @@ def applyBatchValidatesRunnerSetup = { Test it ->
// SDF
excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
// Portability
excludeCategories 'org.apache.beam.sdk.testing.UsesJavaExpansionService'
excludeCategories 'org.apache.beam.sdk.testing.UsesPythonExpansionService'
excludeCategories 'org.apache.beam.sdk.testing.UsesExternalService'
excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
// Ordering
excludeCategories 'org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery'
Expand Down Expand Up @@ -349,8 +348,7 @@ def validatesRunnerStreaming = tasks.register("validatesRunnerStreaming", Test)
excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
excludeCategories 'org.apache.beam.sdk.testing.UsesBoundedSplittableParDo'
// Portability
excludeCategories 'org.apache.beam.sdk.testing.UsesJavaExpansionService'
excludeCategories 'org.apache.beam.sdk.testing.UsesPythonExpansionService'
excludeCategories 'org.apache.beam.sdk.testing.UsesExternalService'
excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
// Ordering
excludeCategories 'org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery'
Expand Down Expand Up @@ -405,8 +403,7 @@ tasks.register("validatesStructuredStreamingRunnerBatch", Test) {
// SDF
excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
// Portability
excludeCategories 'org.apache.beam.sdk.testing.UsesJavaExpansionService'
excludeCategories 'org.apache.beam.sdk.testing.UsesPythonExpansionService'
excludeCategories 'org.apache.beam.sdk.testing.UsesExternalService'
excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
excludeCategories 'org.apache.beam.sdk.testing.UsesTriggeredSideInputs'
}
Expand Down
1 change: 1 addition & 0 deletions runners/twister2/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ def validatesRunnerBatch = tasks.register("validatesRunnerBatch", Test) {
forkEvery 1
useJUnit {
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.FlattenWithHeterogeneousCoders'
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* 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.sdk.testing;

import org.apache.beam.sdk.annotations.Internal;

/**
* Category tag for tests which relies on a pre-defined port, such as expansion service or transform
* service. Tests tagged with {@link UsesExternalService} should initialize such port before the
* test execution.
*/
@Internal
public interface UsesExternalService {}
Original file line number Diff line number Diff line change
Expand Up @@ -24,4 +24,4 @@
* UsesJavaExpansionService} should be run for runners which support cross-language transforms.
*/
@Internal
public interface UsesJavaExpansionService {}
public interface UsesJavaExpansionService extends UsesExternalService {}
Original file line number Diff line number Diff line change
Expand Up @@ -24,4 +24,4 @@
* UsesPythonExpansionService} should be run for runners which support cross-language transforms.
*/
@Internal
public interface UsesPythonExpansionService {}
public interface UsesPythonExpansionService extends UsesExternalService {}

0 comments on commit 9d2fea4

Please sign in to comment.