Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 2 additions & 13 deletions .test-infra/jenkins/job_PerformanceTests_Python.groovy
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ class PerformanceTestConfigurations {
String itModule
// A benchmark defined flag, will pass to benchmark as "--beam_python_sdk_location".
// It's the location of Python SDK distribution archive which is required for TestDataflowRunner.
String pythonSdkLocation = ''
String pythonSdkLocation = 'build/apache-beam.tar.gz'
// A benchmark defined flag, will pass to benchmark as "--beam_runner"
String runner = 'TestDataflowRunner'
// A benchmark defined flag, will pass to benchmark as "--beam_it_timeout"
Expand Down Expand Up @@ -152,8 +152,7 @@ private void createPythonPerformanceTestJob(PerformanceTestConfigurations testCo
beam_it_class : testConfig.itClass,
beam_it_module : testConfig.itModule,
beam_prebuilt : 'true', // Python benchmark don't need to prebuild repo before running
beam_python_sdk_location: getSDKLocationFromModule(testConfig.pythonSdkLocation,
testConfig.itModule),
beam_python_sdk_location: testConfig.pythonSdkLocation,
beam_runner : testConfig.runner,
beam_it_timeout : testConfig.itTimeoutSec.toString(),
beam_it_args : joinPipelineArgs(testConfig.extraPipelineArgs),
Expand All @@ -172,13 +171,3 @@ private static String joinPipelineArgs(Map pipelineArgs) {
})
return pipelineArgList.join(',')
}


// Get relative path of sdk location based on itModule if the location is not provided.
private static String getSDKLocationFromModule(String pythonSDKLocation, String itModule) {
if (!pythonSDKLocation && itModule.startsWith(":sdks:python")) {
def projectPath = (itModule - ':sdks:python').replace(':', '/')
return (projectPath + "/build/apache-beam.tar.gz").substring(1)
}
return pythonSDKLocation
}
Original file line number Diff line number Diff line change
Expand Up @@ -1858,7 +1858,7 @@ class BeamModulePlugin implements Plugin<Project> {
outputs.dirs(project.ext.envdir)
}

