diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts index 0ca748e3eb04..968829caeb8b 100644 --- a/buildSrc/build.gradle.kts +++ b/buildSrc/build.gradle.kts @@ -44,20 +44,19 @@ dependencies { implementation("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") implementation("com.github.spotbugs.snom:spotbugs-gradle-plugin:5.0.14") - runtimeOnly("com.google.protobuf:protobuf-gradle-plugin:0.8.13") // Enable proto code generation - runtimeOnly("com.github.davidmc24.gradle-avro-plugin:gradle-avro-plugin:0.16.0") // Enable Avro code generation - runtimeOnly("com.diffplug.spotless:spotless-plugin-gradle:5.6.1") // Enable a code formatting plugin - runtimeOnly("com.palantir.gradle.docker:gradle-docker:0.34.0") // Enable building Docker containers - runtimeOnly("gradle.plugin.com.dorongold.plugins:task-tree:1.5") // Adds a 'taskTree' task to print task dependency tree - runtimeOnly("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") // Enable shading Java dependencies + runtimeOnly("com.google.protobuf:protobuf-gradle-plugin:0.8.13") // Enable proto code generation + runtimeOnly("com.github.davidmc24.gradle-avro-plugin:gradle-avro-plugin:0.16.0") // Enable Avro code generation + runtimeOnly("com.diffplug.spotless:spotless-plugin-gradle:5.6.1") // Enable a code formatting plugin + runtimeOnly("gradle.plugin.com.dorongold.plugins:task-tree:1.5") // Adds a 'taskTree' task to print task dependency tree + runtimeOnly("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") // Enable shading Java dependencies runtimeOnly("net.linguica.gradle:maven-settings-plugin:0.5") runtimeOnly("gradle.plugin.io.pry.gradle.offline_dependencies:gradle-offline-dependencies-plugin:0.5.0") // Enable creating an offline repository - runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:1.2.1") // Enable errorprone Java static analysis + runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:3.1.0") // Enable errorprone Java static analysis runtimeOnly("org.ajoberstar.grgit:grgit-gradle:4.1.1") // Enable website git publish to asf-site branch - runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks + runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks runtimeOnly("ca.cutterslade.gradle:gradle-dependency-analyze:1.8.3") // Enable dep analysis runtimeOnly("gradle.plugin.net.ossindex:ossindex-gradle-plugin:0.4.11") // Enable dep vulnerability analysis - runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin + runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin } // Because buildSrc is built and tested automatically _before_ gradle diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy new file mode 100644 index 000000000000..442b35439cae --- /dev/null +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy @@ -0,0 +1,325 @@ +/* + * 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. + */ +package org.apache.beam.gradle + +import java.util.regex.Pattern +import org.gradle.api.GradleException +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.Task +import org.gradle.api.file.CopySpec +import org.gradle.api.logging.LogLevel +import org.gradle.api.logging.Logger +import org.gradle.api.logging.Logging +import org.gradle.api.tasks.Copy +import org.gradle.api.tasks.Delete +import org.gradle.api.tasks.Exec + +/** + * A gradle plug-in interacting with docker. Originally replicated from + * com.palantir.docker plugin. + */ +class BeamDockerPlugin implements Plugin { + private static final Logger logger = Logging.getLogger(BeamDockerPlugin.class) + private static final Pattern LABEL_KEY_PATTERN = Pattern.compile('^[a-z0-9.-]*$') + + static class DockerExtension { + Project project + + private static final String DEFAULT_DOCKERFILE_PATH = 'Dockerfile' + String name = null + File dockerfile = null + String dockerComposeTemplate = 'docker-compose.yml.template' + String dockerComposeFile = 'docker-compose.yml' + Set dependencies = [] as Set + Set tags = [] as Set + Map namedTags = [:] + Map labels = [:] + Map buildArgs = [:] + boolean pull = false + boolean noCache = false + String network = null + boolean buildx = false + Set platform = [] as Set + boolean load = false + boolean push = false + String builder = null + + File resolvedDockerfile = null + File resolvedDockerComposeTemplate = null + File resolvedDockerComposeFile = null + + // The CopySpec defining the Docker Build Context files + final CopySpec copySpec + + DockerExtension(Project project) { + this.project = project + this.copySpec = project.copySpec() + } + + void resolvePathsAndValidate() { + if (dockerfile != null) { + resolvedDockerfile = dockerfile + } else { + resolvedDockerfile = project.file(DEFAULT_DOCKERFILE_PATH) + } + resolvedDockerComposeFile = project.file(dockerComposeFile) + resolvedDockerComposeTemplate = project.file(dockerComposeTemplate) + } + + void dependsOn(Task... args) { + this.dependencies = args as Set + } + + Set getDependencies() { + return dependencies + } + + void files(Object... files) { + copySpec.from(files) + } + + void tags(String... args) { + this.tags = args as Set + } + + Set getTags() { + return this.tags + project.getVersion().toString() + } + + Set getPlatform() { + return platform + } + + void platform(String... args) { + this.platform = args as Set + } + } + + @Override + void apply(Project project) { + DockerExtension ext = project.extensions.create('docker', DockerExtension, project) + + Delete clean = project.tasks.create('dockerClean', Delete, { + group = 'Docker' + description = 'Cleans Docker build directory.' + }) + + Copy prepare = project.tasks.create('dockerPrepare', Copy, { + group = 'Docker' + description = 'Prepares Docker build directory.' + dependsOn clean + }) + + Exec exec = project.tasks.create('docker', Exec, { + group = 'Docker' + description = 'Builds Docker image.' + dependsOn prepare + }) + + Task tag = project.tasks.create('dockerTag', { + group = 'Docker' + description = 'Applies all tags to the Docker image.' + dependsOn exec + }) + + Task pushAllTags = project.tasks.create('dockerTagsPush', { + group = 'Docker' + description = 'Pushes all tagged Docker images to configured Docker Hub.' + }) + + project.tasks.create('dockerPush', { + group = 'Docker' + description = 'Pushes named Docker image to configured Docker Hub.' + dependsOn pushAllTags + }) + + project.afterEvaluate { + ext.resolvePathsAndValidate() + String dockerDir = "${project.buildDir}/docker" + clean.delete dockerDir + + prepare.with { + with ext.copySpec + from(ext.resolvedDockerfile) { + rename { fileName -> + fileName.replace(ext.resolvedDockerfile.getName(), 'Dockerfile') + } + } + into dockerDir + } + + exec.with { + workingDir dockerDir + commandLine buildCommandLine(ext) + dependsOn ext.getDependencies() + logging.captureStandardOutput LogLevel.INFO + logging.captureStandardError LogLevel.ERROR + } + + Map tags = ext.namedTags.collectEntries { taskName, tagName -> + [ + generateTagTaskName(taskName), + [ + tagName: tagName, + tagTask: { + -> tagName } + ] + ] + } + + if (!ext.tags.isEmpty()) { + ext.tags.each { unresolvedTagName -> + String taskName = generateTagTaskName(unresolvedTagName) + + if (tags.containsKey(taskName)) { + throw new IllegalArgumentException("Task name '${taskName}' is existed.") + } + + tags[taskName] = [ + tagName: unresolvedTagName, + tagTask: { + -> computeName(ext.name, unresolvedTagName) } + ] + } + } + + tags.each { taskName, tagConfig -> + Exec tagSubTask = project.tasks.create('dockerTag' + taskName, Exec, { + group = 'Docker' + description = "Tags Docker image with tag '${tagConfig.tagName}'" + workingDir dockerDir + commandLine 'docker', 'tag', "${-> ext.name}", "${-> tagConfig.tagTask()}" + dependsOn exec + }) + tag.dependsOn tagSubTask + + Exec pushSubTask = project.tasks.create('dockerPush' + taskName, Exec, { + group = 'Docker' + description = "Pushes the Docker image with tag '${tagConfig.tagName}' to configured Docker Hub" + workingDir dockerDir + commandLine 'docker', 'push', "${-> tagConfig.tagTask()}" + dependsOn tagSubTask + }) + pushAllTags.dependsOn pushSubTask + } + } + } + + private List buildCommandLine(DockerExtension ext) { + List buildCommandLine = ['docker'] + if (ext.buildx) { + buildCommandLine.addAll(['buildx', 'build']) + if (!ext.platform.isEmpty()) { + buildCommandLine.addAll('--platform', String.join(',', ext.platform)) + } + if (ext.load) { + buildCommandLine.add '--load' + } + if (ext.push) { + buildCommandLine.add '--push' + if (ext.load) { + throw new Exception("cannot combine 'push' and 'load' options") + } + } + if (ext.builder != null) { + buildCommandLine.addAll('--builder', ext.builder) + } + } else { + buildCommandLine.add 'build' + } + if (ext.noCache) { + buildCommandLine.add '--no-cache' + } + if (ext.getNetwork() != null) { + buildCommandLine.addAll('--network', ext.network) + } + if (!ext.buildArgs.isEmpty()) { + for (Map.Entry buildArg : ext.buildArgs.entrySet()) { + buildCommandLine.addAll('--build-arg', "${buildArg.getKey()}=${buildArg.getValue()}" as String) + } + } + if (!ext.labels.isEmpty()) { + for (Map.Entry label : ext.labels.entrySet()) { + if (!label.getKey().matches(LABEL_KEY_PATTERN)) { + throw new GradleException(String.format("Docker label '%s' contains illegal characters. " + + "Label keys must only contain lowercase alphanumberic, `.`, or `-` characters (must match %s).", + label.getKey(), LABEL_KEY_PATTERN.pattern())) + } + buildCommandLine.addAll('--label', "${label.getKey()}=${label.getValue()}" as String) + } + } + if (ext.pull) { + buildCommandLine.add '--pull' + } + buildCommandLine.addAll(['-t', "${-> ext.name}", '.']) + logger.debug("${buildCommandLine}" as String) + return buildCommandLine + } + + private static String computeName(String name, String tag) { + int firstAt = tag.indexOf("@") + + String tagValue + if (firstAt > 0) { + tagValue = tag.substring(firstAt + 1, tag.length()) + } else { + tagValue = tag + } + + if (tagValue.contains(':') || tagValue.contains('/')) { + // tag with ':' or '/' -> force use the tag value + return tagValue + } else { + // tag without ':' and '/' -> replace the tag part of original name + int lastColon = name.lastIndexOf(':') + int lastSlash = name.lastIndexOf('/') + + int endIndex; + + // image_name -> this should remain + // host:port/image_name -> this should remain. + // host:port/image_name:v1 -> v1 should be replaced + if (lastColon > lastSlash) endIndex = lastColon + else endIndex = name.length() + + return name.substring(0, endIndex) + ":" + tagValue + } + } + + private static String generateTagTaskName(String name) { + String tagTaskName = name + int firstAt = name.indexOf("@") + + if (firstAt > 0) { + // Get substring of task name + tagTaskName = name.substring(0, firstAt) + } else if (firstAt == 0) { + // Task name must not be empty + throw new GradleException("Task name of docker tag '${name}' must not be empty.") + } else if (name.contains(':') || name.contains('/')) { + // Tags which with repo or name must have a task name + throw new GradleException("Docker tag '${name}' must have a task name.") + } + + StringBuffer sb = new StringBuffer(tagTaskName) + // Uppercase the first letter of task name + sb.replace(0, 1, tagTaskName.substring(0, 1).toUpperCase()); + return sb.toString() + } +} diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy new file mode 100644 index 000000000000..5297c7018139 --- /dev/null +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy @@ -0,0 +1,143 @@ +/* + * 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. + */ +package org.apache.beam.gradle + +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.tasks.Exec + +/** + * A gradle plug-in handling 'docker run' command. Originally replicated from + * com.palantir.docker-run plugin. + */ +class BeamDockerRunPlugin implements Plugin { + + /** A class defining the configurations of dockerRun task. */ + static class DockerRunExtension { + String name + String image + Set ports = [] as Set + Map env = [:] + List arguments = [] + Map volumes = [:] + boolean daemonize = true + boolean clean = false + + public String getName() { + return name + } + + public void setName(String name) { + this.name = name + } + } + + @Override + void apply(Project project) { + DockerRunExtension ext = project.extensions.create('dockerRun', DockerRunExtension) + + Exec dockerRunStatus = project.tasks.create('dockerRunStatus', Exec, { + group = 'Docker Run' + description = 'Checks the run status of the container' + }) + + Exec dockerRun = project.tasks.create('dockerRun', Exec, { + group = 'Docker Run' + description = 'Runs the specified container with port mappings' + }) + + Exec dockerStop = project.tasks.create('dockerStop', Exec, { + group = 'Docker Run' + description = 'Stops the named container if it is running' + ignoreExitValue = true + }) + + Exec dockerRemoveContainer = project.tasks.create('dockerRemoveContainer', Exec, { + group = 'Docker Run' + description = 'Removes the persistent container associated with the Docker Run tasks' + ignoreExitValue = true + }) + + project.afterEvaluate { + /** Inspect status of docker. */ + dockerRunStatus.with { + standardOutput = new ByteArrayOutputStream() + commandLine 'docker', 'inspect', '--format={{.State.Running}}', ext.name + doLast { + if (standardOutput.toString().trim() != 'true') { + println "Docker container '${ext.name}' is STOPPED." + return 1 + } else { + println "Docker container '${ext.name}' is RUNNING." + } + } + } + + /** + * Run a docker container. See {@link DockerRunExtension} for supported + * arguments. + * + * Replication of dockerRun task of com.palantir.docker-run plugin. + */ + dockerRun.with { + List args = new ArrayList() + args.addAll(['docker', 'run']) + + if (ext.daemonize) { + args.add('-d') + } + if (ext.clean) { + args.add('--rm') + } else { + finalizedBy dockerRunStatus + } + for (String port : ext.ports) { + args.add('-p') + args.add(port) + } + for (Map.Entry volume : ext.volumes.entrySet()) { + File localFile = project.file(volume.key) + + if (!localFile.exists()) { + logger.error("ERROR: Local folder ${localFile} doesn't exist. Mounted volume will not be visible to container") + throw new IllegalStateException("Local folder ${localFile} doesn't exist.") + } + args.add('-v') + args.add("${localFile.absolutePath}:${volume.value}") + } + args.addAll(ext.env.collect{ k, v -> ['-e', "${k}=${v}"]}.flatten()) + args.add('--name') + args.add(ext.name) + if (!ext.arguments.isEmpty()) { + args.addAll(ext.arguments) + } + args.add(ext.image) + + commandLine args + } + + dockerStop.with { + commandLine 'docker', 'stop', ext.name + } + + dockerRemoveContainer.with { + commandLine 'docker', 'rm', ext.name + } + } + } +} 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 c31482d577e0..c7a62237086e 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -23,7 +23,6 @@ import static java.util.UUID.randomUUID import com.github.jengelman.gradle.plugins.shadow.tasks.ShadowJar import groovy.json.JsonOutput import groovy.json.JsonSlurper -import java.net.ServerSocket import java.util.logging.Logger import org.gradle.api.attributes.Category import org.gradle.api.GradleException @@ -1252,7 +1251,7 @@ class BeamModulePlugin implements Plugin { if (configuration.shadowClosure) { // Ensure that tests are packaged and part of the artifact set. project.task('packageTests', type: Jar) { - classifier = 'tests-unshaded' + archiveClassifier = 'tests-unshaded' from project.sourceSets.test.output } project.artifacts.archives project.packageTests @@ -1560,13 +1559,13 @@ class BeamModulePlugin implements Plugin { } } - // Always configure the shadowJar classifier and merge service files. + // Always configure the shadowJar archiveClassifier and merge service files. if (configuration.shadowClosure) { // Only set the classifer on the unshaded classes if we are shading. - project.jar { classifier = "unshaded" } + project.jar { archiveClassifier = "unshaded" } project.shadowJar({ - classifier = null + archiveClassifier = null mergeServiceFiles() zip64 true into("META-INF/") { @@ -1575,11 +1574,11 @@ class BeamModulePlugin implements Plugin { } } << configuration.shadowClosure) - // Always configure the shadowTestJar classifier and merge service files. + // Always configure the shadowTestJar archiveClassifier and merge service files. project.task('shadowTestJar', type: ShadowJar, { group = "Shadow" description = "Create a combined JAR of project and test dependencies" - classifier = "tests" + archiveClassifier = "tests" from project.sourceSets.test.output configurations = [ project.configurations.testRuntimeMigration @@ -1639,7 +1638,7 @@ class BeamModulePlugin implements Plugin { project.tasks.register("testJar", Jar) { group = "Jar" description = "Create a JAR of test classes" - classifier = "tests" + archiveClassifier = "tests" from project.sourceSets.test.output zip64 true exclude "META-INF/INDEX.LIST" @@ -1794,18 +1793,18 @@ class BeamModulePlugin implements Plugin { project.task('sourcesJar', type: Jar) { from project.sourceSets.main.allSource - classifier = 'sources' + archiveClassifier = 'sources' } project.artifacts.archives project.sourcesJar project.task('testSourcesJar', type: Jar) { from project.sourceSets.test.allSource - classifier = 'test-sources' + archiveClassifier = 'test-sources' } project.artifacts.archives project.testSourcesJar project.task('javadocJar', type: Jar, dependsOn: project.javadoc) { - classifier = 'javadoc' + archiveClassifier = 'javadoc' from project.javadoc.destinationDir } project.artifacts.archives project.javadocJar @@ -1915,8 +1914,8 @@ class BeamModulePlugin implements Plugin { def dependencyNode = dependenciesNode.appendNode('dependency') def appendClassifier = { dep -> dep.artifacts.each { art -> - if (art.hasProperty('classifier')) { - dependencyNode.appendNode('classifier', art.classifier) + if (art.hasProperty('archiveClassifier')) { + dependencyNode.appendNode('archiveClassifier', art.archiveClassifier) } } } @@ -2210,7 +2209,7 @@ class BeamModulePlugin implements Plugin { /** ***********************************************************************************************/ project.ext.applyDockerNature = { - project.apply plugin: "com.palantir.docker" + project.apply plugin: BeamDockerPlugin project.docker { noCache true } project.tasks.create(name: "copyLicenses", type: Copy) { from "${project.rootProject.projectDir}/LICENSE" @@ -2222,7 +2221,7 @@ class BeamModulePlugin implements Plugin { } project.ext.applyDockerRunNature = { - project.apply plugin: "com.palantir.docker-run" + project.apply plugin: BeamDockerRunPlugin } /** ***********************************************************************************************/ diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy index 061ccf27cce2..97d96e6cf1eb 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy @@ -126,7 +126,7 @@ artifactId=${project.name} } config.exclusions.each { exclude it } - classifier = null + archiveClassifier = null mergeServiceFiles() zip64 true exclude "META-INF/INDEX.LIST" diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar index afba109285af..7f93135c49b7 100644 Binary files a/gradle/wrapper/gradle-wrapper.jar and b/gradle/wrapper/gradle-wrapper.jar differ diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 4e86b9270786..ac72c34e8acc 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,7 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-7.6.2-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.3-bin.zip networkTimeout=10000 +validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists diff --git a/gradlew b/gradlew index 65dcd68d65c8..0adc8e1a5321 100755 --- a/gradlew +++ b/gradlew @@ -83,10 +83,8 @@ done # This is normally unused # shellcheck disable=SC2034 APP_BASE_NAME=${0##*/} -APP_HOME=$( cd "${APP_HOME:-./}" && pwd -P ) || exit - -# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. -DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' +# Discard cd standard output in case $CDPATH is set (https://github.com/gradle/gradle/issues/25036) +APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit # Use the maximum available, or set MAX_FD != -1 to use that value. MAX_FD=maximum @@ -133,10 +131,13 @@ location of your Java installation." fi else JAVACMD=java - which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + if ! command -v java >/dev/null 2>&1 + then + die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. Please set the JAVA_HOME variable in your environment to match the location of your Java installation." + fi fi # Increase the maximum file descriptors if we can. @@ -144,7 +145,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then case $MAX_FD in #( max*) # In POSIX sh, ulimit -H is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC3045 MAX_FD=$( ulimit -H -n ) || warn "Could not query maximum file descriptor limit" esac @@ -152,7 +153,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then '' | soft) :;; #( *) # In POSIX sh, ulimit -n is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC3045 ulimit -n "$MAX_FD" || warn "Could not set maximum file descriptor limit to $MAX_FD" esac @@ -197,6 +198,10 @@ if "$cygwin" || "$msys" ; then done fi + +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' + # Collect all arguments for the java command; # * $DEFAULT_JVM_OPTS, $JAVA_OPTS, and $GRADLE_OPTS can contain fragments of # shell script including quotes and variable substitutions, so put them in diff --git a/playground/kafka-emulator/build.gradle b/playground/kafka-emulator/build.gradle index 486a232f9b99..2d3f70aa9883 100644 --- a/playground/kafka-emulator/build.gradle +++ b/playground/kafka-emulator/build.gradle @@ -24,11 +24,11 @@ plugins { applyJavaNature(exportJavadoc: false, publish: false) distZip { - archiveName "${baseName}.zip" + archiveFileName = "${archiveBaseName}.zip" } distTar { - archiveName "${baseName}.tar" + archiveFileName = "${archiveBaseName}.tar" } dependencies { diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index bf4c122ca91f..cc427494ed6e 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -63,6 +63,8 @@ task copyDockerfileDependencies(type: Copy) { task copySdkHarnessLauncher(type: Copy) { dependsOn ":sdks:java:container:downloadCloudProfilerAgent" + // if licenses are required, they should be present before this task run. + mustRunAfter ":sdks:java:container:pullLicenses" from configurations.sdkHarnessLauncher into "build/target" diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index efc9ff3db9c6..c4a508680186 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -307,8 +307,8 @@ task spannerCodeCoverageReport(type: JacocoReport, dependsOn: test) { sourceDirectories.setFrom(files(project.sourceSets.main.allSource.srcDirs)) executionData.setFrom(file("${buildDir}/jacoco/test.exec")) reports { - html.enabled true - html.destination file("${buildDir}/reports/jacoco/spanner/") + html.getRequired().set(true) + html.getOutputLocation().set(file("${buildDir}/reports/jacoco/spanner/")) } }