Skip to content

Commit

Permalink
[Prism] Enable Java validatesRunner tests on Prism (#31075)
Browse files Browse the repository at this point in the history
* Create :runners:prism:java:validatesRunner task

* Configure minimal needed to run portability tests

* Condence path resolutions in gradle configuration
  • Loading branch information
damondouglas committed May 2, 2024
1 parent ee329ef commit 6995666
Show file tree
Hide file tree
Showing 3 changed files with 75 additions and 2 deletions.
11 changes: 9 additions & 2 deletions runners/portability/java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -54,14 +54,17 @@ dependencies {
validatesRunner project(path: project.path, configuration: "testRuntimeMigration")
}

// Set using -PjobEndpoint=, for example: -PjobEndpoint=localhost:8073
def jobEndpointPropertyName = "jobEndpoint"

project.evaluationDependsOn(":sdks:java:core")
project.evaluationDependsOn(":sdks:python")
project.evaluationDependsOn(":runners:core-java")

ext.pythonSdkDir = project.findProject(":sdks:python").getProjectDir().getAbsoluteFile().toString()

// If this is set via -P then we assume it is already running and will not start it
def needsLocalJobService = !project.hasProperty("localJobServicePortFile")
def needsLocalJobService = !project.hasProperty("localJobServicePortFile") && !project.hasProperty(jobEndpointPropertyName)

ext.localJobServicePortFile = project.hasProperty("localJobServicePortFile") ? project.property("localJobServicePortFile") : "${project.buildDir}/local_job_service_port"

Expand Down Expand Up @@ -139,11 +142,15 @@ def createUlrValidatesRunnerTask = { name, environmentType, dockerImageTask = ""
Task vrTask = tasks.create(name: name, type: Test, group: "Verification") {
description "PortableRunner Java docker ValidatesRunner suite"
classpath = configurations.validatesRunner
var jobEndpointOption = "--localJobServicePortFile=$localJobServicePortFile"
if (project.hasProperty(jobEndpointPropertyName)) {
jobEndpointOption = "--$jobEndpointPropertyName=${project.findProperty(jobEndpointPropertyName)}"
}
systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
"--runner=TestUniversalRunner",
"--experiments=beam_fn_api",
"--defaultEnvironmentType=${environmentType}",
"--localJobServicePortFile=${localJobServicePortFile}"
jobEndpointOption
])
testClassesDirs = files(project(":sdks:java:core").sourceSets.test.output.classesDirs)
useJUnit {
Expand Down
65 changes: 65 additions & 0 deletions runners/prism/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* 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.
*/

plugins { id 'org.apache.beam.module' }
applyGoNature()

description = "Apache Beam :: Runners :: Prism"
ext.summary = "Tasks related to executing the Prism Runner"

// jobPort Property optionally configures the Prism job_port flag.
def jobPort = findProperty("jobPort")

// jobPortFlag is Prism service's job management service port flag name.
def jobPortFlag = "job_port"

// buildTarget is the go build output target path for the Prism service.
def buildTarget = layout.buildDirectory.get().file("tmp/prism")

// modDir is the path containing the go.mod file.
def modDir = project.rootDir.toPath().resolve("sdks")

// prismDir is the directory containing the prism executable.
def prismDir = modDir.resolve("go/cmd/prism")

// Overrides the gradle build task to build the prism executable.
def buildTask = tasks.named("build") {
// goPrepare is a task registered in applyGoNature.
dependsOn("goPrepare")
doLast {
exec {
workingDir = modDir
executable = 'sh'
args = ["-c", "${project.ext.goCmd} build -o $buildTarget $prismDir"]
}
}
}

tasks.register("runServer", Exec) {
dependsOn(buildTask)
group = "application"
description = "Run the Prism server"

List<String> command = new ArrayList<>()
command.add(buildTarget.asFile.path)
if (jobPort != null) {
command.add("-$jobPortFlag=$jobPort" as String)
}

commandLine = command
}
1 change: 1 addition & 0 deletions settings.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,7 @@ include(":runners:java-job-service")
include(":runners:jet")
include(":runners:local-java")
include(":runners:portability:java")
include(":runners:prism")
include(":runners:spark:3")
include(":runners:spark:3:job-server")
include(":runners:spark:3:job-server:container")
Expand Down

0 comments on commit 6995666

Please sign in to comment.