You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by cc...@apache.org on 2019/03/14 01:43:44 UTC

[beam] branch master updated: [BEAM-6527] Use Gradle to parallel Python tox tests

This is an automated email from the ASF dual-hosted git repository.

ccy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 8c3fbd6  [BEAM-6527] Use Gradle to parallel Python tox tests
     new babd948  Merge pull request #7675 from markflyhigh/py3-tox-2
8c3fbd6 is described below

commit 8c3fbd64045678773b031731f5c1e4546ce42867
Author: Mark Liu <ma...@google.com>
AuthorDate: Wed Jan 30 10:39:21 2019 -0800

    [BEAM-6527] Use Gradle to parallel Python tox tests
---
 build.gradle                                       |  5 +-
 .../org/apache/beam/gradle/BeamModulePlugin.groovy | 41 +++++++++++++++-
 sdks/python/build.gradle                           | 56 ++--------------------
 sdks/python/scripts/generate_pydoc.sh              |  2 +-
 sdks/python/scripts/run_integration_test.sh        |  2 +-
 sdks/python/scripts/run_mini_py3lint.sh            |  2 +-
 sdks/python/scripts/run_pylint.sh                  |  2 +-
 sdks/python/scripts/run_pylint_2to3.sh             |  2 +-
 sdks/python/scripts/run_tox.sh                     | 17 +++++--
 sdks/python/scripts/run_tox_cleanup.sh             |  2 +-
 sdks/python/test-suites/tox/py35/build.gradle      | 53 ++++++++++++++++++++
 sdks/python/test-suites/tox/py36/build.gradle      | 34 +++++++++++++
 sdks/python/tox.ini                                |  2 +-
 settings.gradle                                    |  4 ++
 14 files changed, 157 insertions(+), 67 deletions(-)

diff --git a/build.gradle b/build.gradle
index 4c47a26..e34eaa2 100644
--- a/build.gradle
+++ b/build.gradle
@@ -191,7 +191,10 @@ task goIntegrationTests() {
 }
 
 task pythonPreCommit() {
-  dependsOn ":beam-sdks-python:preCommit"
+  dependsOn ":beam-sdks-python:preCommitPy2"
+  dependsOn ":beam-sdks-python-test-suites-tox-py35:preCommitPy35"
+  dependsOn ":beam-sdks-python-test-suites-tox-py36:preCommitPy36"
+  dependsOn ":beam-sdks-python-test-suites-dataflow:preCommitIT"
 }
 
 task pythonPostCommit() {
diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
index 7c50a52..beb27e6 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -1611,16 +1611,38 @@ class BeamModulePlugin implements Plugin<Project> {
         outputs.dirs(project.ext.envdir)
       }
 
+      def pythonSdkDeps = project.files(
+              project.fileTree(
+              dir: "${project.rootDir}",
+              include: ['model/**', 'sdks/python/**'],
+              // Exclude temporary directories used in build and test.
+              exclude: [
+                'sdks/python/build/**',
+                'sdks/python/dist/**',
+                'sdks/python/target/**',
+                'sdks/python/test-suites/**',
+              ])
+              )
+      def copiedSrcRoot = "${project.buildDir}/srcs"
+
       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 ${pythonRootDir} && python setup.py sdist --keep-temp --formats zip,gztar --dist-dir ${project.buildDir}"
+            args '-c', ". ${project.ext.envdir}/bin/activate && cd ${copiedSrcRoot}/sdks/python && python setup.py sdist --formats zip,gztar --dist-dir ${project.buildDir}"
           }
           def collection = project.fileTree("${project.buildDir}"){ include '**/*.tar.gz' exclude '**/apache-beam.tar.gz'}
           println "sdist archive name: ${collection.singleFile}"
+
           // we need a fixed name for the artifact
           project.copy { from collection.singleFile; into "${project.buildDir}"; rename { 'apache-beam.tar.gz' } }
         }
@@ -1646,7 +1668,7 @@ class BeamModulePlugin implements Plugin<Project> {
           project.exec {
             executable 'sh'
             args '-c', "if [ -e ${activate} ]; then " +
-                    ". ${activate} && python ${pythonRootDir}/setup.py clean; " +
+                    ". ${activate} && cd ${pythonRootDir} && python setup.py clean; " +
                     "fi"
           }
           project.delete project.buildDir     // Gradle build directory
@@ -1673,6 +1695,21 @@ class BeamModulePlugin implements Plugin<Project> {
         }
         return argList.join(' ')
       }