def pythonSdkDeps = project.files(
project.ext.pythonSdkDeps = project.files(
project.fileTree(
dir: "${project.rootDir}",
include: ['model/**', 'sdks/python/**'],
Expand All @@ -1871,35 +1871,10 @@ class BeamModulePlugin implements Plugin<Project> {
])
)
def copiedSrcRoot = "${project.buildDir}/srcs"
def tarball = "apache-beam.tar.gz"

project.configurations { distConfig }

project.task('sdist', dependsOn: 'setupVirtualenv') {
doLast {
// Copy sdk sources to an isolated directory
project.copy {
from pythonSdkDeps
into copiedSrcRoot
}

// Build artifact
project.exec {
executable 'sh'
args '-c', ". ${project.ext.envdir}/bin/activate && cd ${copiedSrcRoot}/sdks/python && python setup.py -q sdist --formats zip,gztar --dist-dir ${project.buildDir}"
}
def collection = project.fileTree("${project.buildDir}"){ include '**/*.tar.gz' exclude '**/apache-beam.tar.gz', 'srcs/**'}

// we need a fixed name for the artifact
project.copy { from collection.singleFile; into "${project.buildDir}"; rename { tarball } }
}
inputs.files pythonSdkDeps
outputs.file "${project.buildDir}/${tarball}"
}

project.artifacts {
distConfig file: project.file("${project.buildDir}/${tarball}"), builtBy: project.sdist
}
// Create new configuration distTarBall which represents Python source
// distribution tarball generated by :sdks:python:sdist.
project.configurations { distTarBall }

project.task('installGcpTest', dependsOn: 'setupVirtualenv') {
doLast {
Expand All @@ -1909,7 +1884,7 @@ class BeamModulePlugin implements Plugin<Project> {
}
}
}
project.installGcpTest.mustRunAfter project.sdist
project.installGcpTest.mustRunAfter project.configurations.distTarBall

project.task('cleanPython') {
doLast {
Expand Down Expand Up @@ -1947,15 +1922,23 @@ class BeamModulePlugin implements Plugin<Project> {

project.ext.toxTask = { name, tox_env ->
project.tasks.create(name) {
dependsOn = ['sdist']
dependsOn 'setupVirtualenv'
dependsOn ':sdks:python:sdist'

doLast {
// Python source directory is also tox execution workspace, We want
// to isolate them per tox suite to avoid conflict when running
// multiple tox suites in parallel.
project.copy { from project.pythonSdkDeps; into copiedSrcRoot }

def copiedPyRoot = "${copiedSrcRoot}/sdks/python"
def distTarBall = "${pythonRootDir}/build/apache-beam.tar.gz"
project.exec {
executable 'sh'
args '-c', ". ${project.ext.envdir}/bin/activate && cd ${copiedPyRoot} && scripts/run_tox.sh $tox_env ${project.buildDir}/apache-beam.tar.gz"
args '-c', ". ${project.ext.envdir}/bin/activate && cd ${copiedPyRoot} && scripts/run_tox.sh $tox_env $distTarBall"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unrelated to this PR - do you remember why do we pass a tarball to tox suite? It looks like we started doing that with #8067.

Copy link
Contributor Author

@markflyhigh markflyhigh Aug 7, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

tox will build tarball for venv install automatically if not provided. This build depends on shared file when running in parallel and cause our test flaky. So we prebuild the tarball and pass it from --installpkg flag to avoid that issue.

}
}
inputs.files pythonSdkDeps
inputs.files project.pythonSdkDeps
outputs.files project.fileTree(dir: "${pythonRootDir}/target/.tox/${tox_env}/log/")
}
}
Expand All @@ -1969,7 +1952,7 @@ class BeamModulePlugin implements Plugin<Project> {

project.task('integrationTest') {
dependsOn 'installGcpTest'
dependsOn 'sdist'
dependsOn ':sdks:python:sdist'

doLast {
def argMap = [:]
Expand Down
26 changes: 26 additions & 0 deletions sdks/python/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,32 @@ task buildPython(dependsOn: 'setupVirtualenv') {
}
build.dependsOn buildPython

// Create a Python source distribution tarball.
def tarball = "apache-beam.tar.gz"
task sdist {
dependsOn setupVirtualenv

doLast {
// Build artifact
project.exec {
executable 'sh'
args '-c', ". ${project.ext.envdir}/bin/activate && python setup.py -q sdist --formats zip,gztar --dist-dir ${project.buildDir}"
}

def collection = fileTree(project.buildDir){ include '**/*.tar.gz' exclude '**/apache-beam.tar.gz', 'srcs/**'}

// we need a fixed name for the artifact
copy { from collection.singleFile; into project.buildDir; rename { tarball } }
println "Create distribution tar file ${tarball} in ${project.buildDir}"
}
inputs.files project.pythonSdkDeps
outputs.file "${project.buildDir}/${tarball}"
}

artifacts {
distTarBall file: file("${project.buildDir}/${tarball}"), builtBy: sdist
}


/*************************************************************************************************/
// Non-testing builds and analysis tasks
Expand Down
2 changes: 1 addition & 1 deletion sdks/python/container/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ dependencies {
build name: './github.com/apache/beam/sdks/go', dir: project(':sdks:go').projectDir
test name: './github.com/apache/beam/sdks/go', dir: project(':sdks:go').projectDir
}
sdkSourceTarball project(path: ":sdks:python", configuration: "distConfig")
sdkSourceTarball project(path: ":sdks:python", configuration: "distTarBall")
}

task copyDockerfileDependencies(type: Copy, dependsOn: goBuild) {
Expand Down
2 changes: 1 addition & 1 deletion sdks/python/container/py3/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ configurations {
}

dependencies {
sdkSourceTarball project(path: ":sdks:python", configuration: "distConfig")
sdkSourceTarball project(path: ":sdks:python", configuration: "distTarBall")
sdkHarnessLauncher project(path: ":sdks:python:container", configuration: "sdkHarnessLauncher")
}

Expand Down
24 changes: 14 additions & 10 deletions sdks/python/test-suites/dataflow/py2/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,10 @@ plugins { id 'org.apache.beam.module' }
applyPythonNature()
enablePythonPerformanceTest()

dependencies {
distTarBall project(path: ":sdks:python", configuration: "distTarBall")
}

def runScriptsDir = "${project.rootDir}/sdks/python/scripts"

// Basic test options for ITs running on Jenkins.
Expand All @@ -29,7 +33,7 @@ def basicTestOpts = [
"--process-timeout=4500", // timeout of whole command execution
]

task preCommitIT(dependsOn: ['sdist', 'installGcpTest']) {
task preCommitIT(dependsOn: [':sdks:python:sdist', 'installGcpTest']) {
dependsOn ":runners:google-cloud-dataflow-java:worker:shadowJar"

def dataflowWorkerJar = project(":runners:google-cloud-dataflow-java:worker").shadowJar.archivePath
Expand All @@ -48,7 +52,7 @@ task preCommitIT(dependsOn: ['sdist', 'installGcpTest']) {
]
def cmdArgs = project.mapToArgString([
"test_opts": testOpts,
"sdk_location": "${project.buildDir}/apache-beam.tar.gz",
"sdk_location": files(configurations.distTarBall.files).singleFile,
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This line actually can be removed for all similar integration tests sine the default sdk_location (defined in run_integration_test.sh) is pointing to the correct location. However, I left it here since people who read this configuration will know how it's set and related to :sdks:python:sdist. I'm open to discussion if you think it can be removed.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am ok to keep this. It feels like there may be a shorter way to reference the file defined by this configuration. Did you try configurations.distTarBall? Somehow it seems to work in

from configurations.sdkSourceTarball

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm afraid not. We need full path of this tarball, but configurations.distTarBall returns a string configuration ':sdks:python:test-suites:dataflow:py2:distTarBall'

"worker_jar": dataflowWorkerJar,
"suite": "preCommitIT-df"
])
Expand All @@ -61,7 +65,7 @@ task preCommitIT(dependsOn: ['sdist', 'installGcpTest']) {
}

// Run PostCommit integration tests on default runner (TestDataflowRunner)
task postCommitIT(dependsOn: ['installGcpTest', 'sdist']) {
task postCommitIT(dependsOn: ['installGcpTest', ':sdks:python:sdist']) {
dependsOn ":runners:google-cloud-dataflow-java:worker:shadowJar"

def dataflowWorkerJar = project(":runners:google-cloud-dataflow-java:worker").shadowJar.archivePath
Expand All @@ -70,7 +74,7 @@ task postCommitIT(dependsOn: ['installGcpTest', 'sdist']) {
def testOpts = basicTestOpts + ["--attr=IT"]
def cmdArgs = project.mapToArgString(["test_opts": testOpts,
"worker_jar": dataflowWorkerJar,
"sdk_location": "${project.buildDir}/apache-beam.tar.gz",
"sdk_location": files(configurations.distTarBall.files).singleFile,
"suite": "postCommitIT-df"])
exec {
executable 'sh'
Expand All @@ -79,7 +83,7 @@ task postCommitIT(dependsOn: ['installGcpTest', 'sdist']) {
}
}

task validatesRunnerBatchTests(dependsOn: ['installGcpTest', 'sdist']) {
task validatesRunnerBatchTests(dependsOn: ['installGcpTest', ':sdks:python:sdist']) {
dependsOn ":runners:google-cloud-dataflow-java:worker:shadowJar"

def dataflowWorkerJar = project(":runners:google-cloud-dataflow-java:worker").shadowJar.archivePath
Expand All @@ -88,7 +92,7 @@ task validatesRunnerBatchTests(dependsOn: ['installGcpTest', 'sdist']) {
def testOpts = basicTestOpts + ["--attr=ValidatesRunner"]
def cmdArgs = project.mapToArgString(["test_opts": testOpts,
"worker_jar": dataflowWorkerJar,
"sdk_location": "${project.buildDir}/apache-beam.tar.gz",
"sdk_location": files(configurations.distTarBall.files).singleFile,
"suite": "validatesRunnerBatchTests-df"])
exec {
executable 'sh'
Expand All @@ -97,7 +101,7 @@ task validatesRunnerBatchTests(dependsOn: ['installGcpTest', 'sdist']) {
}
}

task validatesRunnerStreamingTests(dependsOn: ['installGcpTest', 'sdist']) {
task validatesRunnerStreamingTests(dependsOn: ['installGcpTest', ':sdks:python:sdist']) {
dependsOn ":runners:google-cloud-dataflow-java:worker:shadowJar"

def dataflowWorkerJar = project(":runners:google-cloud-dataflow-java:worker").shadowJar.archivePath
Expand All @@ -108,7 +112,7 @@ task validatesRunnerStreamingTests(dependsOn: ['installGcpTest', 'sdist']) {
def argMap = ["test_opts": testOpts,
"streaming": "true",
"worker_jar": dataflowWorkerJar,
"sdk_location": "${project.buildDir}/apache-beam.tar.gz",
"sdk_location": files(configurations.distTarBall.files).singleFile,
"suite": "validatesRunnerStreamingTests-df"]
def cmdArgs = project.mapToArgString(argMap)
exec {
Expand All @@ -118,11 +122,11 @@ task validatesRunnerStreamingTests(dependsOn: ['installGcpTest', 'sdist']) {
}
}

task dataflowChicagoTaxiExample(dependsOn: ['installGcpTest', 'sdist']) {
task dataflowChicagoTaxiExample(dependsOn: ['installGcpTest', ':sdks:python:sdist']) {

def gcsRoot = findProperty('gcsRoot')
def runner = findProperty('runner')
def cliArgs = "${gcsRoot} ${runner} ${project.buildDir}/apache-beam.tar.gz"
def cliArgs = "${gcsRoot} ${runner} ${files(configurations.distTarBall.files).singleFile}"

doLast {
exec {
Expand Down
17 changes: 11 additions & 6 deletions sdks/python/test-suites/dataflow/py35/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,14 @@ apply plugin: org.apache.beam.gradle.BeamModulePlugin
applyPythonNature()
enablePythonPerformanceTest()

dependencies {
distTarBall project(path: ":sdks:python", configuration: "distTarBall")
}

// Required to setup a Python 3 virtualenv.
pythonVersion = '3.5'


def runScriptsDir = "${project.rootDir}/sdks/python/scripts"

// Basic test options for ITs running on Jenkins.
Expand All @@ -32,7 +37,7 @@ def basicTestOpts = [
"--process-timeout=4500", // timeout of whole command execution
]

task postCommitIT(dependsOn: ['sdist', 'installGcpTest']) {
task postCommitIT(dependsOn: [':sdks:python:sdist', 'installGcpTest']) {
dependsOn ":runners:google-cloud-dataflow-java:worker:shadowJar"

def dataflowWorkerJar = project(":runners:google-cloud-dataflow-java:worker").shadowJar.archivePath
Expand All @@ -42,7 +47,7 @@ task postCommitIT(dependsOn: ['sdist', 'installGcpTest']) {

def cmdArgs = project.mapToArgString([
"test_opts": testOpts,
"sdk_location": "${project.buildDir}/apache-beam.tar.gz",
"sdk_location": files(configurations.distTarBall.files).singleFile,
"worker_jar": dataflowWorkerJar,
"suite": "postCommitIT-df-py35"
])
Expand All @@ -53,7 +58,7 @@ task postCommitIT(dependsOn: ['sdist', 'installGcpTest']) {
}
}

task validatesRunnerBatchTests(dependsOn: ['installGcpTest', 'sdist']) {
task validatesRunnerBatchTests(dependsOn: ['installGcpTest', ':sdks:python:sdist']) {
dependsOn ":runners:google-cloud-dataflow-java:worker:shadowJar"

def dataflowWorkerJar = project(":runners:google-cloud-dataflow-java:worker").shadowJar.archivePath
Expand All @@ -62,7 +67,7 @@ task validatesRunnerBatchTests(dependsOn: ['installGcpTest', 'sdist']) {
def testOpts = basicTestOpts + ["--attr=ValidatesRunner"]
def cmdArgs = project.mapToArgString([
"test_opts": testOpts,
"sdk_location": "${project.buildDir}/apache-beam.tar.gz",
"sdk_location": files(configurations.distTarBall.files).singleFile,
"worker_jar": dataflowWorkerJar,
"suite": "validatesRunnerBatchTests-df-py35"
])
Expand All @@ -73,7 +78,7 @@ task validatesRunnerBatchTests(dependsOn: ['installGcpTest', 'sdist']) {
}
}

task validatesRunnerStreamingTests(dependsOn: ['installGcpTest', 'sdist']) {
task validatesRunnerStreamingTests(dependsOn: ['installGcpTest', ':sdks:python:sdist']) {
dependsOn ":runners:google-cloud-dataflow-java:worker:shadowJar"

def dataflowWorkerJar = project(":runners:google-cloud-dataflow-java:worker").shadowJar.archivePath
Expand All @@ -83,7 +88,7 @@ task validatesRunnerStreamingTests(dependsOn: ['installGcpTest', 'sdist']) {
def testOpts = basicTestOpts + ["--attr=ValidatesRunner,!sickbay-streaming"]
def argMap = ["test_opts": testOpts,
"streaming": "true",
"sdk_location": "${project.buildDir}/apache-beam.tar.gz",
"sdk_location": files(configurations.distTarBall.files).singleFile,
"worker_jar": dataflowWorkerJar,
"suite": "validatesRunnerStreamingTests-df-py35"]
def cmdArgs = project.mapToArgString(argMap)
Expand Down
Loading