Browse Source

Make YamlRestTest and ElasticDistributionPlugin cc compatible (#101923)

* Make YamlRestTest and ElasticDistributionPlugin cc compatible

These changes are addressing configuration cache incompatibilities of running
rest tests with the new ClassRule based Elasticsearch test cluster setup.

This allows running those tests with configuration cache enabled:
`./gradlew :modules:ingest-user-info:yamlRestTest --configuration-cache`
Rene Groeschke 1 year ago
parent
commit
aa48860fac

+ 16 - 8
build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/InternalDistributionDownloadPlugin.java

@@ -23,16 +23,17 @@ import org.elasticsearch.gradle.internal.info.BuildParams;
 import org.elasticsearch.gradle.internal.info.GlobalBuildInfoPlugin;
 import org.elasticsearch.gradle.util.GradleUtils;
 import org.gradle.api.GradleException;
-import org.gradle.api.NamedDomainObjectContainer;
 import org.gradle.api.Plugin;
 import org.gradle.api.Project;
 import org.gradle.api.artifacts.Dependency;
+import org.gradle.api.artifacts.dsl.DependencyHandler;
 import org.gradle.api.provider.Provider;
 
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.function.Function;
 
-import static org.elasticsearch.gradle.util.GradleUtils.projectDependency;
-
 /**
  * An internal elasticsearch build plugin that registers additional
  * distribution resolution strategies to the 'elasticsearch.download-distribution' plugin
@@ -64,18 +65,18 @@ public class InternalDistributionDownloadPlugin implements Plugin<Project> {
      * <p>
      * BWC versions are resolved as project to projects under `:distribution:bwc`.
      */
-    private void registerInternalDistributionResolutions(NamedDomainObjectContainer<DistributionResolution> resolutions) {
-        resolutions.register("localBuild", distributionResolution -> distributionResolution.setResolver((project, distribution) -> {
+    private void registerInternalDistributionResolutions(List<DistributionResolution> resolutions) {
+        resolutions.add(new DistributionResolution("local-build", (project, distribution) -> {
             if (isCurrentVersion(distribution)) {
                 // non-external project, so depend on local build
                 return new ProjectBasedDistributionDependency(
-                    config -> projectDependency(project, distributionProjectPath(distribution), config)
+                    config -> projectDependency(project.getDependencies(), distributionProjectPath(distribution), config)
                 );
             }
             return null;
         }));
 
-        resolutions.register("bwc", distributionResolution -> distributionResolution.setResolver((project, distribution) -> {
+        resolutions.add(new DistributionResolution("bwc", (project, distribution) -> {
             BwcVersions.UnreleasedVersionInfo unreleasedInfo = BuildParams.getBwcVersions()
                 .unreleasedInfo(Version.fromString(distribution.getVersion()));
             if (unreleasedInfo != null) {
@@ -89,7 +90,7 @@ public class InternalDistributionDownloadPlugin implements Plugin<Project> {
                 }
                 String projectConfig = getProjectConfig(distribution, unreleasedInfo);
                 return new ProjectBasedDistributionDependency(
-                    (config) -> projectDependency(project, unreleasedInfo.gradleProjectPath(), projectConfig)
+                    (config) -> projectDependency(project.getDependencies(), unreleasedInfo.gradleProjectPath(), projectConfig)
                 );
             }
             return null;
@@ -116,6 +117,13 @@ public class InternalDistributionDownloadPlugin implements Plugin<Project> {
         }
     }
 
+    private static Dependency projectDependency(DependencyHandler dependencyHandler, String projectPath, String projectConfig) {
+        Map<String, Object> depConfig = new HashMap<>();
+        depConfig.put("path", projectPath);
+        depConfig.put("configuration", projectConfig);
+        return dependencyHandler.project(depConfig);
+    }
+
     private static String distributionProjectPath(ElasticsearchDistribution distribution) {
         String projectPath = ":distribution";
         if (distribution.getType() == ElasticsearchDistributionTypes.INTEG_TEST_ZIP) {

+ 29 - 0
build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/LegacyRestTestBasePlugin.java

@@ -22,12 +22,18 @@ import org.elasticsearch.gradle.util.GradleUtils;
 import org.gradle.api.NamedDomainObjectContainer;
 import org.gradle.api.Plugin;
 import org.gradle.api.Project;
+import org.gradle.api.Task;
 import org.gradle.api.plugins.JavaBasePlugin;
 import org.gradle.api.provider.Provider;
 import org.gradle.api.provider.ProviderFactory;
+import org.gradle.api.specs.NotSpec;
+import org.gradle.api.specs.Spec;
 import org.gradle.api.tasks.Sync;
+import org.gradle.api.tasks.TaskContainer;
 import org.gradle.api.tasks.bundling.Zip;
 
+import java.util.Collections;
+
 import javax.inject.Inject;
 
 import static org.elasticsearch.gradle.internal.RestrictedBuildApiService.BUILD_API_RESTRICTIONS_SYS_PROPERTY;
@@ -47,6 +53,7 @@ public class LegacyRestTestBasePlugin implements Plugin<Project> {
     private static final String TESTS_CLUSTER_REMOTE_ACCESS = "tests.cluster.remote_access";
 
     private ProviderFactory providerFactory;
+    private Project project;
 
     @Inject
     public LegacyRestTestBasePlugin(ProviderFactory providerFactory) {
@@ -55,6 +62,7 @@ public class LegacyRestTestBasePlugin implements Plugin<Project> {
 
     @Override
     public void apply(Project project) {
+        this.project = project;
         Provider<RestrictedBuildApiService> serviceProvider = project.getGradle()
             .getSharedServices()
             .registerIfAbsent("restrictedBuildAPI", RestrictedBuildApiService.class, spec -> {
@@ -97,6 +105,7 @@ public class LegacyRestTestBasePlugin implements Plugin<Project> {
                     );
                 }
             }
+            configureCacheability(restIntegTestTask);
         });
 
         project.getTasks()
@@ -121,6 +130,26 @@ public class LegacyRestTestBasePlugin implements Plugin<Project> {
         });
     }
 
+    private void configureCacheability(RestIntegTestTask restIntegTestTask) {
+        TaskContainer tasks = project.getTasks();
+        Spec<Task> taskSpec = t -> tasks.withType(StandaloneRestIntegTestTask.class)
+            .stream()
+            .filter(task -> task != restIntegTestTask)
+            .anyMatch(task -> Collections.disjoint(task.getClusters(), restIntegTestTask.getClusters()) == false);
+        restIntegTestTask.getOutputs()
+            .doNotCacheIf(
+                "Caching disabled for this task since it uses a cluster shared by other tasks",
+                /*
+                 * Look for any other tasks which use the same cluster as this task. Since tests often have side effects for the cluster
+                 * they execute against, this state can cause issues when trying to cache tests results of tasks that share a cluster. To
+                 * avoid any undesired behavior we simply disable the cache if we detect that this task uses a cluster shared between
+                 * multiple tasks.
+                 */
+                taskSpec
+            );
+        restIntegTestTask.getOutputs().upToDateWhen(new NotSpec(taskSpec));
+    }
+
     private String systemProperty(String propName) {
         return providerFactory.systemProperty(propName).getOrNull();
     }

+ 17 - 11
build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/rest/RestTestBasePlugin.java

@@ -38,6 +38,8 @@ import org.gradle.api.artifacts.Dependency;
 import org.gradle.api.artifacts.ProjectDependency;
 import org.gradle.api.artifacts.type.ArtifactTypeDefinition;
 import org.gradle.api.attributes.Attribute;
+import org.gradle.api.file.ConfigurableFileCollection;
+import org.gradle.api.file.FileCollection;
 import org.gradle.api.file.FileTree;
 import org.gradle.api.provider.ProviderFactory;
 import org.gradle.api.tasks.ClasspathNormalizer;
@@ -134,16 +136,20 @@ public class RestTestBasePlugin implements Plugin<Project> {
             task.systemProperty("tests.system_call_filter", "false");
 
             // Register plugins and modules as task inputs and pass paths as system properties to tests
-            nonInputSystemProperties.systemProperty(TESTS_CLUSTER_MODULES_PATH_SYSPROP, modulesConfiguration::getAsPath);
-            registerConfigurationInputs(task, modulesConfiguration);
-            nonInputSystemProperties.systemProperty(TESTS_CLUSTER_PLUGINS_PATH_SYSPROP, pluginsConfiguration::getAsPath);
-            registerConfigurationInputs(task, extractedPluginsConfiguration);
+            var modulePath = project.getObjects().fileCollection().from(modulesConfiguration);
+            nonInputSystemProperties.systemProperty(TESTS_CLUSTER_MODULES_PATH_SYSPROP, modulePath::getAsPath);
+            registerConfigurationInputs(task, modulesConfiguration.getName(), modulePath);
+            var pluginPath = project.getObjects().fileCollection().from(pluginsConfiguration);
+            nonInputSystemProperties.systemProperty(TESTS_CLUSTER_PLUGINS_PATH_SYSPROP, pluginPath::getAsPath);
+            registerConfigurationInputs(
+                task,
+                extractedPluginsConfiguration.getName(),
+                project.getObjects().fileCollection().from(extractedPluginsConfiguration)
+            );
 
             // Wire up integ-test distribution by default for all test tasks
-            nonInputSystemProperties.systemProperty(
-                INTEG_TEST_DISTRIBUTION_SYSPROP,
-                () -> integTestDistro.getExtracted().getSingleFile().getPath()
-            );
+            FileCollection extracted = integTestDistro.getExtracted();
+            nonInputSystemProperties.systemProperty(INTEG_TEST_DISTRIBUTION_SYSPROP, () -> extracted.getSingleFile().getPath());
             nonInputSystemProperties.systemProperty(TESTS_RUNTIME_JAVA_SYSPROP, BuildParams.getRuntimeJavaHome());
 
             // Add `usesDefaultDistribution()` extension method to test tasks to indicate they require the default distro
@@ -216,15 +222,15 @@ public class RestTestBasePlugin implements Plugin<Project> {
         return distribution.getExtracted().getAsFileTree().matching(patternFilter);
     }
 
-    private void registerConfigurationInputs(Task task, Configuration configuration) {
+    private void registerConfigurationInputs(Task task, String configurationName, ConfigurableFileCollection configuration) {
         task.getInputs()
             .files(providerFactory.provider(() -> configuration.getAsFileTree().filter(f -> f.getName().endsWith(".jar") == false)))
-            .withPropertyName(configuration.getName() + "-files")
+            .withPropertyName(configurationName + "-files")
             .withPathSensitivity(PathSensitivity.RELATIVE);
 
         task.getInputs()
             .files(providerFactory.provider(() -> configuration.getAsFileTree().filter(f -> f.getName().endsWith(".jar"))))
-            .withPropertyName(configuration.getName() + "-classpath")
+            .withPropertyName(configurationName + "-classpath")
             .withNormalizer(ClasspathNormalizer.class);
     }
 

+ 1 - 1
build-tools/src/integTest/groovy/org/elasticsearch/gradle/TestClustersPluginFuncTest.groovy

@@ -34,7 +34,7 @@ class TestClustersPluginFuncTest extends AbstractGradleFuncTest {
                 id 'elasticsearch.testclusters'
             }
 
-            class SomeClusterAwareTask extends DefaultTask implements TestClustersAware {
+            abstract class SomeClusterAwareTask extends DefaultTask implements TestClustersAware {
 
                 private Collection<ElasticsearchCluster> clusters = new HashSet<>();
 

+ 60 - 52
build-tools/src/main/java/org/elasticsearch/gradle/DistributionDownloadPlugin.java

@@ -11,6 +11,7 @@ package org.elasticsearch.gradle;
 import org.elasticsearch.gradle.distribution.ElasticsearchDistributionTypes;
 import org.elasticsearch.gradle.transform.SymbolicLinkPreservingUntarTransform;
 import org.elasticsearch.gradle.transform.UnzipTransform;
+import org.gradle.api.Action;
 import org.gradle.api.NamedDomainObjectContainer;
 import org.gradle.api.Plugin;
 import org.gradle.api.Project;
@@ -22,7 +23,8 @@ import org.gradle.api.model.ObjectFactory;
 import org.gradle.api.provider.Property;
 import org.gradle.api.provider.Provider;
 
-import java.util.Comparator;
+import java.util.ArrayList;
+import java.util.List;
 
 import javax.inject.Inject;
 
@@ -42,9 +44,10 @@ public class DistributionDownloadPlugin implements Plugin<Project> {
     private static final String DOWNLOAD_REPO_NAME = "elasticsearch-downloads";
     private static final String SNAPSHOT_REPO_NAME = "elasticsearch-snapshots";
     public static final String DISTRO_EXTRACTED_CONFIG_PREFIX = "es_distro_extracted_";
+    public static final String DISTRO_CONFIG_PREFIX = "es_distro_file_";
 
     private NamedDomainObjectContainer<ElasticsearchDistribution> distributionsContainer;
-    private NamedDomainObjectContainer<DistributionResolution> distributionsResolutionStrategiesContainer;
+    private List<DistributionResolution> distributionsResolutionStrategies;
 
     private Property<Boolean> dockerAvailability;
 
@@ -77,7 +80,7 @@ public class DistributionDownloadPlugin implements Plugin<Project> {
 
     private void setupDistributionContainer(Project project, Property<Boolean> dockerAvailable) {
         distributionsContainer = project.container(ElasticsearchDistribution.class, name -> {
-            Configuration fileConfiguration = project.getConfigurations().create("es_distro_file_" + name);
+            Configuration fileConfiguration = project.getConfigurations().create(DISTRO_CONFIG_PREFIX + name);
             Configuration extractedConfiguration = project.getConfigurations().create(DISTRO_EXTRACTED_CONFIG_PREFIX + name);
             extractedConfiguration.getAttributes()
                 .attribute(ArtifactTypeDefinition.ARTIFACT_TYPE_ATTRIBUTE, ArtifactTypeDefinition.DIRECTORY_TYPE);
@@ -85,21 +88,17 @@ public class DistributionDownloadPlugin implements Plugin<Project> {
                 name,
                 project.getObjects(),
                 dockerAvailability,
-                fileConfiguration,
-                extractedConfiguration,
-                (dist) -> finalizeDistributionDependencies(project, dist)
+                project.getObjects().fileCollection().from(fileConfiguration),
+                project.getObjects().fileCollection().from(extractedConfiguration),
+                new FinalizeDistributionAction(distributionsResolutionStrategies, project)
             );
         });
         project.getExtensions().add(CONTAINER_NAME, distributionsContainer);
     }
 
     private void setupResolutionsContainer(Project project) {
-        distributionsResolutionStrategiesContainer = project.container(DistributionResolution.class);
-        // We want this ordered in the same resolution strategies are added
-        distributionsResolutionStrategiesContainer.whenObjectAdded(
-            resolveDependencyNotation -> resolveDependencyNotation.setPriority(distributionsResolutionStrategiesContainer.size())
-        );
-        project.getExtensions().add(RESOLUTION_CONTAINER_NAME, distributionsResolutionStrategiesContainer);
+        distributionsResolutionStrategies = new ArrayList<>();
+        project.getExtensions().add(RESOLUTION_CONTAINER_NAME, distributionsResolutionStrategies);
     }
 
     @SuppressWarnings("unchecked")
@@ -108,30 +107,8 @@ public class DistributionDownloadPlugin implements Plugin<Project> {
     }
 
     @SuppressWarnings("unchecked")
-    public static NamedDomainObjectContainer<DistributionResolution> getRegistrationsContainer(Project project) {
-        return (NamedDomainObjectContainer<DistributionResolution>) project.getExtensions().getByName(RESOLUTION_CONTAINER_NAME);
-    }
-
-    private void finalizeDistributionDependencies(Project project, ElasticsearchDistribution distribution) {
-        DependencyHandler dependencies = project.getDependencies();
-        // for the distribution as a file, just depend on the artifact directly
-        DistributionDependency distributionDependency = resolveDependencyNotation(project, distribution);
-        dependencies.add(distribution.configuration.getName(), distributionDependency.getDefaultNotation());
-        // no extraction needed for rpm, deb or docker
-        if (distribution.getType().shouldExtract()) {
-            // The extracted configuration depends on the artifact directly but has
-            // an artifact transform registered to resolve it as an unpacked folder.
-            dependencies.add(distribution.getExtracted().getName(), distributionDependency.getExtractedNotation());
-        }
-    }
-
-    private DistributionDependency resolveDependencyNotation(Project p, ElasticsearchDistribution distribution) {
-        return distributionsResolutionStrategiesContainer.stream()
-            .sorted(Comparator.comparingInt(DistributionResolution::getPriority))
-            .map(r -> r.getResolver().resolve(p, distribution))
-            .filter(d -> d != null)
-            .findFirst()
-            .orElseGet(() -> DistributionDependency.of(dependencyNotation(distribution)));
+    public static List<DistributionResolution> getRegistrationsContainer(Project project) {
+        return (List<DistributionResolution>) project.getExtensions().getByName(RESOLUTION_CONTAINER_NAME);
     }
 
     private static void addIvyRepo(Project project, String name, String url, String group) {
@@ -155,22 +132,53 @@ public class DistributionDownloadPlugin implements Plugin<Project> {
         addIvyRepo(project, SNAPSHOT_REPO_NAME, "https://snapshots-no-kpi.elastic.co", FAKE_SNAPSHOT_IVY_GROUP);
     }
 
-    /**
-     * Returns a dependency object representing the given distribution.
-     * <p>
-     * The returned object is suitable to be passed to {@link DependencyHandler}.
-     * The concrete type of the object will be a set of maven coordinates as a {@link String}.
-     * Maven coordinates point to either the integ-test-zip coordinates on maven central, or a set of artificial
-     * coordinates that resolve to the Elastic download service through an ivy repository.
-     */
-    private String dependencyNotation(ElasticsearchDistribution distribution) {
-        if (distribution.getType() == ElasticsearchDistributionTypes.INTEG_TEST_ZIP) {
-            return "org.elasticsearch.distribution.integ-test-zip:elasticsearch:" + distribution.getVersion() + "@zip";
+    private record FinalizeDistributionAction(List<DistributionResolution> resolutionList, Project project)
+        implements
+            Action<ElasticsearchDistribution> {
+        @Override
+
+        public void execute(ElasticsearchDistribution distro) {
+            finalizeDistributionDependencies(project, distro);
+        }
+
+        private void finalizeDistributionDependencies(Project project, ElasticsearchDistribution distribution) {
+            // for the distribution as a file, just depend on the artifact directly
+            DistributionDependency distributionDependency = resolveDependencyNotation(project, distribution);
+            project.getDependencies().add(DISTRO_CONFIG_PREFIX + distribution.getName(), distributionDependency.getDefaultNotation());
+            // no extraction needed for rpm, deb or docker
+            if (distribution.getType().shouldExtract()) {
+                // The extracted configuration depends on the artifact directly but has
+                // an artifact transform registered to resolve it as an unpacked folder.
+                project.getDependencies()
+                    .add(DISTRO_EXTRACTED_CONFIG_PREFIX + distribution.getName(), distributionDependency.getExtractedNotation());
+            }
+        }
+
+        private DistributionDependency resolveDependencyNotation(Project project, ElasticsearchDistribution distro) {
+            return resolutionList.stream()
+                .map(r -> r.getResolver().resolve(project, distro))
+                .filter(d -> d != null)
+                .findFirst()
+                .orElseGet(() -> DistributionDependency.of(dependencyNotation(distro)));
+        }
+
+        /**
+         * Returns a dependency object representing the given distribution.
+         * <p>
+         * The returned object is suitable to be passed to {@link DependencyHandler}.
+         * The concrete type of the object will be a set of maven coordinates as a {@link String}.
+         * Maven coordinates point to either the integ-test-zip coordinates on maven central, or a set of artificial
+         * coordinates that resolve to the Elastic download service through an ivy repository.
+         */
+        private String dependencyNotation(ElasticsearchDistribution distribution) {
+            if (distribution.getType() == ElasticsearchDistributionTypes.INTEG_TEST_ZIP) {
+                return "org.elasticsearch.distribution.integ-test-zip:elasticsearch:" + distribution.getVersion() + "@zip";
+            }
+            Version distroVersion = Version.fromString(distribution.getVersion());
+            String extension = distribution.getType().getExtension(distribution.getPlatform());
+            String classifier = distribution.getType().getClassifier(distribution.getPlatform(), distroVersion);
+            String group = distribution.getVersion().endsWith("-SNAPSHOT") ? FAKE_SNAPSHOT_IVY_GROUP : FAKE_IVY_GROUP;
+            return group + ":elasticsearch" + ":" + distribution.getVersion() + classifier + "@" + extension;
         }
-        Version distroVersion = Version.fromString(distribution.getVersion());
-        String extension = distribution.getType().getExtension(distribution.getPlatform());
-        String classifier = distribution.getType().getClassifier(distribution.getPlatform(), distroVersion);
-        String group = distribution.getVersion().endsWith("-SNAPSHOT") ? FAKE_SNAPSHOT_IVY_GROUP : FAKE_IVY_GROUP;
-        return group + ":elasticsearch" + ":" + distribution.getVersion() + classifier + "@" + extension;
     }
 }

+ 6 - 1
build-tools/src/main/java/org/elasticsearch/gradle/DistributionResolution.java

@@ -12,9 +12,14 @@ import org.gradle.api.Project;
 
 public class DistributionResolution {
     private Resolver resolver;
-    private String name;
+    private final String name;
     private int priority;
 
+    public DistributionResolution(String name, Resolver resolver) {
+        this(name);
+        this.resolver = resolver;
+    }
+
     public DistributionResolution(String name) {
         this.name = name;
     }

+ 7 - 6
build-tools/src/main/java/org/elasticsearch/gradle/ElasticsearchDistribution.java

@@ -11,7 +11,8 @@ package org.elasticsearch.gradle;
 import org.elasticsearch.gradle.distribution.ElasticsearchDistributionTypes;
 import org.gradle.api.Action;
 import org.gradle.api.Buildable;
-import org.gradle.api.artifacts.Configuration;
+import org.gradle.api.file.ConfigurableFileCollection;
+import org.gradle.api.file.FileCollection;
 import org.gradle.api.model.ObjectFactory;
 import org.gradle.api.provider.Property;
 import org.gradle.api.tasks.TaskDependency;
@@ -44,7 +45,7 @@ public class ElasticsearchDistribution implements Buildable, Iterable<File> {
     private final String name;
     private final Property<Boolean> dockerAvailability;
     // pkg private so plugin can configure
-    final Configuration configuration;
+    final FileCollection configuration;
 
     private final Property<Architecture> architecture;
     private final Property<String> version;
@@ -52,7 +53,7 @@ public class ElasticsearchDistribution implements Buildable, Iterable<File> {
     private final Property<Platform> platform;
     private final Property<Boolean> bundledJdk;
     private final Property<Boolean> failIfUnavailable;
-    private final Configuration extracted;
+    private final ConfigurableFileCollection extracted;
     private Action<ElasticsearchDistribution> distributionFinalizer;
     private boolean frozen = false;
 
@@ -60,8 +61,8 @@ public class ElasticsearchDistribution implements Buildable, Iterable<File> {
         String name,
         ObjectFactory objectFactory,
         Property<Boolean> dockerAvailability,
-        Configuration fileConfiguration,
-        Configuration extractedConfiguration,
+        ConfigurableFileCollection fileConfiguration,
+        ConfigurableFileCollection extractedConfiguration,
         Action<ElasticsearchDistribution> distributionFinalizer
     ) {
         this.name = name;
@@ -172,7 +173,7 @@ public class ElasticsearchDistribution implements Buildable, Iterable<File> {
         return configuration.getSingleFile().toString();
     }
 
-    public Configuration getExtracted() {
+    public ConfigurableFileCollection getExtracted() {
         if (getType().shouldExtract() == false) {
             throw new UnsupportedOperationException(
                 "distribution type [" + getType().getName() + "] for " + "elasticsearch distribution [" + name + "] cannot be extracted"

+ 1 - 1
build-tools/src/main/java/org/elasticsearch/gradle/testclusters/DefaultTestClustersTask.java

@@ -12,7 +12,7 @@ import org.gradle.api.DefaultTask;
 import java.util.Collection;
 import java.util.HashSet;
 
-public class DefaultTestClustersTask extends DefaultTask implements TestClustersAware {
+public abstract class DefaultTestClustersTask extends DefaultTask implements TestClustersAware {
 
     private Collection<ElasticsearchCluster> clusters = new HashSet<>();
 

+ 7 - 20
build-tools/src/main/java/org/elasticsearch/gradle/testclusters/StandaloneRestIntegTestTask.java

@@ -8,11 +8,9 @@
 package org.elasticsearch.gradle.testclusters;
 
 import org.elasticsearch.gradle.FileSystemOperationsAware;
-import org.gradle.api.Task;
+import org.gradle.api.provider.ProviderFactory;
 import org.gradle.api.services.internal.BuildServiceProvider;
 import org.gradle.api.services.internal.BuildServiceRegistryInternal;
-import org.gradle.api.specs.NotSpec;
-import org.gradle.api.specs.Spec;
 import org.gradle.api.tasks.CacheableTask;
 import org.gradle.api.tasks.Internal;
 import org.gradle.api.tasks.Nested;
@@ -28,6 +26,8 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 
+import javax.inject.Inject;
+
 import static org.elasticsearch.gradle.testclusters.TestClustersPlugin.THROTTLE_SERVICE_NAME;
 
 /**
@@ -42,23 +42,6 @@ public abstract class StandaloneRestIntegTestTask extends Test implements TestCl
     private boolean debugServer = false;
 
     public StandaloneRestIntegTestTask() {
-        Spec<Task> taskSpec = t -> getProject().getTasks()
-            .withType(StandaloneRestIntegTestTask.class)
-            .stream()
-            .filter(task -> task != this)
-            .anyMatch(task -> Collections.disjoint(task.getClusters(), getClusters()) == false);
-        this.getOutputs()
-            .doNotCacheIf(
-                "Caching disabled for this task since it uses a cluster shared by other tasks",
-                /*
-                 * Look for any other tasks which use the same cluster as this task. Since tests often have side effects for the cluster
-                 * they execute against, this state can cause issues when trying to cache tests results of tasks that share a cluster. To
-                 * avoid any undesired behavior we simply disable the cache if we detect that this task uses a cluster shared between
-                 * multiple tasks.
-                 */
-                taskSpec
-            );
-        this.getOutputs().upToDateWhen(new NotSpec(taskSpec));
         this.getOutputs()
             .doNotCacheIf(
                 "Caching disabled for this task since it is configured to preserve data directory",
@@ -79,6 +62,10 @@ public abstract class StandaloneRestIntegTestTask extends Test implements TestCl
         return clusters;
     }
 
+    @Override
+    @Inject
+    public abstract ProviderFactory getProviderFactory();
+
     @Override
     @Internal
     public List<ResourceLock> getSharedResources() {

+ 7 - 0
build-tools/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersAware.java

@@ -9,17 +9,24 @@ package org.elasticsearch.gradle.testclusters;
 
 import org.gradle.api.Task;
 import org.gradle.api.artifacts.Configuration;
+import org.gradle.api.provider.Property;
 import org.gradle.api.provider.Provider;
+import org.gradle.api.services.ServiceReference;
 import org.gradle.api.tasks.Nested;
 
 import java.util.Collection;
 import java.util.concurrent.Callable;
 
+import static org.elasticsearch.gradle.testclusters.TestClustersPlugin.REGISTRY_SERVICE_NAME;
+
 public interface TestClustersAware extends Task {
 
     @Nested
     Collection<ElasticsearchCluster> getClusters();
 
+    @ServiceReference(REGISTRY_SERVICE_NAME)
+    Property<TestClustersRegistry> getRegistery();
+
     default void useCluster(ElasticsearchCluster cluster) {
         if (cluster.getPath().equals(getProject().getPath()) == false) {
             throw new TestClustersException("Task " + getPath() + " can't use test cluster from" + " another project " + cluster);

+ 11 - 2
build-tools/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java

@@ -37,6 +37,7 @@ import org.gradle.tooling.events.task.TaskFinishEvent;
 import java.io.File;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.function.Function;
 
 import javax.inject.Inject;
@@ -49,7 +50,7 @@ public class TestClustersPlugin implements Plugin<Project> {
     public static final String THROTTLE_SERVICE_NAME = "testClustersThrottle";
 
     private static final String LIST_TASK_NAME = "listTestClusters";
-    private static final String REGISTRY_SERVICE_NAME = "testClustersRegistry";
+    public static final String REGISTRY_SERVICE_NAME = "testClustersRegistry";
     private static final Logger logger = Logging.getLogger(TestClustersPlugin.class);
     private final ProviderFactory providerFactory;
     private Provider<File> runtimeJavaProvider;
@@ -222,13 +223,21 @@ public class TestClustersPlugin implements Plugin<Project> {
                         testClusterTasksService.get().register(awareTask.getPath(), awareTask);
                         awareTask.doFirst(task -> {
                             awareTask.beforeStart();
-                            awareTask.getClusters().forEach(registry::maybeStartCluster);
+                            awareTask.getClusters().forEach(awareTask.getRegistery().get()::maybeStartCluster);
                         });
                     });
             });
         }
     }
 
+    public static void maybeStartCluster(ElasticsearchCluster cluster, Set<ElasticsearchCluster> runningClusters) {
+        if (runningClusters.contains(cluster)) {
+            return;
+        }
+        runningClusters.add(cluster);
+        cluster.start();
+    }
+
     static public abstract class TaskEventsService implements BuildService<BuildServiceParameters.None>, OperationCompletionListener {
 
         Map<String, TestClustersAware> tasksMap = new HashMap<>();

+ 0 - 12
build-tools/src/main/java/org/elasticsearch/gradle/util/GradleUtils.java

@@ -13,7 +13,6 @@ import org.gradle.api.Project;
 import org.gradle.api.Task;
 import org.gradle.api.UnknownTaskException;
 import org.gradle.api.artifacts.Configuration;
-import org.gradle.api.artifacts.Dependency;
 import org.gradle.api.artifacts.ModuleDependency;
 import org.gradle.api.artifacts.ProjectDependency;
 import org.gradle.api.plugins.JavaBasePlugin;
@@ -34,7 +33,6 @@ import org.gradle.plugins.ide.idea.model.IdeaModel;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.function.Function;
@@ -183,16 +181,6 @@ public abstract class GradleUtils {
         }
     }
 
-    public static Dependency projectDependency(Project project, String projectPath, String projectConfig) {
-        if (project.findProject(projectPath) == null) {
-            throw new GradleException("no project [" + projectPath + "], project names: " + project.getRootProject().getAllprojects());
-        }
-        Map<String, Object> depConfig = new HashMap<>();
-        depConfig.put("path", projectPath);
-        depConfig.put("configuration", projectConfig);
-        return project.getDependencies().project(depConfig);
-    }
-
     /**
      * To calculate the project path from a task path without relying on Task#getProject() which is discouraged during
      * task execution time.

+ 3 - 1
distribution/tools/java-version-checker/build.gradle

@@ -8,15 +8,17 @@ tasks.named(sourceSets.unsupportedJdkVersionEntrypoint.compileJavaTaskName).conf
   targetCompatibility = JavaVersion.VERSION_1_8
 }
 
+
 tasks.named("jar") {
   manifest {
     attributes("Multi-Release": "true")
   }
 
+  FileCollection mainOutput = sourceSets.main.output;
   from(sourceSets.unsupportedJdkVersionEntrypoint.output)
   eachFile { details ->
     if (details.path.equals("org/elasticsearch/tools/java_version_checker/JavaVersionChecker.class") &&
-      sourceSets.main.output.asFileTree.contains(details.file)) {
+      mainOutput.asFileTree.contains(details.file)) {
       details.relativePath = details.relativePath.prepend("META-INF/versions/17")
     }
   }