浏览代码

Add gradle plugin for downloading jdk (#41461)

We currently download 3 variants of the same version of the jdk for
bundling into the distributions. Additionally, the vagrant images do
their own downloading. This commit moves the jdk downloading into a
utility gradle plugin. This will be used in a future PR by the packaging
tests.

The new plugin exposes a "jdks" project extension which allows creating
named jdks. Once the jdk version and platform are set for a named jdk,
the jdk object may be used as a lazy String for the jdk home path, or a
file collection for copying.
Ryan Ernst 6 年之前
父节点
当前提交
c7db902604

+ 1 - 0
buildSrc/build.gradle

@@ -124,6 +124,7 @@ dependencies {
   compile 'com.avast.gradle:gradle-docker-compose-plugin:0.8.12'
   testCompile "junit:junit:${props.getProperty('junit')}"
   testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${props.getProperty('randomizedrunner')}"
+  testCompile 'com.github.tomakehurst:wiremock-jre8-standalone:2.23.2'
 }
 
 /*****************************************************************************

+ 112 - 0
buildSrc/src/main/java/org/elasticsearch/gradle/Jdk.java

@@ -0,0 +1,112 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.gradle;
+
+import org.gradle.api.Buildable;
+import org.gradle.api.Project;
+import org.gradle.api.artifacts.Configuration;
+import org.gradle.api.provider.Property;
+import org.gradle.api.tasks.TaskDependency;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.regex.Pattern;
+
+public class Jdk implements Buildable, Iterable<File> {
+
+    static final Pattern VERSION_PATTERN = Pattern.compile("(\\d+)(\\.\\d+\\.\\d+)?\\+(\\d+)(@([a-f0-9]{32}))?");
+    private static final List<String> ALLOWED_PLATFORMS = Collections.unmodifiableList(Arrays.asList("linux", "windows", "darwin"));
+
+    private final String name;
+    private final Configuration configuration;
+
+    private final Property<String> version;
+    private final Property<String> platform;
+
+
+    Jdk(String name, Project project) {
+        this.name = name;
+        this.configuration = project.getConfigurations().create("jdk_" + name);
+        this.version = project.getObjects().property(String.class);
+        this.platform = project.getObjects().property(String.class);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getVersion() {
+        return version.get();
+    }
+
+    public void setVersion(String version) {
+        if (VERSION_PATTERN.matcher(version).matches() == false) {
+            throw new IllegalArgumentException("malformed version [" + version + "] for jdk [" + name + "]");
+        }
+        this.version.set(version);
+    }
+
+    public String getPlatform() {
+        return platform.get();
+    }
+
+    public void setPlatform(String platform) {
+        if (ALLOWED_PLATFORMS.contains(platform) == false) {
+            throw new IllegalArgumentException(
+                "unknown platform [" + platform + "] for jdk [" + name + "], must be one of " + ALLOWED_PLATFORMS);
+        }
+        this.platform.set(platform);
+    }
+
+    // pkg private, for internal use
+    Configuration getConfiguration() {
+        return configuration;
+    }
+
+    @Override
+    public String toString() {
+        return configuration.getSingleFile().toString();
+    }
+
+    @Override
+    public TaskDependency getBuildDependencies() {
+        return configuration.getBuildDependencies();
+    }
+
+    // internal, make this jdks configuration unmodifiable
+    void finalizeValues() {
+        if (version.isPresent() == false) {
+            throw new IllegalArgumentException("version not specified for jdk [" + name + "]");
+        }
+        if (platform.isPresent() == false) {
+            throw new IllegalArgumentException("platform not specified for jdk [" + name + "]");
+        }
+        version.finalizeValue();
+        platform.finalizeValue();
+    }
+
+    @Override
+    public Iterator<File> iterator() {
+        return configuration.iterator();
+    }
+}

+ 170 - 0
buildSrc/src/main/java/org/elasticsearch/gradle/JdkDownloadPlugin.java

@@ -0,0 +1,170 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.gradle;
+
+import org.gradle.api.Action;
+import org.gradle.api.NamedDomainObjectContainer;
+import org.gradle.api.Plugin;
+import org.gradle.api.Project;
+import org.gradle.api.UnknownTaskException;
+import org.gradle.api.artifacts.Configuration;
+import org.gradle.api.artifacts.ConfigurationContainer;
+import org.gradle.api.artifacts.dsl.DependencyHandler;
+import org.gradle.api.artifacts.repositories.IvyArtifactRepository;
+import org.gradle.api.file.CopySpec;
+import org.gradle.api.file.FileTree;
+import org.gradle.api.file.RelativePath;
+import org.gradle.api.tasks.Copy;
+import org.gradle.api.tasks.TaskProvider;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.function.Supplier;
+import java.util.regex.Matcher;
+
+public class JdkDownloadPlugin implements Plugin<Project> {
+
+    @Override
+    public void apply(Project project) {
+        NamedDomainObjectContainer<Jdk> jdksContainer = project.container(Jdk.class, name ->
+            new Jdk(name, project)
+        );
+        project.getExtensions().add("jdks", jdksContainer);
+
+        project.afterEvaluate(p -> {
+            for (Jdk jdk : jdksContainer) {
+                jdk.finalizeValues();
+                String version = jdk.getVersion();
+                String platform = jdk.getPlatform();
+
+                // depend on the jdk directory "artifact" from the root project
+                DependencyHandler dependencies = project.getDependencies();
+                Map<String, Object> depConfig = new HashMap<>();
+                depConfig.put("path", ":"); // root project
+                depConfig.put("configuration", configName("extracted_jdk", version, platform));
+                dependencies.add(jdk.getConfiguration().getName(), dependencies.project(depConfig));
+
+                // ensure a root level jdk download task exists
+                setupRootJdkDownload(project.getRootProject(), platform, version);
+            }
+        });
+    }
+
+    private static void setupRootJdkDownload(Project rootProject, String platform, String version) {
+        String extractTaskName = "extract" + capitalize(platform) + "Jdk" + version;
+        // NOTE: this is *horrendous*, but seems to be the only way to check for the existence of a registered task
+        try {
+            rootProject.getTasks().named(extractTaskName);
+            // already setup this version
+            return;
+        } catch (UnknownTaskException e) {
+            // fall through: register the task
+        }
+
+        // decompose the bundled jdk version, broken into elements as: [feature, interim, update, build]
+        // Note the "patch" version is not yet handled here, as it has not yet been used by java.
+        Matcher jdkVersionMatcher = Jdk.VERSION_PATTERN.matcher(version);
+        if (jdkVersionMatcher.matches() == false) {
+            throw new IllegalArgumentException("Malformed jdk version [" + version + "]");
+        }
+        String jdkVersion = jdkVersionMatcher.group(1) + (jdkVersionMatcher.group(2) != null ? (jdkVersionMatcher.group(2)) : "");
+        String jdkMajor = jdkVersionMatcher.group(1);
+        String jdkBuild = jdkVersionMatcher.group(3);
+        String hash = jdkVersionMatcher.group(5);
+
+        // add fake ivy repo for jdk url
+        String repoName = "jdk_repo_" + version;
+        if (rootProject.getRepositories().findByName(repoName) == null) {
+            // simpler legacy pattern from JDK 9 to JDK 12 that we are advocating to Oracle to bring back
+            rootProject.getRepositories().ivy(ivyRepo -> {
+                ivyRepo.setName(repoName);
+                ivyRepo.setUrl("https://download.oracle.com");
+                ivyRepo.metadataSources(IvyArtifactRepository.MetadataSources::artifact);
+                ivyRepo.patternLayout(layout ->
+                    layout.artifact("java/GA/jdk" + jdkMajor + "/" + jdkBuild + "/GPL/openjdk-[revision]_[module]-x64_bin.[ext]"));
+                ivyRepo.content(content -> content.includeGroup("jdk"));
+            });
+            // current pattern since 12.0.1
+            rootProject.getRepositories().ivy(ivyRepo -> {
+                ivyRepo.setName(repoName + "_with_hash");
+                ivyRepo.setUrl("https://download.oracle.com");
+                ivyRepo.metadataSources(IvyArtifactRepository.MetadataSources::artifact);
+                ivyRepo.patternLayout(layout -> layout.artifact(
+                    "java/GA/jdk" + jdkVersion + "/" + hash + "/" + jdkBuild + "/GPL/openjdk-[revision]_[module]-x64_bin.[ext]"));
+                ivyRepo.content(content -> content.includeGroup("jdk"));
+            });
+        }
+
+        // add the jdk as a "dependency"
+        final ConfigurationContainer configurations = rootProject.getConfigurations();
+        String remoteConfigName = configName("openjdk", version, platform);
+        String localConfigName = configName("extracted_jdk", version, platform);
+        Configuration jdkConfig = configurations.findByName(remoteConfigName);
+        if (jdkConfig == null) {
+            jdkConfig = configurations.create(remoteConfigName);
+            configurations.create(localConfigName);
+        }
+        String extension = platform.equals("windows") ? "zip" : "tar.gz";
+        String jdkDep = "jdk:" + (platform.equals("darwin") ? "osx" : platform) + ":" + jdkVersion + "@" + extension;
+        rootProject.getDependencies().add(configName("openjdk", version, platform), jdkDep);
+
+        // add task for extraction
+        // TODO: look into doing this as an artifact transform, which are cacheable starting in gradle 5.3
+        int rootNdx = platform.equals("darwin") ? 2 : 1;
+        Action<CopySpec> removeRootDir = copy -> {
+            // remove extra unnecessary directory levels
+            copy.eachFile(details -> {
+                String[] pathSegments = details.getRelativePath().getSegments();
+                String[] newPathSegments = Arrays.copyOfRange(pathSegments, rootNdx, pathSegments.length);
+                details.setRelativePath(new RelativePath(true, newPathSegments));
+            });
+            copy.setIncludeEmptyDirs(false);
+        };
+        // delay resolving jdkConfig until runtime
+        Supplier<File> jdkArchiveGetter = jdkConfig::getSingleFile;
+        final Callable<FileTree> fileGetter;
+        if (extension.equals("zip")) {
+            fileGetter = () -> rootProject.zipTree(jdkArchiveGetter.get());
+        } else {
+            fileGetter = () -> rootProject.tarTree(rootProject.getResources().gzip(jdkArchiveGetter.get()));
+        }
+        String extractDir = rootProject.getBuildDir().toPath().resolve("jdks/openjdk-" + jdkVersion + "_" + platform).toString();
+        TaskProvider<Copy> extractTask = rootProject.getTasks().register(extractTaskName, Copy.class, copyTask -> {
+            copyTask.doFirst(t -> rootProject.delete(extractDir));
+            copyTask.into(extractDir);
+            copyTask.from(fileGetter, removeRootDir);
+        });
+        rootProject.getArtifacts().add(localConfigName,
+            rootProject.getLayout().getProjectDirectory().dir(extractDir),
+            artifact -> artifact.builtBy(extractTask));
+    }
+
+    private static String configName(String prefix, String version, String platform) {
+        return prefix + "_" + version + "_" + platform;
+    }
+
+    private static String capitalize(String s) {
+        return s.substring(0, 1).toUpperCase(Locale.ROOT) + s.substring(1);
+    }
+}

+ 1 - 0
buildSrc/src/main/resources/META-INF/gradle-plugins/elasticsearch.jdk-download.properties

@@ -0,0 +1 @@
+implementation-class=org.elasticsearch.gradle.JdkDownloadPlugin

+ 110 - 0
buildSrc/src/test/java/org/elasticsearch/gradle/JdkDownloadPluginIT.java

@@ -0,0 +1,110 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.gradle;
+
+import com.github.tomakehurst.wiremock.WireMockServer;
+import org.elasticsearch.gradle.test.GradleIntegrationTestCase;
+import org.gradle.testkit.runner.BuildResult;
+import org.gradle.testkit.runner.GradleRunner;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.function.Consumer;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
+import static com.github.tomakehurst.wiremock.client.WireMock.get;
+import static com.github.tomakehurst.wiremock.client.WireMock.head;
+import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo;
+import static org.hamcrest.CoreMatchers.equalTo;
+
+public class JdkDownloadPluginIT extends GradleIntegrationTestCase {
+
+    private static final String FAKE_JDK_VERSION = "1.0.2+99";
+    private static final Pattern JDK_HOME_LOGLINE = Pattern.compile("JDK HOME: (.*)");
+    private static final Pattern NUM_CONFIGS_LOGLINE = Pattern.compile("NUM CONFIGS: (.*)");
+
+    public void testLinuxExtraction() throws IOException {
+        assertExtraction("getLinuxJdk", "linux", "bin/java");
+    }
+
+    public void testDarwinExtraction() throws IOException {
+        assertExtraction("getDarwinJdk", "osx", "Contents/Home/bin/java");
+    }
+
+    public void testWindowsExtraction() throws IOException {
+        assertExtraction("getWindowsJdk", "windows", "bin/java");
+    }
+
+    public void testCrossProjectReuse() throws IOException {
+        runBuild("numConfigurations", "linux", result -> {
+            Matcher matcher = NUM_CONFIGS_LOGLINE.matcher(result.getOutput());
+            assertTrue("could not find num configs in output: " + result.getOutput(), matcher.find());
+            assertThat(Integer.parseInt(matcher.group(1)), equalTo(6)); // 3 import configs, 3 export configs
+        });
+    }
+
+    public void assertExtraction(String taskname, String platform, String javaBin) throws IOException {
+        runBuild(taskname, platform, result -> {
+            Matcher matcher = JDK_HOME_LOGLINE.matcher(result.getOutput());
+            assertTrue("could not find jdk home in output: " + result.getOutput(), matcher.find());
+            String jdkHome = matcher.group(1);
+            Path javaPath = Paths.get(jdkHome, javaBin);
+            assertTrue(javaPath.toString(), Files.exists(javaPath));
+        });
+    }
+
+    private void runBuild(String taskname, String platform, Consumer<BuildResult> assertions) throws IOException {
+        WireMockServer wireMock = new WireMockServer(0);
+        try {
+            String extension = platform.equals("windows") ? "zip" : "tar.gz";
+            String filename = "openjdk-1.0.2_" + platform + "-x64_bin." + extension;
+            wireMock.stubFor(head(urlEqualTo("/java/GA/jdk1/99/GPL/" + filename))
+                .willReturn(aResponse().withStatus(200)));
+            final byte[] filebytes;
+            try (InputStream stream = JdkDownloadPluginIT.class.getResourceAsStream(filename)) {
+                filebytes = stream.readAllBytes();
+            }
+            wireMock.stubFor(get(urlEqualTo("/java/GA/jdk1/99/GPL/" + filename))
+                .willReturn(aResponse().withStatus(200).withBody(filebytes)));
+            wireMock.start();
+
+            GradleRunner runner = GradleRunner.create().withProjectDir(getProjectDir("jdk-download"))
+                .withArguments(taskname,
+                    "-Dlocal.repo.path=" + getLocalTestRepoPath(),
+                    "-Dtests.jdk_version=" + FAKE_JDK_VERSION,
+                    "-Dtests.jdk_repo=" + wireMock.baseUrl())
+                .withPluginClasspath();
+
+            BuildResult result = runner.build();
+            assertions.accept(result);
+        } catch (Exception e) {
+            // for debugging
+            System.err.println("missed requests: " + wireMock.findUnmatchedRequests().getRequests());
+            throw e;
+        } finally {
+            wireMock.stop();
+        }
+    }
+}

+ 78 - 0
buildSrc/src/test/java/org/elasticsearch/gradle/JdkDownloadPluginTests.java

@@ -0,0 +1,78 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.gradle;
+
+import org.elasticsearch.gradle.test.GradleUnitTestCase;
+import org.gradle.api.NamedDomainObjectContainer;
+import org.gradle.api.Project;
+import org.gradle.testfixtures.ProjectBuilder;
+import org.junit.BeforeClass;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+
+public class JdkDownloadPluginTests extends GradleUnitTestCase {
+    private static Project rootProject;
+
+    @BeforeClass
+    public static void setupRoot() {
+         rootProject = ProjectBuilder.builder().build();
+    }
+
+    public void testMissingVersion() {
+        assertJdkError(createProject(), "testjdk", null, "linux", "version not specified for jdk [testjdk]");
+    }
+
+    public void testMissingPlatform() {
+        assertJdkError(createProject(), "testjdk", "11.0.2+33", null, "platform not specified for jdk [testjdk]");
+    }
+
+    public void testUnknownPlatform() {
+        assertJdkError(createProject(), "testjdk", "11.0.2+33", "unknown",
+            "unknown platform [unknown] for jdk [testjdk], must be one of [linux, windows, darwin]");
+    }
+
+    public void testBadVersionFormat() {
+        assertJdkError(createProject(), "testjdk", "badversion", "linux", "malformed version [badversion] for jdk [testjdk]");
+    }
+
+    private void assertJdkError(Project project, String name, String version, String platform, String message) {
+        IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createJdk(project, name, version, platform));
+        assertThat(e.getMessage(), equalTo(message));
+    }
+
+    private void createJdk(Project project, String name, String version, String platform) {
+        @SuppressWarnings("unchecked")
+        NamedDomainObjectContainer<Jdk> jdks = (NamedDomainObjectContainer<Jdk>) project.getExtensions().getByName("jdks");
+        jdks.create(name, jdk -> {
+            if (version != null) {
+                jdk.setVersion(version);
+            }
+            if (platform != null) {
+                jdk.setPlatform(platform);
+            }
+        }).finalizeValues();
+    }
+
+    private Project createProject() {
+        Project project = ProjectBuilder.builder().withParent(rootProject).build();
+        project.getPlugins().apply("elasticsearch.jdk-download");
+        return project;
+    }
+}

+ 21 - 0
buildSrc/src/test/java/org/elasticsearch/gradle/test/BaseTestCase.java

@@ -22,6 +22,7 @@ import com.carrotsearch.randomizedtesting.JUnit4MethodProvider;
 import com.carrotsearch.randomizedtesting.RandomizedRunner;
 import com.carrotsearch.randomizedtesting.annotations.TestMethodProviders;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
+import junit.framework.AssertionFailedError;
 import org.junit.Assert;
 import org.junit.runner.RunWith;
 
@@ -32,4 +33,24 @@ import org.junit.runner.RunWith;
 })
 @ThreadLeakLingering(linger = 5000) // wait for "Connection worker" to die
 public abstract class BaseTestCase extends Assert {
+
+    // add expectThrows from junit 5
+    @FunctionalInterface
+    public interface ThrowingRunnable {
+        void run() throws Throwable;
+    }
+    public static <T extends Throwable> T expectThrows(Class<T> expectedType, ThrowingRunnable runnable) {
+        try {
+            runnable.run();
+        } catch (Throwable e) {
+            if (expectedType.isInstance(e)) {
+                return expectedType.cast(e);
+            }
+            AssertionFailedError assertion =
+                new AssertionFailedError("Unexpected exception type, expected " + expectedType.getSimpleName() + " but got " + e);
+            assertion.initCause(e);
+            throw assertion;
+        }
+        throw new AssertionFailedError("Expected exception "+ expectedType.getSimpleName() + " but no exception was thrown");
+    }
 }

二进制
buildSrc/src/test/resources/org/elasticsearch/gradle/openjdk-1.0.2_linux-x64_bin.tar.gz


二进制
buildSrc/src/test/resources/org/elasticsearch/gradle/openjdk-1.0.2_osx-x64_bin.tar.gz


二进制
buildSrc/src/test/resources/org/elasticsearch/gradle/openjdk-1.0.2_windows-x64_bin.zip


+ 15 - 0
buildSrc/src/testKit/jdk-download/build.gradle

@@ -0,0 +1,15 @@
+
+project.gradle.projectsEvaluated {
+    // wire the jdk repo to wiremock
+    String fakeJdkRepo = Objects.requireNonNull(System.getProperty('tests.jdk_repo'))
+    String fakeJdkVersion = Objects.requireNonNull(System.getProperty('tests.jdk_version'))
+    println rootProject.repositories.asMap.keySet()
+    IvyArtifactRepository repository = (IvyArtifactRepository) rootProject.repositories.getByName("jdk_repo_${fakeJdkVersion}")
+    repository.setUrl(fakeJdkRepo)
+}
+
+task numConfigurations {
+    doLast {
+        println "NUM CONFIGS: ${project.configurations.size()}"
+    }
+}

+ 9 - 0
buildSrc/src/testKit/jdk-download/reuse/build.gradle

@@ -0,0 +1,9 @@
+evaluationDependsOn ':subproj'
+
+String fakeJdkVersion = Objects.requireNonNull(System.getProperty('tests.jdk_version'))
+jdks {
+    linux_jdk {
+        version = fakeJdkVersion
+        platform = "linux"
+    }
+}

+ 1 - 0
buildSrc/src/testKit/jdk-download/settings.gradle

@@ -0,0 +1 @@
+include 'subproj'

+ 41 - 0
buildSrc/src/testKit/jdk-download/subproj/build.gradle

@@ -0,0 +1,41 @@
+plugins {
+    id 'elasticsearch.jdk-download'
+}
+
+
+String fakeJdkVersion = Objects.requireNonNull(System.getProperty('tests.jdk_version'))
+jdks {
+    linux {
+        version = fakeJdkVersion
+        platform = "linux"
+    }
+    darwin {
+        version = fakeJdkVersion
+        platform = "darwin"
+    }
+    windows {
+        version = fakeJdkVersion
+        platform = "windows"
+    }
+}
+
+task getLinuxJdk {
+    dependsOn jdks.linux
+    doLast {
+        println "JDK HOME: " + jdks.linux
+    }
+}
+
+task getDarwinJdk {
+    dependsOn jdks.darwin
+    doLast {
+        println "JDK HOME: " + jdks.darwin
+    }
+}
+
+task getWindowsJdk {
+    dependsOn jdks.windows
+    doLast {
+        println "JDK HOME: " + jdks.windows
+    }
+}

+ 1 - 1
distribution/archives/build.gradle

@@ -61,7 +61,7 @@ CopySpec archiveFiles(CopySpec modulesFiles, String distributionType, String pla
       }
       if (jdk) {
         into('jdk') {
-          with jdkFiles(platform)
+          with jdkFiles(project, platform)
         }
       }
       into('') {

+ 15 - 74
distribution/build.gradle

@@ -17,18 +17,16 @@
  * under the License.
  */
 
+
+import org.apache.tools.ant.filters.FixCrLfFilter
 import org.elasticsearch.gradle.ConcatFilesTask
 import org.elasticsearch.gradle.MavenFilteringHack
 import org.elasticsearch.gradle.NoticeTask
 import org.elasticsearch.gradle.VersionProperties
 import org.elasticsearch.gradle.test.RunTask
-import org.apache.tools.ant.filters.FixCrLfFilter
 
 import java.nio.file.Files
-import java.nio.file.Path
-import java.util.regex.Matcher
-import java.util.regex.Pattern
-
+import java.nio.file.Path 
 /*****************************************************************************
  *                  Third party dependencies report                          *
  *****************************************************************************/
@@ -219,72 +217,6 @@ xpack.subprojects.findAll { it.parent == xpack }.each { Project xpackModule ->
   copyLog4jProperties(buildDefaultLog4jConfig, xpackModule)
 }
 
-/*****************************************************************************
- *                                    JDKs                                   *
- *****************************************************************************/
-// extract the bundled jdk version, broken into elements as: [feature, interim, update, build]
-// Note the "patch" version is not yet handled here, as it has not yet been used by java.
-Pattern JDK_VERSION = Pattern.compile("(\\d+)(\\.\\d+\\.\\d+)?\\+(\\d+)@([a-f0-9]{32})?")
-Matcher jdkVersionMatcher = JDK_VERSION.matcher(VersionProperties.bundledJdk)
-if (jdkVersionMatcher.matches() == false) {
-  throw new IllegalArgumentException("Malformed jdk version [" + VersionProperties.bundledJdk + "]")
-}
-String jdkVersion = jdkVersionMatcher.group(1) + (jdkVersionMatcher.group(2) != null ? (jdkVersionMatcher.group(2)) : "")
-String jdkMajor = jdkVersionMatcher.group(1)
-String jdkBuild = jdkVersionMatcher.group(3)
-String hash = jdkVersionMatcher.group(4)
-
-repositories {
-  // simpler legacy pattern from JDK 9 to JDK 12 that we are advocating to Oracle to bring back
-  ivy {
-    name "legacy-jdk"
-    url "https://download.oracle.com"
-    metadataSources {
-      artifact()
-    }
-    patternLayout {
-      artifact "java/GA/jdk${jdkMajor}/${jdkBuild}/GPL/openjdk-[revision]_[module]-x64_bin.[ext]"
-    }
-  }
-  // current pattern since 12.0.1
-  ivy {
-    name "jdk"
-    url "https://download.oracle.com"
-    metadataSources {
-      artifact()
-    }
-    patternLayout {
-      artifact "java/GA/jdk${jdkVersion}/${hash}/${jdkBuild}/GPL/openjdk-[revision]_[module]-x64_bin.[ext]"
-    }
-  }
-}
-for (String platform : ['linux', 'darwin', 'windows']) {
-  String jdkConfigName = "jdk_${platform}"
-  Configuration jdkConfig = configurations.create(jdkConfigName)
-  String extension = platform.equals('windows') ? 'zip' : 'tar.gz'
-  dependencies.add(jdkConfigName, "jdk:${platform.equals('darwin') ? 'osx' : platform}:${jdkVersion}@${extension}")
-
-  int rootNdx = platform.equals('darwin') ? 2 : 1
-  Closure removeRootDir = {
-    it.eachFile { FileCopyDetails details ->
-      details.relativePath = new RelativePath(true, details.relativePath.segments[rootNdx..-1] as String[])
-    }
-    it.includeEmptyDirs false
-  }
-  String extractDir = "${buildDir}/jdks/openjdk-${jdkVersion}_${platform}"
-  project.task("extract${platform.capitalize()}Jdk", type: Copy) {
-    doFirst {
-      project.delete(extractDir)
-    }
-    into extractDir
-    if (extension.equals('zip')) {
-      from({ zipTree(jdkConfig.singleFile) }, removeRootDir)
-    } else {
-      from({ tarTree(resources.gzip(jdkConfig.singleFile)) }, removeRootDir)
-    }
-  }
-}
-
 // make sure we have a clean task since we aren't a java project, but we have tasks that
 // put stuff in the build dir
 task clean(type: Delete) {
@@ -292,6 +224,9 @@ task clean(type: Delete) {
 }
 
 configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) {
+
+  apply plugin: 'elasticsearch.jdk-download'
+
   // TODO: the map needs to be an input of the tasks, so that when it changes, the task will re-run...
   /*****************************************************************************
    *             Properties to expand when copying packaging files             *
@@ -433,9 +368,15 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) {
       }
     }
 
-    jdkFiles = { platform ->
-      copySpec {
-        from project(':distribution').tasks.getByName("extract${platform.capitalize()}Jdk")
+    jdkFiles = { project, platform ->
+      project.jdks {
+        "bundled_${platform}" {
+          it.platform = platform
+          it.version = VersionProperties.bundledJdk
+        }
+      }
+      return copySpec {
+        from project.jdks."bundled_${platform}"
         eachFile { FileCopyDetails details ->
           if (details.relativePath.segments[-2] == 'bin' || details.relativePath.segments[-1] == 'jspawnhelper') {
             details.mode = 0755

+ 1 - 1
distribution/packages/build.gradle

@@ -143,7 +143,7 @@ Closure commonPackageConfig(String type, boolean oss, boolean jdk) {
       }
       if (jdk) {
         into('jdk') {
-          with jdkFiles('linux')
+          with jdkFiles(project, 'linux')
         }
       }
       // we need to specify every intermediate directory in these paths so the package managers know they are explicitly