From 45aaec5df3821ccecef1e023ee7569cff774a431 Mon Sep 17 00:00:00 2001 From: Damon Douglas Date: Tue, 19 Nov 2024 16:11:17 -0800 Subject: [PATCH 1/3] Enable Java SDK Distroless container image variant --- .../trigger_files/beam_PostCommit_Java.json | 5 +- .../beam_PostCommit_Java_DataflowV2.json | 2 +- .../google-cloud-dataflow-java/build.gradle | 81 +++++++++++++++++++ sdks/java/container/Dockerfile-distroless | 36 +++++++++ 4 files changed, 122 insertions(+), 2 deletions(-) create mode 100644 sdks/java/container/Dockerfile-distroless diff --git a/.github/trigger_files/beam_PostCommit_Java.json b/.github/trigger_files/beam_PostCommit_Java.json index 9e26dfeeb6e6..920c8d132e4a 100644 --- a/.github/trigger_files/beam_PostCommit_Java.json +++ b/.github/trigger_files/beam_PostCommit_Java.json @@ -1 +1,4 @@ -{} \ No newline at end of file +{ + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 1 +} \ No newline at end of file diff --git a/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json b/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json index 1efc8e9e4405..3f63c0c9975f 100644 --- a/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json +++ b/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 2 } diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 4906d9cf9cb8..5e2b82f35a2b 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -273,6 +273,87 @@ def createRunnerV2ValidatesRunnerTest = { Map args -> } } +tasks.register('googleCloudPlatformRunnerV2DistrolessIntegrationTest', 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' + exclude '**/BigQueryIOStorageReadTableRowIT.class' + exclude '**/SpannerWriteIT.class' + exclude '**/*KmsKeyIT.class' + exclude '**/FhirIOReadIT.class' + exclude '**/DicomIOReadIT.class' + exclude '**/FhirIOWriteIT.class' + exclude '**/FhirIOLROIT.class' + exclude '**/FhirIOSearchIT.class' + exclude '**/FhirIOPatientEverythingIT.class' + // failing due to pane index not incrementing after Reshuffle: + // https://github.com/apache/beam/issues/28219 + exclude '**/FileLoadsStreamingIT.class' + + maxParallelForks 4 + classpath = configurations.googleCloudPlatformIntegrationTest + testClassesDirs = files(project(":sdks:java:io:google-cloud-platform").sourceSets.test.output.classesDirs) + useJUnit { + filter { + // Only needs to run on direct runner + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithEmptyMutationFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithEmptyRowFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithInvalidTimestampFailures' + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithOversizedQualifierFailures' + } + } +} + +tasks.register('buildAndPushDistrolessContainerImage', Task.class) { + def allowed = ["java17", "java21"] + 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", + '.' + ] + } + exec { + executable 'docker' + args = ['push', imageURL] + } + } +} + // 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 diff --git a/sdks/java/container/Dockerfile-distroless b/sdks/java/container/Dockerfile-distroless new file mode 100644 index 000000000000..752528636ece --- /dev/null +++ b/sdks/java/container/Dockerfile-distroless @@ -0,0 +1,36 @@ +############################################################################### +# 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. +############################################################################### + +# ARG BEAM_BASE is the Beam SDK container image built using sdks/python/container/Dockerfile. +ARG BEAM_BASE + +# ARG DISTROLESS_BASE is the distroless container image URL. For available distroless Java images, +# see https://github.com/GoogleContainerTools/distroless/tree/main?tab=readme-ov-file#what-images-are-available. +# Only Java versions 17 and 21 are supported. +ARG DISTROLESS_BASE +FROM ${BEAM_BASE} AS base +ARG TARGETARCH +LABEL Author="Apache Beam " + +RUN if [ -z "${TARGETARCH}" ]; then echo "fatal: TARGETARCH not set; run as docker buildx build or use --build-arg=TARGETARCH=amd64|arm64" >&2; exit 1; fi + +FROM ${DISTROLESS_BASE}:latest-${TARGETARCH} AS distroless + +COPY --from=base /opt /opt + +ENTRYPOINT ["/opt/apache/beam/boot"] From 86b391af17075a69414c67fe6cb3ff4a7704adbf Mon Sep 17 00:00:00 2001 From: Damon Douglas Date: Tue, 19 Nov 2024 16:20:22 -0800 Subject: [PATCH 2/3] Add LANG environment and /usr/lib/locale --- sdks/java/container/Dockerfile-distroless | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sdks/java/container/Dockerfile-distroless b/sdks/java/container/Dockerfile-distroless index 752528636ece..328c4dc6a7b3 100644 --- a/sdks/java/container/Dockerfile-distroless +++ b/sdks/java/container/Dockerfile-distroless @@ -25,6 +25,8 @@ ARG BEAM_BASE ARG DISTROLESS_BASE FROM ${BEAM_BASE} AS base ARG TARGETARCH +ENV LANG C.UTF-8 + LABEL Author="Apache Beam " RUN if [ -z "${TARGETARCH}" ]; then echo "fatal: TARGETARCH not set; run as docker buildx build or use --build-arg=TARGETARCH=amd64|arm64" >&2; exit 1; fi @@ -33,4 +35,8 @@ FROM ${DISTROLESS_BASE}:latest-${TARGETARCH} AS distroless COPY --from=base /opt /opt +# Along with the LANG environment variable above, prevents internally discovered failing bugs related to Dataflow Flex +# template character encodings. +COPY --from=base /usr/lib/locale /usr/lib/locale + ENTRYPOINT ["/opt/apache/beam/boot"] From 694537c783fe2e6f47cba825b5dc0625b72b54df Mon Sep 17 00:00:00 2001 From: Damon Douglas Date: Thu, 21 Nov 2024 18:18:20 -0800 Subject: [PATCH 3/3] Use examples tests instead --- .../google-cloud-dataflow-java/build.gradle | 30 ++++--------------- 1 file changed, 6 insertions(+), 24 deletions(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 5e2b82f35a2b..3a337684bf18 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -273,7 +273,7 @@ def createRunnerV2ValidatesRunnerTest = { Map args -> } } -tasks.register('googleCloudPlatformRunnerV2DistrolessIntegrationTest', Test.class) { +tasks.register('examplesJavaRunnerV2IntegrationTestDistroless', Test.class) { group = "verification" dependsOn 'buildAndPushDistrolessContainerImage' def javaVer = project.findProperty('testJavaVersion') @@ -292,34 +292,16 @@ tasks.register('googleCloudPlatformRunnerV2DistrolessIntegrationTest', Test.clas systemProperty "beamTestPipelineOptions", JsonOutput.toJson(pipelineOptions) include '**/*IT.class' - exclude '**/BigQueryIOStorageReadTableRowIT.class' - exclude '**/SpannerWriteIT.class' - exclude '**/*KmsKeyIT.class' - exclude '**/FhirIOReadIT.class' - exclude '**/DicomIOReadIT.class' - exclude '**/FhirIOWriteIT.class' - exclude '**/FhirIOLROIT.class' - exclude '**/FhirIOSearchIT.class' - exclude '**/FhirIOPatientEverythingIT.class' - // failing due to pane index not incrementing after Reshuffle: - // https://github.com/apache/beam/issues/28219 - exclude '**/FileLoadsStreamingIT.class' maxParallelForks 4 - classpath = configurations.googleCloudPlatformIntegrationTest - testClassesDirs = files(project(":sdks:java:io:google-cloud-platform").sourceSets.test.output.classesDirs) - useJUnit { - filter { - // Only needs to run on direct runner - excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithEmptyMutationFailures' - excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithEmptyRowFailures' - excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithInvalidTimestampFailures' - excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteIT.testE2EBigtableWriteWithOversizedQualifierFailures' - } - } + 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"] doLast { def javaVer = project.findProperty('testJavaVersion')