Skip to content

Commit

Permalink
Add flags to build multiarch Python SDK containers and test (#27723)
Browse files Browse the repository at this point in the history
  • Loading branch information
celeste-zeng authored Aug 1, 2023
1 parent 52352d6 commit d4f09da
Show file tree
Hide file tree
Showing 7 changed files with 183 additions and 16 deletions.
1 change: 1 addition & 0 deletions .github/workflows/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -137,3 +137,4 @@ Please note that jobs with matrix need to have matrix element in the comment. Ex
| [ PreCommit Website ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) | N/A |`Run Website PreCommit`| [![.github/workflows/beam_PreCommit_Website.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website.yml) |
| [ PreCommit Website Stage GCS ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) | N/A |`Run Website_Stage_GCS PreCommit`| [![PreCommit Website Stage GCS](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Website_Stage_GCS.yml) |
| [ PreCommit Whitespace ](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) | N/A |`Run Whitespace PreCommit`| [![.github/workflows/beam_PreCommit_Whitespace.yml](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PreCommit_Whitespace.yml) |
| [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] | `Run Python Validates Container Dataflow ARM (matrix_element)`| [![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) |
125 changes: 125 additions & 0 deletions .github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
# 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.

name: Python ValidatesContainer Dataflow ARM

on:
issue_comment:
types: [created]
push:
tags: ['v*']
branches: ['master', 'release-*']
paths: ["sdks/python/**",".github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml"]
schedule:
- cron: '0 */6 * * *'
workflow_dispatch:

#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event
permissions:
actions: write
pull-requests: read
checks: read
contents: read
deployments: read
id-token: none
issues: read
discussions: read
packages: read
pages: read
repository-projects: read
security-events: read
statuses: read

# This allows a subsequently queued workflow run to interrupt previous runs
concurrency:
group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.ref }}'
cancel-in-progress: true

jobs:
beam_Python_ValidatesContainer_Dataflow_ARM:
name: beam_Python_ValidatesContainer_Dataflow_ARM
strategy:
fail-fast: false
matrix:
python_version: ['3.8','3.9','3.10','3.11']
if: |
github.event_name == 'push' ||
startsWith(github.event.comment.body, 'Run Python ValidatesContainer Dataflow ARM') ||
github.event_name == 'schedule'
runs-on: [self-hosted, ubuntu-20.04, main]
env:
MULTIARCH_TAG: $(date +'%Y%m%d-%H%M%S%N')
steps:
- name: Check out repository code
uses: actions/checkout@v3
with:
ref: ${{ github.event.pull_request.head.sha }}
- name: Set comment body with matrix
id: set_comment_body
run: |
echo "comment_body=Run Python ValidatesContainer Dataflow ARM (${{ matrix.python_version }})" >> $GITHUB_OUTPUT
- name: Rerun on comment
if: github.event.comment.body == steps.set_comment_body.outputs.comment_body
uses: ./.github/actions/rerun-job-action
with:
pull_request_url: ${{ github.event.issue.pull_request.url }}
github_repository: ${{ github.repository }}
github_token: ${{ secrets.GITHUB_TOKEN }}
github_job: "${{ github.job }} (${{ matrix.python_version }})"
github_current_run_id: ${{ github.run_id }}
- name: Install Python
uses: actions/setup-python@v4
with:
python-version: ${{ matrix.python_version }}
- name: Setup Gradle
uses: gradle/gradle-build-action@v2
with:
cache-read-only: false
- name: Set up Docker Buildx
uses: docker/setup-buildx-action@v2
- name: Authenticate on GCP
uses: google-github-actions/setup-gcloud@v0
with:
service_account_email: ${{ secrets.GCP_SA_EMAIL }}
service_account_key: ${{ secrets.GCP_SA_KEY }}
project_id: ${{ secrets.GCP_PROJECT_ID }}
export_default_credentials: true
- name: GCloud Docker credential helper
run: |
gcloud auth configure-docker us.gcr.io
- name: Set PY_VER_CLEAN
id: set_py_ver_clean
run: |
PY_VER=${{ matrix.python_version }}
PY_VER_CLEAN=${PY_VER//.}
echo "py_ver_clean=$PY_VER_CLEAN" >> $GITHUB_OUTPUT
- name: run Python validatesContainerARM script
id: run_script
run: |
gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:validatesContainerARM
arguments: |
-PpythonVersion=${{ matrix.python_version }} \
-Pcontainer-architecture-list=arm64,amd64 \
-Pdocker-repository=us.gcr.io/apache-beam-testing/github-actions \
-Pdocker-tag=${{ env.MULTIARCH_TAG }} \
-Ppush-containers
env:
MULTIARCH_TAG: ${{ env.MULTIARCH_TAG }}
USER: github-actions
- name: Archive code coverage results
uses: actions/upload-artifact@v3
with:
name: python-code-coverage-report
path: "**/pytest*.xml"
4 changes: 4 additions & 0 deletions sdks/python/apache_beam/examples/wordcount_it_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,10 @@ def test_wordcount_it_with_prebuilt_sdk_container_cloud_build(self):
def test_wordcount_it_with_use_sibling_sdk_workers(self):
self._run_wordcount_it(wordcount.run, experiment='use_sibling_sdk_workers')

@pytest.mark.it_dataflow_arm
def test_wordcount_fnapi_it_arm(self):
self._run_wordcount_it(wordcount.run, machine_type='t2a-standard-1')

def _run_wordcount_it(self, run_wordcount, **opts):
test_pipeline = TestPipeline(is_integration_test=True)
extra_opts = {}
Expand Down
7 changes: 6 additions & 1 deletion sdks/python/container/common.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,9 @@ def copyLauncherDependencies = tasks.register("copyLauncherDependencies", Copy)
}
}

def useBuildx = project.containerPlatforms() != [project.nativeArchitecture()]
def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers")

docker {
name containerImageName(
name: project.docker_image_default_repo_prefix + "python${project.ext.pythonVersion}_sdk",
Expand All @@ -80,8 +83,10 @@ docker {
buildArgs(['py_version': "${project.ext.pythonVersion}",
'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) ||
project.rootProject.hasProperty(["isRelease"])])
buildx project.containerPlatforms() != [project.nativeArchitecture()]
buildx useBuildx
platform(*project.containerPlatforms())
load useBuildx && !pushContainers
push pushContainers
}

dockerPrepare.dependsOn copyLauncherDependencies
Expand Down
45 changes: 30 additions & 15 deletions sdks/python/container/run_validatescontainer.sh
Original file line number Diff line number Diff line change
Expand Up @@ -24,17 +24,22 @@
# REGION -> Region name to use for Dataflow
#
# Execute from the root of the repository:
# test Python3.8 container:
# test Python3.8 x86 container:
# ./gradlew :sdks:python:test-suites:dataflow:py38:validatesContainer
# or test all supported python versions together:
# or test all supported python versions x86 containers together:
# ./gradlew :sdks:python:test-suites:dataflow:validatesContainer
#
# Note: ARM test suites only run on github actions. For example, to test Python3.8 ARM containers,
# commenting `Run Python ValidatesContainer Dataflow ARM (3.8)` will trigger the test.

echo "This script must be executed in the root of beam project. Please set GCS_LOCATION, PROJECT and REGION as desired."

if [[ $# != 2 ]]; then
printf "Usage: \n$> ./sdks/python/container/run_validatescontainer.sh <python_version> <sdk_location>"
if [[ $# < 2 ]]; then
printf "Usage: \n$> ./sdks/python/container/run_validatescontainer.sh <python_version> <sdk_location> <cpu_architecture>"
printf "\n\tpython_version: [required] Python version used for container build and run tests."
printf " Sample value: 3.9"
printf "\n\tcpu_architecture: [optional] CPU architecture used for container build and run tests."
printf " Sample value: ARM or x86"
exit 1
fi

Expand All @@ -50,9 +55,11 @@ REGION=${REGION:-us-central1}
IMAGE_PREFIX="$(grep 'docker_image_default_repo_prefix' gradle.properties | cut -d'=' -f2)"
SDK_VERSION="$(grep 'sdk_version' gradle.properties | cut -d'=' -f2)"
PY_VERSION=$1
ARCH=${3:-"x86"}
IMAGE_NAME="${IMAGE_PREFIX}python${PY_VERSION}_sdk"
CONTAINER_PROJECT="sdks:python:container:py${PY_VERSION//.}" # Note: we substitute away the dot in the version.
PY_INTERPRETER="python${PY_VERSION}"
TEST_SUITE_TAG="it_validatescontainer"

XUNIT_FILE="pytest-$IMAGE_NAME.xml"

Expand All @@ -65,23 +72,32 @@ command -v gcloud
docker -v
gcloud -v

# Verify docker image has been built.
docker images | grep "apache/$IMAGE_NAME" | grep "$SDK_VERSION"

TAG=$(date +%Y%m%d-%H%M%S%N)
CONTAINER=us.gcr.io/$PROJECT/$USER/$IMAGE_NAME
PREBUILD_SDK_CONTAINER_REGISTRY_PATH=us.gcr.io/$PROJECT/$USER/prebuild_python${PY_VERSION//.}_sdk
echo "Using container $CONTAINER"

# Tag the docker container.
docker tag "apache/$IMAGE_NAME:$SDK_VERSION" "$CONTAINER:$TAG"
if [[ "$ARCH" == "x86" ]]; then
# Verify docker image has been built.
docker images | grep "apache/$IMAGE_NAME" | grep "$SDK_VERSION"

# Tag the docker container.
docker tag "apache/$IMAGE_NAME:$SDK_VERSION" "$CONTAINER:$TAG"

# Push the container.
gcloud docker -- push $CONTAINER:$TAG
# Push the container
gcloud docker -- push $CONTAINER:$TAG
else
# Note: ARM test suites only run on github actions, where multi-arch Python SDK containers are already pushed during build.
# Reset the test suite tag to run ARM pipelines.
TEST_SUITE_TAG="it_dataflow_arm"

# Reset the multi-arch Python SDK container image tag.
TAG=$MULTIARCH_TAG
fi

function cleanup_container {
# Delete the container locally and remotely
docker rmi $CONTAINER:$TAG || echo "Failed to remove container image"
docker rmi $CONTAINER:$TAG || echo "Built container image was not removed. Possibly, it was not not saved locally."
for image in $(docker images --format '{{.Repository}}:{{.Tag}}' | grep $PREBUILD_SDK_CONTAINER_REGISTRY_PATH)
do docker rmi $image || echo "Failed to remove prebuilt sdk container image"
done
Expand All @@ -99,10 +115,9 @@ echo ">>> Successfully built and push container $CONTAINER"
cd sdks/python
SDK_LOCATION=$2

# Run ValidatesRunner tests on Google Cloud Dataflow service
echo ">>> RUNNING DATAFLOW RUNNER VALIDATESCONTAINER TEST"
pytest -o junit_suite_name=$IMAGE_NAME \
-m="it_validatescontainer" \
-m=$TEST_SUITE_TAG \
--show-capture=no \
--numprocesses=1 \
--timeout=1800 \
Expand All @@ -121,4 +136,4 @@ pytest -o junit_suite_name=$IMAGE_NAME \
--num_workers=1 \
--docker_registry_push_url=$PREBUILD_SDK_CONTAINER_REGISTRY_PATH"

echo ">>> SUCCESS DATAFLOW RUNNER VALIDATESCONTAINER TEST"
echo ">>> SUCCESS DATAFLOW RUNNER VALIDATESCONTAINER TEST"
1 change: 1 addition & 0 deletions sdks/python/pytest.ini
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ markers =
no_sickbay_streaming: run without sickbay-streaming
no_sickbay_batch: run without sickbay-batch
it_validatescontainer: collect for ValidatesContainer integration test runs
it_dataflow_arm: collect for DataflowArm integration test runs
examples_postcommit: collect for post-commit test examples runs
sickbay_direct: run without sickbay-direct
sickbay_spark: run without sickbay-spark
Expand Down
16 changes: 16 additions & 0 deletions sdks/python/test-suites/dataflow/common.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -348,6 +348,22 @@ task validatesContainer() {
}
}

task validatesContainerARM() {
def pyversion = "${project.ext.pythonVersion.replace('.', '')}"
dependsOn 'initializeForDataflowJob'
dependsOn ":sdks:python:container:py${pyversion}:docker"
def runScriptsPath = "${rootDir}/sdks/python/container/run_validatescontainer.sh"
doLast {
exec {
executable 'sh'
args '-c', ". ${envdir}/bin/activate && cd ${rootDir} && ${runScriptsPath} " +
"${project.ext.pythonVersion} " +
"${project.ext.sdkLocation}" +
"ARM"
}
}
}

def tensorRTTests = tasks.create("tensorRTtests") {
dependsOn 'installGcpTest'
dependsOn ':sdks:python:sdist'
Expand Down

0 comments on commit d4f09da

Please sign in to comment.