+
+      project.ext.toxTask = { name, tox_env ->
+        project.tasks.create(name) {
+          dependsOn = ['sdist']
+          doLast {
+            def copiedPyRoot = "${copiedSrcRoot}/sdks/python"
+            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"
+            }
+          }
+          inputs.files pythonSdkDeps
+          outputs.files project.fileTree(dir: "${pythonRootDir}/target/.tox/${tox_env}/log/")
+        }
+      }
     }
   }
 }
diff --git a/sdks/python/build.gradle b/sdks/python/build.gradle
index b83e614..f13eb63 100644
--- a/sdks/python/build.gradle
+++ b/sdks/python/build.gradle
@@ -38,34 +38,8 @@ build.dependsOn buildPython
 
 
 /*************************************************************************************************/
-// Unit testing
-
-def pythonSdkDeps = files(
-    fileTree(dir: 'apache_beam', includes: ['**/*.py', '**/*.pyx', '**/*.pxd']),
-    fileTree(dir: 'apache_beam/testing/data'),
-    fileTree(dir: "${project.rootDir}/model"),
-    fileTree(dir: 'scripts'),
-    ".pylintrc",
-    "MANIFEST.in",
-    "gen_protos.py",
-    "setup.cfg",
-    "setup.py",
-    "test_config.py",
-    "tox.ini")
-
-def toxTask = {
-  name, tox_env -> tasks.create(name) {
-    dependsOn = ['setupVirtualenv']
-    doLast {
-      exec {
-        executable 'sh'
-        args '-c', ". ${project.ext.envdir}/bin/activate && ./scripts/run_tox.sh $tox_env"
-      }
-    }
-    inputs.files pythonSdkDeps
-    outputs.files fileTree(dir: "${project.rootDir}/sdks/python/target/.tox/${tox_env}/log/")
-  }
-}
+// Unit tests for Python 2
+// See Python 3 tests in test-suites/tox
 
 task lint {}
 check.dependsOn lint
@@ -76,24 +50,12 @@ lint.dependsOn lintPy27
 toxTask "lintPy27_3", "py27-lint3"
 lint.dependsOn lintPy27_3
 
-toxTask "lintPy35", "py35-lint"
-lint.dependsOn lintPy35
-
 toxTask "testPy2Gcp", "py27-gcp"
 test.dependsOn testPy2Gcp
 
-toxTask "testPy35Gcp", "py35-gcp"
-test.dependsOn testPy35Gcp
-
 toxTask "testPython2", "py27"
 test.dependsOn testPython2
 
-toxTask "testPython35", "py35"
-test.dependsOn testPython35
-
-toxTask "testPython36", "py36"
-test.dependsOn testPython36
-
 toxTask "testPy2Cython", "py27-cython"
 test.dependsOn testPy2Cython
 // Ensure that testPy2Cython runs exclusively to other tests. This line is not
@@ -101,29 +63,17 @@ test.dependsOn testPy2Cython
 // project.
 testPy2Cython.mustRunAfter testPython2, testPy2Gcp
 
-toxTask "testPy35Cython", "py35-cython"
-test.dependsOn testPy35Cython
-// Ensure that testPy35Cython runs exclusively to other tests. This line is not
-// actually required, since gradle doesn't do parallel execution within a
-// project.
-testPy35Cython.mustRunAfter testPython35, testPy35Gcp
-
 toxTask "docs", "docs"
 assemble.dependsOn docs
 
 toxTask "cover", "cover"
 
