Skip to content

Commit

Permalink
Refactor distroless variants
Browse files Browse the repository at this point in the history
  • Loading branch information
damondouglas committed Jan 4, 2025
1 parent 3c78746 commit e3e3782
Show file tree
Hide file tree
Showing 22 changed files with 468 additions and 94 deletions.

This file was deleted.

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,11 @@ jobs:
with:
gradle-command: :runners:google-cloud-dataflow-java:examplesJavaRunnerV2IntegrationTest
max-workers: 12
- name: run PostCommit Java Distroless Examples Dataflow V2 script
uses: ./.github/actions/gradle-command-self-hosted-action
with:
gradle-command: :runners:google-cloud-dataflow-java:examplesJavaDistrolessRunnerV2PreCommit
max-workers: 12
- name: Archive JUnit Test Results
uses: actions/upload-artifact@v4
if: ${{ !success() }}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,14 @@ jobs:
gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:validatesContainer
arguments: |
-PpythonVersion=${{ matrix.python_version }} \
- name: Run validatesDistrolessContainer script
env:
USER: github-actions
uses: ./.github/actions/gradle-command-self-hosted-action
with:
gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:validatesDistrolessContainer
arguments: |
-PpythonVersion=${{ matrix.python_version }}
- name: Archive Python Test Results
uses: actions/upload-artifact@v4
if: failure()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,15 @@ jobs:
arguments: |
-PtestRCDependencies=true \
-PpythonVersion=${{ matrix.python_version }} \
- name: Run validatesDistrolessContainer script
env:
USER: github-actions
uses: ./.github/actions/gradle-command-self-hosted-action
with:
gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:validatesDistrolessContainer
arguments: |
-PtestRCDependencies=true \
-PpythonVersion=${{ matrix.python_version }}
- name: Archive Python Test Results
uses: actions/upload-artifact@v4
if: failure()
Expand Down
6 changes: 6 additions & 0 deletions .github/workflows/beam_Publish_Beam_SDK_Snapshots.yml
Original file line number Diff line number Diff line change
Expand Up @@ -66,10 +66,16 @@ jobs:
- "java:container:java11"
- "java:container:java17"
- "java:container:java21"
- "java:container:distroless:java17"
- "java:container:distroless:java21"
- "python:container:py39"
- "python:container:py310"
- "python:container:py311"
- "python:container:py312"
- "python:container:distroless:py39"
- "python:container:distroless:py310"
- "python:container:distroless:py311"
- "python:container:distroless:py312"
- "java:expansion-service:container"
steps:
- uses: actions/checkout@v4
Expand Down
94 changes: 32 additions & 62 deletions runners/google-cloud-dataflow-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -275,81 +275,38 @@ def createRunnerV2ValidatesRunnerTest = { Map args ->
}
}

tasks.register('examplesJavaRunnerV2IntegrationTestDistroless', Test.class) {
group = "verification"
dependsOn 'buildAndPushDistrolessContainerImage'
def javaVer = project.findProperty('testJavaVersion')
def repository = "us.gcr.io/apache-beam-testing/${System.getenv('USER')}"
def tag = project.findProperty('dockerTag')
def imageURL = "${repository}/beam_${javaVer}_sdk_distroless:${tag}"
def pipelineOptions = [
"--runner=TestDataflowRunner",
"--project=${gcpProject}",
"--region=${gcpRegion}",
"--tempRoot=${dataflowValidatesTempRoot}",
"--sdkContainerImage=${imageURL}",
"--experiments=use_unified_worker,use_runner_v2",
"--firestoreDb=${firestoreDb}",
]
systemProperty "beamTestPipelineOptions", JsonOutput.toJson(pipelineOptions)

include '**/*IT.class'

maxParallelForks 4
classpath = configurations.examplesJavaIntegrationTest
testClassesDirs = files(project(":examples:java").sourceSets.test.output.classesDirs)
useJUnit { }
}