-task preCommit() {
+task preCommitPy2() {
   dependsOn "docs"
   dependsOn "testPy2Cython"
-  dependsOn "testPy35Cython"
   dependsOn "testPython2"
-  dependsOn "testPython35"
-  dependsOn "testPython36"
   dependsOn "testPy2Gcp"
-  dependsOn "testPy35Gcp"
   dependsOn "lint"
-  dependsOn ":beam-sdks-python-test-suites-dataflow:preCommitIT"
 }
 
 task portablePreCommit() {
diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh
index 4929c5e..a9bffe3 100755
--- a/sdks/python/scripts/generate_pydoc.sh
+++ b/sdks/python/scripts/generate_pydoc.sh
@@ -29,7 +29,7 @@ if [[ $PWD != *sdks/python* ]]; then
 fi
 
 # Go to the Apache Beam Python SDK root
-if [[ "*sdks/python" != $PWD ]]; then
+if [[ $PWD != *sdks/python ]]; then
   cd $(pwd | sed 's/sdks\/python.*/sdks\/python/')
 fi
 
diff --git a/sdks/python/scripts/run_integration_test.sh b/sdks/python/scripts/run_integration_test.sh
index 273f963..f0c01ed 100755
--- a/sdks/python/scripts/run_integration_test.sh
+++ b/sdks/python/scripts/run_integration_test.sh
@@ -163,7 +163,7 @@ if [[ -z $PIPELINE_OPTS ]]; then
   fi
 
   # Go to the Apache Beam Python SDK root
-  if [[ "*sdks/python" != $PWD ]]; then
+  if [[ $PWD != *sdks/python ]]; then
     cd $(pwd | sed 's/sdks\/python.*/sdks\/python/')
   fi
 
diff --git a/sdks/python/scripts/run_mini_py3lint.sh b/sdks/python/scripts/run_mini_py3lint.sh
index 0729c7b..27ca3ce 100755
--- a/sdks/python/scripts/run_mini_py3lint.sh
+++ b/sdks/python/scripts/run_mini_py3lint.sh
@@ -29,7 +29,7 @@ if [[ $PWD != *sdks/python* ]]; then
 fi
 
 # Go to the Apache Beam Python SDK root
-if [[ "*sdks/python" != $PWD ]]; then
+if [[ $PWD != *sdks/python ]]; then
   cd $(pwd | sed 's/sdks\/python.*/sdks\/python/')
 fi
 
diff --git a/sdks/python/scripts/run_pylint.sh b/sdks/python/scripts/run_pylint.sh
index 17f6aa7..2814b4f 100755
--- a/sdks/python/scripts/run_pylint.sh
+++ b/sdks/python/scripts/run_pylint.sh
@@ -30,7 +30,7 @@ if [[ $PWD != *sdks/python* ]]; then
 fi
 
 # Go to the Apache Beam Python SDK root
-if [[ "*sdks/python" != $PWD ]]; then
+if [[ $PWD != *sdks/python ]]; then
   cd $(pwd | sed 's/sdks\/python.*/sdks\/python/')
 fi
 
diff --git a/sdks/python/scripts/run_pylint_2to3.sh b/sdks/python/scripts/run_pylint_2to3.sh
index 2957dd9..0853b09 100755
--- a/sdks/python/scripts/run_pylint_2to3.sh
+++ b/sdks/python/scripts/run_pylint_2to3.sh
@@ -29,7 +29,7 @@ if [[ $PWD != *sdks/python* ]]; then
 fi
 
 # Go to the Apache Beam Python SDK root
-if [[ "*sdks/python" != $PWD ]]; then
+if [[ $PWD != *sdks/python ]]; then
   cd $(pwd | sed 's/sdks\/python.*/sdks\/python/')
 fi
 
diff --git a/sdks/python/scripts/run_tox.sh b/sdks/python/scripts/run_tox.sh
index 24ccb20..e799042 100755
--- a/sdks/python/scripts/run_tox.sh
+++ b/sdks/python/scripts/run_tox.sh
@@ -24,9 +24,10 @@
 
 ###########################################################################
 # Usage check.
-if [[ $# != 1 ]]; then
-  printf "Usage: \n$> ./scripts/run_tox.sh <tox_environment>"
+if [[ $# < 1 || $# > 2 ]]; then
+  printf "Usage: \n$> ./scripts/run_tox.sh <tox_environment> [<sdk_location>]"
   printf "\n\ttox_environment: [required] Tox environment to run the test in.\n"
+  printf "\n\tsdk_location: [optional] SDK tarball artifact location.\n"
   exit 1
 fi
 
@@ -37,11 +38,19 @@ if [[ $PWD != *sdks/python* ]]; then
 fi
 
 # Go to the Apache Beam Python SDK root
-if [[ "*sdks/python" != $PWD ]]; then
+if [[ $PWD != *sdks/python ]]; then
   cd $(pwd | sed 's/sdks\/python.*/sdks\/python/')
 fi
 
-tox -c tox.ini --recreate -e $1
+# Used in tox.ini to isolate toxworkdir of each environment.
+export ENV_NAME=.tox-$1
+
+if [[ ! -z $2 ]]; then
+  tox -c tox.ini --recreate -e $1 --installpkg $2
+else
+  tox -c tox.ini --recreate -e $1
+fi
+
 exit_code=$?
 # Retry once for the specific exit code 245.
 if [[ $exit_code == 245 ]]; then
diff --git a/sdks/python/scripts/run_tox_cleanup.sh b/sdks/python/scripts/run_tox_cleanup.sh
index 87081da..be44095 100755
--- a/sdks/python/scripts/run_tox_cleanup.sh
+++ b/sdks/python/scripts/run_tox_cleanup.sh
@@ -26,7 +26,7 @@ if [[ $PWD != *sdks/python* ]]; then
 fi
 
 # Go to the Apache Beam Python SDK root
-if [[ "*sdks/python" != $PWD ]]; then
+if [[ $PWD != *sdks/python ]]; then
   cd $(pwd | sed 's/sdks\/python.*/sdks\/python/')
 fi
 
diff --git a/sdks/python/test-suites/tox/py35/build.gradle b/sdks/python/test-suites/tox/py35/build.gradle
new file mode 100644
index 0000000..5f86fe9
--- /dev/null
+++ b/sdks/python/test-suites/tox/py35/build.gradle
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+
+/**
+ * Unit tests for Python 3.5
+ */
+
+plugins { id 'org.apache.beam.module' }
+applyPythonNature()
+
+// Required to setup a Python 3 virtualenv.
+project.ext.python3 = true
+
+task lint {}
+check.dependsOn lint
+
+toxTask "lintPy35", "py35-lint"
+lint.dependsOn lintPy35
+
+toxTask "testPython35", "py35"
+test.dependsOn testPython35
+
+toxTask "testPy35Gcp", "py35-gcp"
+test.dependsOn testPy35Gcp
+
+toxTask "testPy35Cython", "py35-cython"
+test.dependsOn testPy35Cython
+// Ensure that testPy35Cython runs exclusively to other tests. This line is not
+// actually required, since gradle doesn't do parallel execution within a
+// project.
+testPy35Cython.mustRunAfter testPython35, testPy35Gcp
+
+task preCommitPy35() {
+    dependsOn "testPython35"
+    dependsOn "testPy35Gcp"
+    dependsOn "testPy35Cython"
+    dependsOn "lint"
+}
diff --git a/sdks/python/test-suites/tox/py36/build.gradle b/sdks/python/test-suites/tox/py36/build.gradle
new file mode 100644
index 0000000..1bdfb78
--- /dev/null
+++ b/sdks/python/test-suites/tox/py36/build.gradle
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/**
+ * Unit tests for Python 3.6
+ */
+
+plugins { id 'org.apache.beam.module' }
+applyPythonNature()
+
+// Required to setup a Python 3 virtualenv.
+project.ext.python3 = true
+
+toxTask "testPython36", "py36"
+test.dependsOn testPython36
+
+task preCommitPy36() {
+  dependsOn "testPython36"
+}
diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini
index cd19a6b..2b765f8 100644
--- a/sdks/python/tox.ini
+++ b/sdks/python/tox.ini
@@ -18,7 +18,7 @@
 [tox]
 # new environments will be excluded by default unless explicitly added to envlist.
 envlist = py27,py35,py36,py27-{gcp,cython,lint,lint3},py35-{gcp,cython,lint},docs
-toxworkdir = {toxinidir}/target/.tox
+toxworkdir = {toxinidir}/target/{env:ENV_NAME:.tox}
 
 [pycodestyle]
 # Disable all errors and warnings except for the ones related to blank lines.
diff --git a/settings.gradle b/settings.gradle
index b8fc633..c453272 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -207,6 +207,10 @@ include "beam-sdks-python-test-suites-dataflow-py3"
 project(":beam-sdks-python-test-suites-dataflow-py3").dir = file("sdks/python/test-suites/dataflow/py3")
 include "beam-sdks-python-test-suites-direct-py3"
 project(":beam-sdks-python-test-suites-direct-py3").dir = file("sdks/python/test-suites/direct/py3")
+include "beam-sdks-python-test-suites-tox-py35"
+project(":beam-sdks-python-test-suites-tox-py35").dir = file("sdks/python/test-suites/tox/py35")
+include "beam-sdks-python-test-suites-tox-py36"
+project(":beam-sdks-python-test-suites-tox-py36").dir = file("sdks/python/test-suites/tox/py36")
 include "beam-sdks-python-load-tests"
 project(":beam-sdks-python-load-tests").dir = file("sdks/python/apache_beam/testing/load_tests")
 include "beam-vendor-grpc-1_13_1"