tasks.register('buildAndPushDistrolessContainerImage', Task.class) {
// Only Java 17 and 21 are supported.
// See https://github.com/GoogleContainerTools/distroless/tree/main/java#image-contents.
def allowed = ["java17", "java21"]
// Push docker images to a container registry for use within tests.
// NB: Tasks which consume docker images from the registry should depend on this
// task directly ('dependsOn buildAndPushDockerJavaContainer'). This ensures the correct
// task ordering such that the registry doesn't get cleaned up prior to task completion.
def buildAndPushDockerJavaContainer = tasks.register("buildAndPushDockerJavaContainer") {
def javaVer = getSupportedJavaVersion()
if(project.hasProperty('testJavaVersion')) {
javaVer = "java${project.getProperty('testJavaVersion')}"
}
dependsOn ":sdks:java:container:${javaVer}:docker"
def defaultDockerImageName = containerImageName(
name: "${project.docker_image_default_repo_prefix}${javaVer}_sdk",
root: "apache",
tag: project.sdk_version)
doLast {
def javaVer = project.findProperty('testJavaVersion')
if (!allowed.contains(javaVer)) {
throw new GradleException("testJavaVersion must be one of ${allowed}, got: ${javaVer}")
}
if (!project.hasProperty('dockerTag')) {
throw new GradleException("dockerTag is missing but required")
}
def repository = "us.gcr.io/apache-beam-testing/${System.getenv('USER')}"
def tag = project.findProperty('dockerTag')
def imageURL = "${repository}/beam_${javaVer}_sdk_distroless:${tag}"
exec {
executable 'docker'
workingDir rootDir
args = [
'buildx',
'build',
'-t',
imageURL,
'-f',
'sdks/java/container/Dockerfile-distroless',
"--build-arg=BEAM_BASE=gcr.io/apache-beam-testing/beam-sdk/beam_${javaVer}_sdk",
"--build-arg=DISTROLESS_BASE=gcr.io/distroless/${javaVer}-debian12",
'.'
]
commandLine "docker", "tag", "${defaultDockerImageName}", "${dockerJavaImageName}"
}
exec {
executable 'docker'
args = ['push', imageURL]
commandLine "gcloud", "docker", "--", "push", "${dockerJavaImageName}"
}
}
}

// Push docker images to a container registry for use within tests.
// NB: Tasks which consume docker images from the registry should depend on this
// task directly ('dependsOn buildAndPushDockerJavaContainer'). This ensures the correct
// task ordering such that the registry doesn't get cleaned up prior to task completion.
def buildAndPushDockerJavaContainer = tasks.register("buildAndPushDockerJavaContainer") {
tasks.register("buildAndPushDockerJavaDistrolessContainer") {
def javaVer = getSupportedJavaVersion()
if(project.hasProperty('testJavaVersion')) {
javaVer = "java${project.getProperty('testJavaVersion')}"
}
dependsOn ":sdks:java:container:${javaVer}:docker"
dependsOn ":sdks:java:container:distroless:${javaVer}:docker"
def defaultDockerImageName = containerImageName(
name: "${project.docker_image_default_repo_prefix}${javaVer}_sdk",
name: "${project.docker_image_default_repo_prefix}${javaVer}_sdk_distroless",
root: "apache",
tag: project.sdk_version)
doLast {
Expand Down Expand Up @@ -719,6 +676,19 @@ task examplesJavaRunnerV2PreCommit(type: Test) {
useJUnit { }
}

task examplesJavaDistrolessRunnerV2PreCommit(type: Test) {
group = "Verification"
dependsOn buildAndPushDockerJavaDistrolessContainer
systemProperty "beamTestPipelineOptions", JsonOutput.toJson(runnerV2PipelineOptions)
include '**/WordCountIT.class'
include '**/WindowedWordCountIT.class'

maxParallelForks 4
classpath = configurations.examplesJavaIntegrationTest
testClassesDirs = files(project(":examples:java").sourceSets.test.output.classesDirs)
useJUnit { }
}

task examplesJavaRunnerV2IntegrationTest(type: Test) {
group = "Verification"
dependsOn buildAndPushDockerJavaContainer
Expand Down
File renamed without changes.
32 changes: 32 additions & 0 deletions sdks/java/container/distroless/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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'
}

description = "Apache Beam :: SDKs :: Java :: Container :: Distroless"

configurations {
dockerDependency
}

task pushAll {
dependsOn ":sdks:java:container:distroless:java17:docker"
dependsOn ":sdks:java:container:distroless:java21:docker"
}
71 changes: 71 additions & 0 deletions sdks/java/container/distroless/common.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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.
*/

apply plugin: 'org.apache.beam.module'
apply plugin: 'base'
applyDockerNature()

if (!project.hasProperty('imageJavaVersion')) {
throw new GradleException('imageJavaVersion project property must be set')
}

def imageJavaVersion = project.findProperty('imageJavaVersion')

description = "Apache Beam :: SDKs :: Java :: Container :: Java ${imageJavaVersion} Container"

configurations {
dockerDependency
}

dependencies {
dockerDependency project(path: ":sdks:java:container:distroless", configuration: "dockerDependency")
}

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

def base_name = "${project.docker_image_default_repo_prefix}java${imageJavaVersion}_sdk"
def root = project.rootProject.hasProperty(["docker-repository-root"]) ?
project.rootProject["docker-repository-root"] :
project.docker_image_default_repo_root
def tag = project.rootProject.hasProperty(["docker-tag"]) ?
project.rootProject["docker-tag"] : project.sdk_version

def base = containerImageName(
name: base_name,
root: root,
tag: tag)

docker {
name containerImageName(
name: "${base_name}_distroless",
root: root,
tag: tag)
tags containerImageTags()
dockerfile project.file("../Dockerfile")
files "./build/"
buildArgs([
'BEAM_BASE': base,
'DISTROLESS_BASE': "gcr.io/distroless/java${imageJavaVersion}-debian12"
])
buildx project.useBuildx()
platform(*project.containerPlatforms())
load project.useBuildx() && !pushContainers
push pushContainers
}

dockerPrepare.dependsOn ":sdks:java:container:java${imageJavaVersion}:docker"
24 changes: 24 additions & 0 deletions sdks/java/container/distroless/java17/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
/*
* 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.
*/

project.ext {
imageJavaVersion = '17'
}

// Load the main build script which contains all build logic.
apply from: "../common.gradle"
24 changes: 24 additions & 0 deletions sdks/java/container/distroless/java21/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
/*
* 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.
*/

project.ext {
imageJavaVersion = '21'
}

// Load the main build script which contains all build logic.
apply from: "../common.gradle"
File renamed without changes.
Loading

0 comments on commit e3e3782

Please sign in to comment.