[Build] Make test cluster plugin configuration cache compatible (#116890)

This adds infrastructure to make the legacy test cluster plugin and
the legacy test cluster based test plugins generally configuration cache compatible.
This commit is contained in:
Rene Groeschke 2024-11-21 08:00:05 +01:00 committed by GitHub
parent ea4b41fca8
commit 8c20ac5884
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
22 changed files with 561 additions and 153 deletions

View file

@ -15,16 +15,12 @@ import org.elasticsearch.gradle.internal.test.AntFixture
import org.gradle.api.file.FileSystemOperations import org.gradle.api.file.FileSystemOperations
import org.gradle.api.file.ProjectLayout import org.gradle.api.file.ProjectLayout
import org.gradle.api.provider.ProviderFactory import org.gradle.api.provider.ProviderFactory
import org.gradle.api.tasks.Internal
import org.gradle.process.ExecOperations import org.gradle.process.ExecOperations
import javax.inject.Inject import javax.inject.Inject
abstract class AntFixtureStop extends LoggedExec implements FixtureStop { abstract class AntFixtureStop extends LoggedExec implements FixtureStop {
@Internal
AntFixture fixture
@Inject @Inject
AntFixtureStop(ProjectLayout projectLayout, AntFixtureStop(ProjectLayout projectLayout,
ExecOperations execOperations, ExecOperations execOperations,
@ -34,12 +30,12 @@ abstract class AntFixtureStop extends LoggedExec implements FixtureStop {
} }
void setFixture(AntFixture fixture) { void setFixture(AntFixture fixture) {
assert this.fixture == null def pidFile = fixture.pidFile
this.fixture = fixture; def fixtureName = fixture.name
final Object pid = "${-> this.fixture.pid}" final Object pid = "${-> Integer.parseInt(pidFile.getText('UTF-8').trim())}"
onlyIf("pidFile exists") { fixture.pidFile.exists() } onlyIf("pidFile exists") { pidFile.exists() }
doFirst { doFirst {
logger.info("Shutting down ${fixture.name} with pid ${pid}") logger.info("Shutting down ${fixtureName} with pid ${pid}")
} }
if (OS.current() == OS.WINDOWS) { if (OS.current() == OS.WINDOWS) {
@ -51,9 +47,8 @@ abstract class AntFixtureStop extends LoggedExec implements FixtureStop {
} }
doLast { doLast {
fileSystemOperations.delete { fileSystemOperations.delete {
it.delete(fixture.pidFile) it.delete(pidFile)
} }
} }
this.fixture = fixture
} }
} }

View file

@ -29,11 +29,6 @@ import java.nio.charset.Charset
*/ */
public abstract class AntTask extends DefaultTask { public abstract class AntTask extends DefaultTask {
/**
* A buffer that will contain the output of the ant code run,
* if the output was not already written directly to stdout.
*/
public final ByteArrayOutputStream outputBuffer = new ByteArrayOutputStream()
@Inject @Inject
protected FileSystemOperations getFileSystemOperations() { protected FileSystemOperations getFileSystemOperations() {
@ -57,6 +52,11 @@ public abstract class AntTask extends DefaultTask {
// otherwise groovy replaces System.out, and you have no chance to debug // otherwise groovy replaces System.out, and you have no chance to debug
// ant.saveStreams = false // ant.saveStreams = false
/**
* A buffer that will contain the output of the ant code run,
* if the output was not already written directly to stdout.
*/
ByteArrayOutputStream outputBuffer = new ByteArrayOutputStream()
final int outputLevel = logger.isDebugEnabled() ? Project.MSG_DEBUG : Project.MSG_INFO final int outputLevel = logger.isDebugEnabled() ? Project.MSG_DEBUG : Project.MSG_INFO
final PrintStream stream = useStdout() ? System.out : new PrintStream(outputBuffer, true, Charset.defaultCharset().name()) final PrintStream stream = useStdout() ? System.out : new PrintStream(outputBuffer, true, Charset.defaultCharset().name())

View file

@ -10,22 +10,37 @@
package org.elasticsearch.gradle.internal.test package org.elasticsearch.gradle.internal.test
import org.elasticsearch.gradle.OS import org.elasticsearch.gradle.OS
import org.elasticsearch.gradle.internal.AntFixtureStop import org.elasticsearch.gradle.internal.AntFixtureStop
import org.elasticsearch.gradle.internal.AntTask import org.elasticsearch.gradle.internal.AntTask
import org.elasticsearch.gradle.testclusters.TestClusterInfo
import org.elasticsearch.gradle.testclusters.TestClusterValueSource
import org.elasticsearch.gradle.testclusters.TestClustersRegistry
import org.gradle.api.GradleException import org.gradle.api.GradleException
import org.gradle.api.file.ProjectLayout
import org.gradle.api.provider.Property
import org.gradle.api.provider.Provider
import org.gradle.api.provider.ProviderFactory
import org.gradle.api.provider.ValueSource
import org.gradle.api.provider.ValueSourceParameters
import org.gradle.api.tasks.Input
import org.gradle.api.tasks.Internal import org.gradle.api.tasks.Internal
import org.gradle.api.tasks.TaskProvider import org.gradle.api.tasks.TaskProvider
import javax.inject.Inject
/** /**
* A fixture for integration tests which runs in a separate process launched by Ant. * A fixture for integration tests which runs in a separate process launched by Ant.
*/ */
class AntFixture extends AntTask implements Fixture { class AntFixture extends AntTask {
/** The path to the executable that starts the fixture. */ /** The path to the executable that starts the fixture. */
@Internal @Internal
String executable String executable
private final List<Object> arguments = new ArrayList<>() private final List<Object> arguments = new ArrayList<>()
private ProjectLayout projectLayout
private final ProviderFactory providerFactory
void args(Object... args) { void args(Object... args) {
arguments.addAll(args) arguments.addAll(args)
@ -69,19 +84,14 @@ class AntFixture extends AntTask implements Fixture {
return tmpFile.exists() return tmpFile.exists()
} }
private final TaskProvider<AntFixtureStop> stopTask @Inject
AntFixture(ProjectLayout projectLayout, ProviderFactory providerFactory) {
AntFixture() { this.providerFactory = providerFactory
stopTask = createStopTask() this.projectLayout = projectLayout;
TaskProvider<AntFixtureStop> stopTask = createStopTask()
finalizedBy(stopTask) finalizedBy(stopTask)
} }
@Override
@Internal
TaskProvider<AntFixtureStop> getStopTask() {
return stopTask
}
@Override @Override
protected void runAnt(AntBuilder ant) { protected void runAnt(AntBuilder ant) {
// reset everything // reset everything
@ -231,7 +241,7 @@ class AntFixture extends AntTask implements Fixture {
*/ */
@Internal @Internal
protected File getBaseDir() { protected File getBaseDir() {
return new File(project.buildDir, "fixtures/${name}") return new File(projectLayout.getBuildDirectory().getAsFile().get(), "fixtures/${name}")
} }
/** Returns the working directory for the process. Defaults to "cwd" inside baseDir. */ /** Returns the working directory for the process. Defaults to "cwd" inside baseDir. */
@ -242,7 +252,7 @@ class AntFixture extends AntTask implements Fixture {
/** Returns the file the process writes its pid to. Defaults to "pid" inside baseDir. */ /** Returns the file the process writes its pid to. Defaults to "pid" inside baseDir. */
@Internal @Internal
protected File getPidFile() { File getPidFile() {
return new File(baseDir, 'pid') return new File(baseDir, 'pid')
} }
@ -264,6 +274,12 @@ class AntFixture extends AntTask implements Fixture {
return portsFile.readLines("UTF-8").get(0) return portsFile.readLines("UTF-8").get(0)
} }
@Internal
Provider<String> getAddressAndPortProvider() {
File thePortFile = portsFile
return providerFactory.provider(() -> thePortFile.readLines("UTF-8").get(0))
}
/** Returns a file that wraps around the actual command when {@code spawn == true}. */ /** Returns a file that wraps around the actual command when {@code spawn == true}. */
@Internal @Internal
protected File getWrapperScript() { protected File getWrapperScript() {
@ -281,4 +297,22 @@ class AntFixture extends AntTask implements Fixture {
protected File getRunLog() { protected File getRunLog() {
return new File(cwd, 'run.log') return new File(cwd, 'run.log')
} }
@Internal
Provider<AntFixtureValueSource> getAddressAndPortSource() {
return providerFactory.of(AntFixtureValueSource.class, spec -> {
spec.getParameters().getPortFile().set(portsFile);
});
}
static abstract class AntFixtureValueSource implements ValueSource<String, AntFixtureValueSource.Parameters> {
@Override
String obtain() {
return getParameters().getPortFile().map { it.readLines("UTF-8").get(0) }.get()
}
interface Parameters extends ValueSourceParameters {
Property<File> getPortFile();
}
}
} }

View file

@ -1,21 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/
package org.elasticsearch.gradle.internal.test;
/**
* Any object that can produce an accompanying stop task, meant to tear down
* a previously instantiated service.
*/
public interface Fixture {
/** A task which will stop this fixture. This should be used as a finalizedBy for any tasks that use the fixture. */
Object getStopTask();
}

View file

@ -26,6 +26,7 @@ import org.gradle.api.Task;
import org.gradle.api.artifacts.Configuration; import org.gradle.api.artifacts.Configuration;
import org.gradle.api.artifacts.Dependency; import org.gradle.api.artifacts.Dependency;
import org.gradle.api.file.Directory; import org.gradle.api.file.Directory;
import org.gradle.api.file.FileCollection;
import org.gradle.api.file.ProjectLayout; import org.gradle.api.file.ProjectLayout;
import org.gradle.api.file.RelativePath; import org.gradle.api.file.RelativePath;
import org.gradle.api.internal.file.FileOperations; import org.gradle.api.internal.file.FileOperations;
@ -244,10 +245,11 @@ public abstract class AbstractYamlRestCompatTestPlugin implements Plugin<Project
yamlRestCompatTestTask.configure(testTask -> { yamlRestCompatTestTask.configure(testTask -> {
testTask.systemProperty("tests.restCompat", true); testTask.systemProperty("tests.restCompat", true);
// Use test runner and classpath from "normal" yaml source set // Use test runner and classpath from "normal" yaml source set
FileCollection outputFileCollection = yamlCompatTestSourceSet.getOutput();
testTask.setTestClassesDirs( testTask.setTestClassesDirs(
yamlTestSourceSet.getOutput().getClassesDirs().plus(yamlCompatTestSourceSet.getOutput().getClassesDirs()) yamlTestSourceSet.getOutput().getClassesDirs().plus(yamlCompatTestSourceSet.getOutput().getClassesDirs())
); );
testTask.onlyIf("Compatibility tests are available", t -> yamlCompatTestSourceSet.getOutput().isEmpty() == false); testTask.onlyIf("Compatibility tests are available", t -> outputFileCollection.isEmpty() == false);
testTask.setClasspath( testTask.setClasspath(
yamlCompatTestSourceSet.getRuntimeClasspath() yamlCompatTestSourceSet.getRuntimeClasspath()
// remove the "normal" api and tests // remove the "normal" api and tests

View file

@ -76,6 +76,7 @@ public class ElasticsearchCluster implements TestClusterConfiguration, Named {
private final LinkedHashMap<String, Predicate<TestClusterConfiguration>> waitConditions = new LinkedHashMap<>(); private final LinkedHashMap<String, Predicate<TestClusterConfiguration>> waitConditions = new LinkedHashMap<>();
private final transient Project project; private final transient Project project;
private final Provider<ReaperService> reaper; private final Provider<ReaperService> reaper;
private final Provider<TestClustersRegistry> testClustersRegistryProvider;
private final FileSystemOperations fileSystemOperations; private final FileSystemOperations fileSystemOperations;
private final ArchiveOperations archiveOperations; private final ArchiveOperations archiveOperations;
private final ExecOperations execOperations; private final ExecOperations execOperations;
@ -87,11 +88,14 @@ public class ElasticsearchCluster implements TestClusterConfiguration, Named {
private boolean shared = false; private boolean shared = false;
private int claims = 0;
public ElasticsearchCluster( public ElasticsearchCluster(
String path, String path,
String clusterName, String clusterName,
Project project, Project project,
Provider<ReaperService> reaper, Provider<ReaperService> reaper,
Provider<TestClustersRegistry> testClustersRegistryProvider,
FileSystemOperations fileSystemOperations, FileSystemOperations fileSystemOperations,
ArchiveOperations archiveOperations, ArchiveOperations archiveOperations,
ExecOperations execOperations, ExecOperations execOperations,
@ -104,6 +108,7 @@ public class ElasticsearchCluster implements TestClusterConfiguration, Named {
this.clusterName = clusterName; this.clusterName = clusterName;
this.project = project; this.project = project;
this.reaper = reaper; this.reaper = reaper;
this.testClustersRegistryProvider = testClustersRegistryProvider;
this.fileSystemOperations = fileSystemOperations; this.fileSystemOperations = fileSystemOperations;
this.archiveOperations = archiveOperations; this.archiveOperations = archiveOperations;
this.execOperations = execOperations; this.execOperations = execOperations;
@ -120,6 +125,7 @@ public class ElasticsearchCluster implements TestClusterConfiguration, Named {
clusterName + "-0", clusterName + "-0",
project, project,
reaper, reaper,
testClustersRegistryProvider,
fileSystemOperations, fileSystemOperations,
archiveOperations, archiveOperations,
execOperations, execOperations,
@ -177,6 +183,7 @@ public class ElasticsearchCluster implements TestClusterConfiguration, Named {
clusterName + "-" + i, clusterName + "-" + i,
project, project,
reaper, reaper,
testClustersRegistryProvider,
fileSystemOperations, fileSystemOperations,
archiveOperations, archiveOperations,
execOperations, execOperations,
@ -408,6 +415,7 @@ public class ElasticsearchCluster implements TestClusterConfiguration, Named {
public void freeze() { public void freeze() {
nodes.forEach(ElasticsearchNode::freeze); nodes.forEach(ElasticsearchNode::freeze);
configurationFrozen.set(true); configurationFrozen.set(true);
nodes.whenObjectAdded(node -> { throw new IllegalStateException("Cannot add nodes to test cluster after is has been frozen"); });
} }
private void checkFrozen() { private void checkFrozen() {
@ -663,4 +671,11 @@ public class ElasticsearchCluster implements TestClusterConfiguration, Named {
return "cluster{" + path + ":" + clusterName + "}"; return "cluster{" + path + ":" + clusterName + "}";
} }
int addClaim() {
return ++this.claims;
}
int removeClaim() {
return --this.claims;
}
} }

View file

@ -124,6 +124,8 @@ public class ElasticsearchNode implements TestClusterConfiguration {
private final String name; private final String name;
transient private final Project project; transient private final Project project;
private final Provider<ReaperService> reaperServiceProvider; private final Provider<ReaperService> reaperServiceProvider;
private final Provider<TestClustersRegistry> testClustersRegistryProvider;
private final FileSystemOperations fileSystemOperations; private final FileSystemOperations fileSystemOperations;
private final ArchiveOperations archiveOperations; private final ArchiveOperations archiveOperations;
private final ExecOperations execOperations; private final ExecOperations execOperations;
@ -164,7 +166,6 @@ public class ElasticsearchNode implements TestClusterConfiguration {
private final List<ElasticsearchDistribution> distributions = new ArrayList<>(); private final List<ElasticsearchDistribution> distributions = new ArrayList<>();
private int currentDistro = 0; private int currentDistro = 0;
private TestDistribution testDistribution; private TestDistribution testDistribution;
private volatile Process esProcess;
private Function<String, String> nameCustomization = s -> s; private Function<String, String> nameCustomization = s -> s;
private boolean isWorkingDirConfigured = false; private boolean isWorkingDirConfigured = false;
private String httpPort = "0"; private String httpPort = "0";
@ -179,6 +180,7 @@ public class ElasticsearchNode implements TestClusterConfiguration {
String name, String name,
Project project, Project project,
Provider<ReaperService> reaperServiceProvider, Provider<ReaperService> reaperServiceProvider,
Provider<TestClustersRegistry> testClustersRegistryProvider,
FileSystemOperations fileSystemOperations, FileSystemOperations fileSystemOperations,
ArchiveOperations archiveOperations, ArchiveOperations archiveOperations,
ExecOperations execOperations, ExecOperations execOperations,
@ -191,6 +193,7 @@ public class ElasticsearchNode implements TestClusterConfiguration {
this.name = name; this.name = name;
this.project = project; this.project = project;
this.reaperServiceProvider = reaperServiceProvider; this.reaperServiceProvider = reaperServiceProvider;
this.testClustersRegistryProvider = testClustersRegistryProvider;
this.fileSystemOperations = fileSystemOperations; this.fileSystemOperations = fileSystemOperations;
this.archiveOperations = archiveOperations; this.archiveOperations = archiveOperations;
this.execOperations = execOperations; this.execOperations = execOperations;
@ -892,11 +895,13 @@ public class ElasticsearchNode implements TestClusterConfiguration {
} }
} }
LOGGER.info("Running `{}` in `{}` for {} env: {}", command, workingDir, this, environment); LOGGER.info("Running `{}` in `{}` for {} env: {}", command, workingDir, this, environment);
Process esProcess;
try { try {
esProcess = processBuilder.start(); esProcess = processBuilder.start();
} catch (IOException e) { } catch (IOException e) {
throw new TestClustersException("Failed to start ES process for " + this, e); throw new TestClustersException("Failed to start ES process for " + this, e);
} }
testClustersRegistryProvider.get().storeProcess(id(), esProcess);
reaperServiceProvider.get().registerPid(toString(), esProcess.pid()); reaperServiceProvider.get().registerPid(toString(), esProcess.pid());
} }
@ -982,6 +987,7 @@ public class ElasticsearchNode implements TestClusterConfiguration {
} catch (IOException e) { } catch (IOException e) {
throw new UncheckedIOException(e); throw new UncheckedIOException(e);
} }
Process esProcess = testClustersRegistryProvider.get().getProcess(id());
if (esProcess == null && tailLogs) { if (esProcess == null && tailLogs) {
// This is a special case. If start() throws an exception the plugin will still call stop // This is a special case. If start() throws an exception the plugin will still call stop
// Another exception here would eat the orriginal. // Another exception here would eat the orriginal.
@ -1574,6 +1580,7 @@ public class ElasticsearchNode implements TestClusterConfiguration {
@Override @Override
@Internal @Internal
public boolean isProcessAlive() { public boolean isProcessAlive() {
Process esProcess = testClustersRegistryProvider.get().getProcess(id());
requireNonNull(esProcess, "Can't wait for `" + this + "` as it's not started. Does the task have `useCluster` ?"); requireNonNull(esProcess, "Can't wait for `" + this + "` as it's not started. Does the task have `useCluster` ?");
return esProcess.isAlive(); return esProcess.isAlive();
} }
@ -1602,6 +1609,10 @@ public class ElasticsearchNode implements TestClusterConfiguration {
@Override @Override
public String toString() { public String toString() {
return id() + " (" + System.identityHashCode(this) + ")";
}
private String id() {
return "node{" + path + ":" + name + "}"; return "node{" + path + ":" + name + "}";
} }
@ -1702,7 +1713,7 @@ public class ElasticsearchNode implements TestClusterConfiguration {
} }
} }
private record FeatureFlag(String feature, Version from, Version until) { public record FeatureFlag(String feature, Version from, Version until) {
@Input @Input
public String getFeature() { public String getFeature() {

View file

@ -0,0 +1,36 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/
package org.elasticsearch.gradle.testclusters;
import java.io.File;
import java.util.List;
public class TestClusterInfo {
private final List<String> allHttpSocketURI;
private final List<String> allTransportPortURI;
private final List<File> auditLogs;
public TestClusterInfo(List<String> allHttpSocketURI, List<String> allTransportPortURI, List<File> auditLogs) {
this.allHttpSocketURI = allHttpSocketURI;
this.allTransportPortURI = allTransportPortURI;
this.auditLogs = auditLogs;
}
public List<String> getAllHttpSocketURI() {
return allHttpSocketURI;
}
public List<String> getAllTransportPortURI() {
return allTransportPortURI;
}
public List<File> getAuditLogs() {
return auditLogs;
}
}

View file

@ -0,0 +1,34 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/
package org.elasticsearch.gradle.testclusters;
import org.gradle.api.provider.Property;
import org.gradle.api.provider.ValueSource;
import org.gradle.api.provider.ValueSourceParameters;
import org.jetbrains.annotations.Nullable;
public abstract class TestClusterValueSource implements ValueSource<TestClusterInfo, TestClusterValueSource.Parameters> {
@Nullable
@Override
public TestClusterInfo obtain() {
String clusterName = getParameters().getClusterName().get();
String path = getParameters().getPath().get();
return getParameters().getService().get().getClusterDetails(path, clusterName);
}
interface Parameters extends ValueSourceParameters {
Property<String> getClusterName();
Property<String> getPath();
Property<TestClustersRegistry> getService();
}
}

View file

@ -8,6 +8,7 @@
*/ */
package org.elasticsearch.gradle.testclusters; package org.elasticsearch.gradle.testclusters;
import org.elasticsearch.gradle.ElasticsearchDistribution;
import org.gradle.api.Task; import org.gradle.api.Task;
import org.gradle.api.provider.Property; import org.gradle.api.provider.Property;
import org.gradle.api.provider.Provider; import org.gradle.api.provider.Provider;
@ -34,10 +35,15 @@ public interface TestClustersAware extends Task {
if (cluster.getPath().equals(getProject().getPath()) == false) { if (cluster.getPath().equals(getProject().getPath()) == false) {
throw new TestClustersException("Task " + getPath() + " can't use test cluster from" + " another project " + cluster); throw new TestClustersException("Task " + getPath() + " can't use test cluster from" + " another project " + cluster);
} }
if (cluster.getName().equals(getName())) {
cluster.getNodes() for (ElasticsearchNode node : cluster.getNodes()) {
.all(node -> node.getDistributions().forEach(distro -> dependsOn(getProject().provider(() -> distro.maybeFreeze())))); for (ElasticsearchDistribution distro : node.getDistributions()) {
dependsOn(cluster.getPluginAndModuleConfigurations()); ElasticsearchDistribution frozenDistro = distro.maybeFreeze();
dependsOn(frozenDistro);
}
}
dependsOn(cluster.getPluginAndModuleConfigurations());
}
getClusters().add(cluster); getClusters().add(cluster);
} }

View file

@ -26,6 +26,7 @@ import org.gradle.api.internal.file.FileOperations;
import org.gradle.api.invocation.Gradle; import org.gradle.api.invocation.Gradle;
import org.gradle.api.logging.Logger; import org.gradle.api.logging.Logger;
import org.gradle.api.logging.Logging; import org.gradle.api.logging.Logging;
import org.gradle.api.provider.Property;
import org.gradle.api.provider.Provider; import org.gradle.api.provider.Provider;
import org.gradle.api.provider.ProviderFactory; import org.gradle.api.provider.ProviderFactory;
import org.gradle.api.services.BuildService; import org.gradle.api.services.BuildService;
@ -106,15 +107,22 @@ public class TestClustersPlugin implements Plugin<Project> {
runtimeJavaProvider = providerFactory.provider( runtimeJavaProvider = providerFactory.provider(
() -> System.getenv("RUNTIME_JAVA_HOME") == null ? Jvm.current().getJavaHome() : new File(System.getenv("RUNTIME_JAVA_HOME")) () -> System.getenv("RUNTIME_JAVA_HOME") == null ? Jvm.current().getJavaHome() : new File(System.getenv("RUNTIME_JAVA_HOME"))
); );
// register cluster registry as a global build service
Provider<TestClustersRegistry> testClustersRegistryProvider = project.getGradle()
.getSharedServices()
.registerIfAbsent(REGISTRY_SERVICE_NAME, TestClustersRegistry.class, noop());
// enable the DSL to describe clusters // enable the DSL to describe clusters
NamedDomainObjectContainer<ElasticsearchCluster> container = createTestClustersContainerExtension(project, reaperServiceProvider); NamedDomainObjectContainer<ElasticsearchCluster> container = createTestClustersContainerExtension(
project,
testClustersRegistryProvider,
reaperServiceProvider
);
// provide a task to be able to list defined clusters. // provide a task to be able to list defined clusters.
createListClustersTask(project, container); createListClustersTask(project, container);
// register cluster registry as a global build service
project.getGradle().getSharedServices().registerIfAbsent(REGISTRY_SERVICE_NAME, TestClustersRegistry.class, noop());
// register throttle so we only run at most max-workers/2 nodes concurrently // register throttle so we only run at most max-workers/2 nodes concurrently
Provider<TestClustersThrottle> testClustersThrottleProvider = project.getGradle() Provider<TestClustersThrottle> testClustersThrottleProvider = project.getGradle()
.getSharedServices() .getSharedServices()
@ -145,6 +153,7 @@ public class TestClustersPlugin implements Plugin<Project> {
private NamedDomainObjectContainer<ElasticsearchCluster> createTestClustersContainerExtension( private NamedDomainObjectContainer<ElasticsearchCluster> createTestClustersContainerExtension(
Project project, Project project,
Provider<TestClustersRegistry> testClustersRegistryProvider,
Provider<ReaperService> reaper Provider<ReaperService> reaper
) { ) {
// Create an extensions that allows describing clusters // Create an extensions that allows describing clusters
@ -155,6 +164,7 @@ public class TestClustersPlugin implements Plugin<Project> {
name, name,
project, project,
reaper, reaper,
testClustersRegistryProvider,
getFileSystemOperations(), getFileSystemOperations(),
getArchiveOperations(), getArchiveOperations(),
getExecOperations(), getExecOperations(),
@ -199,7 +209,9 @@ public class TestClustersPlugin implements Plugin<Project> {
Provider<TaskEventsService> testClusterTasksService = project.getGradle() Provider<TaskEventsService> testClusterTasksService = project.getGradle()
.getSharedServices() .getSharedServices()
.registerIfAbsent(TEST_CLUSTER_TASKS_SERVICE, TaskEventsService.class, spec -> {}); .registerIfAbsent(TEST_CLUSTER_TASKS_SERVICE, TaskEventsService.class, spec -> {
spec.getParameters().getRegistry().set(registryProvider);
});
TestClustersRegistry registry = registryProvider.get(); TestClustersRegistry registry = registryProvider.get();
// When we know what tasks will run, we claim the clusters of those task to differentiate between clusters // When we know what tasks will run, we claim the clusters of those task to differentiate between clusters
@ -209,7 +221,7 @@ public class TestClustersPlugin implements Plugin<Project> {
configureClaimClustersHook(project.getGradle(), registry); configureClaimClustersHook(project.getGradle(), registry);
// Before each task, we determine if a cluster needs to be started for that task. // Before each task, we determine if a cluster needs to be started for that task.
configureStartClustersHook(project.getGradle(), registry, testClusterTasksService); configureStartClustersHook(project.getGradle());
// After each task we determine if there are clusters that are no longer needed. // After each task we determine if there are clusters that are no longer needed.
getEventsListenerRegistry().onTaskCompletion(testClusterTasksService); getEventsListenerRegistry().onTaskCompletion(testClusterTasksService);
@ -228,12 +240,7 @@ public class TestClustersPlugin implements Plugin<Project> {
}); });
} }
private void configureStartClustersHook( private void configureStartClustersHook(Gradle gradle) {
Gradle gradle,
TestClustersRegistry registry,
Provider<TaskEventsService> testClusterTasksService
) {
testClusterTasksService.get().registry(registry);
gradle.getTaskGraph().whenReady(taskExecutionGraph -> { gradle.getTaskGraph().whenReady(taskExecutionGraph -> {
taskExecutionGraph.getAllTasks() taskExecutionGraph.getAllTasks()
.stream() .stream()
@ -249,19 +256,14 @@ public class TestClustersPlugin implements Plugin<Project> {
} }
} }
static public abstract class TaskEventsService implements BuildService<BuildServiceParameters.None>, OperationCompletionListener { static public abstract class TaskEventsService implements BuildService<TaskEventsService.Params>, OperationCompletionListener {
Map<String, TestClustersAware> tasksMap = new HashMap<>(); Map<String, TestClustersAware> tasksMap = new HashMap<>();
private TestClustersRegistry registryProvider;
public void register(TestClustersAware task) { public void register(TestClustersAware task) {
tasksMap.put(task.getPath(), task); tasksMap.put(task.getPath(), task);
} }
public void registry(TestClustersRegistry registry) {
this.registryProvider = registry;
}
@Override @Override
public void onFinish(FinishEvent finishEvent) { public void onFinish(FinishEvent finishEvent) {
if (finishEvent instanceof TaskFinishEvent taskFinishEvent) { if (finishEvent instanceof TaskFinishEvent taskFinishEvent) {
@ -273,11 +275,18 @@ public class TestClustersPlugin implements Plugin<Project> {
if (task.getDidWork()) { if (task.getDidWork()) {
task.getClusters() task.getClusters()
.forEach( .forEach(
cluster -> registryProvider.stopCluster(cluster, taskFinishEvent.getResult() instanceof TaskFailureResult) cluster -> getParameters().getRegistry()
.get()
.stopCluster(cluster, taskFinishEvent.getResult() instanceof TaskFailureResult)
); );
} }
} }
} }
} }
// Some parameters for the web server
interface Params extends BuildServiceParameters {
Property<TestClustersRegistry> getRegistry();
}
} }
} }

View file

@ -10,6 +10,8 @@ package org.elasticsearch.gradle.testclusters;
import org.gradle.api.logging.Logger; import org.gradle.api.logging.Logger;
import org.gradle.api.logging.Logging; import org.gradle.api.logging.Logging;
import org.gradle.api.provider.Provider;
import org.gradle.api.provider.ProviderFactory;
import org.gradle.api.services.BuildService; import org.gradle.api.services.BuildService;
import org.gradle.api.services.BuildServiceParameters; import org.gradle.api.services.BuildServiceParameters;
@ -17,20 +19,23 @@ import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors;
import javax.inject.Inject;
public abstract class TestClustersRegistry implements BuildService<BuildServiceParameters.None> { public abstract class TestClustersRegistry implements BuildService<BuildServiceParameters.None> {
private static final Logger logger = Logging.getLogger(TestClustersRegistry.class); private static final Logger logger = Logging.getLogger(TestClustersRegistry.class);
private static final String TESTCLUSTERS_INSPECT_FAILURE = "testclusters.inspect.failure"; private static final String TESTCLUSTERS_INSPECT_FAILURE = "testclusters.inspect.failure";
private final Boolean allowClusterToSurvive = Boolean.valueOf(System.getProperty(TESTCLUSTERS_INSPECT_FAILURE, "false")); private final Boolean allowClusterToSurvive = Boolean.valueOf(System.getProperty(TESTCLUSTERS_INSPECT_FAILURE, "false"));
private final Map<ElasticsearchCluster, Integer> claimsInventory = new HashMap<>();
private final Set<ElasticsearchCluster> runningClusters = new HashSet<>(); private final Set<ElasticsearchCluster> runningClusters = new HashSet<>();
private final Map<String, Process> nodeProcesses = new HashMap<>();
@Inject
public abstract ProviderFactory getProviderFactory();
public void claimCluster(ElasticsearchCluster cluster) { public void claimCluster(ElasticsearchCluster cluster) {
cluster.freeze(); int claims = cluster.addClaim();
int claim = claimsInventory.getOrDefault(cluster, 0) + 1; if (claims > 1) {
claimsInventory.put(cluster, claim);
if (claim > 1) {
cluster.setShared(true); cluster.setShared(true);
} }
} }
@ -43,6 +48,13 @@ public abstract class TestClustersRegistry implements BuildService<BuildServiceP
cluster.start(); cluster.start();
} }
public Provider<TestClusterInfo> getClusterInfo(String clusterName) {
return getProviderFactory().of(TestClusterValueSource.class, spec -> {
spec.getParameters().getService().set(TestClustersRegistry.this);
spec.getParameters().getClusterName().set(clusterName);
});
}
public void stopCluster(ElasticsearchCluster cluster, boolean taskFailed) { public void stopCluster(ElasticsearchCluster cluster, boolean taskFailed) {
if (taskFailed) { if (taskFailed) {
// If the task fails, and other tasks use this cluster, the other task will likely never be // If the task fails, and other tasks use this cluster, the other task will likely never be
@ -67,8 +79,7 @@ public abstract class TestClustersRegistry implements BuildService<BuildServiceP
runningClusters.remove(cluster); runningClusters.remove(cluster);
} }
} else { } else {
int currentClaims = claimsInventory.getOrDefault(cluster, 0) - 1; int currentClaims = cluster.removeClaim();
claimsInventory.put(cluster, currentClaims);
if (currentClaims <= 0 && runningClusters.contains(cluster)) { if (currentClaims <= 0 && runningClusters.contains(cluster)) {
cluster.stop(false); cluster.stop(false);
runningClusters.remove(cluster); runningClusters.remove(cluster);
@ -76,4 +87,33 @@ public abstract class TestClustersRegistry implements BuildService<BuildServiceP
} }
} }
public TestClusterInfo getClusterDetails(String path, String clusterName) {
ElasticsearchCluster cluster = runningClusters.stream()
.filter(c -> c.getPath().equals(path))
.filter(c -> c.getName().equals(clusterName))
.findFirst()
.orElseThrow();
return new TestClusterInfo(
cluster.getAllHttpSocketURI(),
cluster.getAllTransportPortURI(),
cluster.getNodes().stream().map(n -> n.getAuditLog()).collect(Collectors.toList())
);
}
public void restart(String path, String clusterName) {
ElasticsearchCluster cluster = runningClusters.stream()
.filter(c -> c.getPath().equals(path))
.filter(c -> c.getName().equals(clusterName))
.findFirst()
.orElseThrow();
cluster.restart();
}
public void storeProcess(String id, Process esProcess) {
nodeProcesses.put(id, esProcess);
}
public Process getProcess(String id) {
return nodeProcesses.get(id);
}
} }

View file

@ -8,7 +8,6 @@
*/ */
import org.apache.tools.ant.filters.ReplaceTokens import org.apache.tools.ant.filters.ReplaceTokens
import org.elasticsearch.gradle.internal.info.BuildParams
import org.elasticsearch.gradle.internal.test.AntFixture import org.elasticsearch.gradle.internal.test.AntFixture
import org.elasticsearch.gradle.internal.test.RestIntegTestTask import org.elasticsearch.gradle.internal.test.RestIntegTestTask
import org.elasticsearch.gradle.internal.test.rest.LegacyYamlRestTestPlugin import org.elasticsearch.gradle.internal.test.rest.LegacyYamlRestTestPlugin
@ -55,8 +54,9 @@ tasks.named("yamlRestTest").configure { enabled = false }
['KeyStore', 'EnvVariables', 'SystemProperties', 'ContainerCredentials', 'InstanceProfile'].forEach { action -> ['KeyStore', 'EnvVariables', 'SystemProperties', 'ContainerCredentials', 'InstanceProfile'].forEach { action ->
TaskProvider<AntFixture> fixture = tasks.register("ec2Fixture${action}", AntFixture) { TaskProvider<AntFixture> fixture = tasks.register("ec2Fixture${action}", AntFixture) {
dependsOn project.sourceSets.yamlRestTest.runtimeClasspath dependsOn project.sourceSets.yamlRestTest.runtimeClasspath
env 'CLASSPATH', "${-> project.sourceSets.yamlRestTest.runtimeClasspath.asPath}" FileCollection cp = project.sourceSets.yamlRestTest.runtimeClasspath
executable = "${buildParams.runtimeJavaHome.get()}/bin/java" env 'CLASSPATH', "${-> cp.asPath}"
executable = "${buildParams.runtimeJavaHome.get() }/bin/java"
args 'org.elasticsearch.discovery.ec2.AmazonEC2Fixture', baseDir, "${buildDir}/testclusters/yamlRestTest${action}-1/config/unicast_hosts.txt" args 'org.elasticsearch.discovery.ec2.AmazonEC2Fixture', baseDir, "${buildDir}/testclusters/yamlRestTest${action}-1/config/unicast_hosts.txt"
} }
@ -68,9 +68,18 @@ tasks.named("yamlRestTest").configure { enabled = false }
classpath = yamlRestTestSourceSet.getRuntimeClasspath() classpath = yamlRestTestSourceSet.getRuntimeClasspath()
} }
if(action == 'ContainerCredentials') {
def addressAndPortSource = fixture.get().addressAndPortSource
testClusters.matching { it.name == "yamlRestTestContainerCredentials" }.configureEach {
environment 'AWS_CONTAINER_CREDENTIALS_FULL_URI',
() -> addressAndPortSource.map{ addr -> "http://${addr}/ecs_credentials_endpoint" }.get(), IGNORE_VALUE
}
}
tasks.named("check").configure { tasks.named("check").configure {
dependsOn(yamlRestTestTask) dependsOn(yamlRestTestTask)
} }
def addressAndPortSource = fixture.get().addressAndPortSource
testClusters.matching { it.name == yamlRestTestTask.name}.configureEach { testClusters.matching { it.name == yamlRestTestTask.name}.configureEach {
numberOfNodes = ec2NumberOfNodes numberOfNodes = ec2NumberOfNodes
@ -78,9 +87,9 @@ tasks.named("yamlRestTest").configure { enabled = false }
setting 'discovery.seed_providers', 'ec2' setting 'discovery.seed_providers', 'ec2'
setting 'network.host', '_ec2_' setting 'network.host', '_ec2_'
setting 'discovery.ec2.endpoint', { "http://${-> fixture.get().addressAndPort}" }, IGNORE_VALUE setting 'discovery.ec2.endpoint', { "http://${-> addressAndPortSource.get()}" }, IGNORE_VALUE
systemProperty "com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", { "http://${-> fixture.get().addressAndPort}" }, IGNORE_VALUE systemProperty "com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", { "http://${-> addressAndPortSource.get()}" }, IGNORE_VALUE
} }
} }
@ -107,11 +116,6 @@ tasks.named("ec2FixtureContainerCredentials").configure {
env 'ACTIVATE_CONTAINER_CREDENTIALS', true env 'ACTIVATE_CONTAINER_CREDENTIALS', true
} }
testClusters.matching { it.name == "yamlRestTestContainerCredentials" }.configureEach {
environment 'AWS_CONTAINER_CREDENTIALS_FULL_URI',
{ "http://${-> tasks.findByName("ec2FixtureContainerCredentials").addressAndPort}/ecs_credentials_endpoint" }, IGNORE_VALUE
}
// Extra config for InstanceProfile // Extra config for InstanceProfile
tasks.named("ec2FixtureInstanceProfile").configure { tasks.named("ec2FixtureInstanceProfile").configure {
env 'ACTIVATE_INSTANCE_PROFILE', true env 'ACTIVATE_INSTANCE_PROFILE', true

View file

@ -11,6 +11,10 @@
import org.elasticsearch.gradle.Version import org.elasticsearch.gradle.Version
import org.elasticsearch.gradle.VersionProperties import org.elasticsearch.gradle.VersionProperties
import org.elasticsearch.gradle.testclusters.StandaloneRestIntegTestTask import org.elasticsearch.gradle.testclusters.StandaloneRestIntegTestTask
import org.elasticsearch.gradle.testclusters.TestClusterValueSource
import org.elasticsearch.gradle.testclusters.TestClustersRegistry
import org.elasticsearch.gradle.util.GradleUtils
import org.elasticsearch.gradle.testclusters.TestClustersPlugin
apply plugin: 'elasticsearch.internal-testclusters' apply plugin: 'elasticsearch.internal-testclusters'
apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.standalone-rest-test'
@ -63,6 +67,8 @@ excludeList.add('indices.resolve_index/20_resolve_system_index/*')
// Excluded because the error has changed // Excluded because the error has changed
excludeList.add('aggregations/percentiles_hdr_metric/Negative values test') excludeList.add('aggregations/percentiles_hdr_metric/Negative values test')
def clusterPath = getPath()
buildParams.bwcVersions.withWireCompatible { bwcVersion, baseName -> buildParams.bwcVersions.withWireCompatible { bwcVersion, baseName ->
if (bwcVersion != VersionProperties.getElasticsearchVersion()) { if (bwcVersion != VersionProperties.getElasticsearchVersion()) {
/* This project runs the core REST tests against a 4 node cluster where two of /* This project runs the core REST tests against a 4 node cluster where two of
@ -84,18 +90,42 @@ buildParams.bwcVersions.withWireCompatible { bwcVersion, baseName ->
tasks.register("${baseName}#mixedClusterTest", StandaloneRestIntegTestTask) { tasks.register("${baseName}#mixedClusterTest", StandaloneRestIntegTestTask) {
useCluster baseCluster useCluster baseCluster
mustRunAfter("precommit") mustRunAfter("precommit")
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def baseInfo = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set(baseName)
it.parameters.service = serviceProvider
}.map { it.getAllHttpSocketURI() }
def baseInfoAfterOneNodeUpdate = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set(baseName)
it.parameters.service = serviceProvider
}.map { it.getAllHttpSocketURI() }
def baseInfoAfterTwoNodesUpdate = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set(baseName)
it.parameters.service = serviceProvider
}.map { it.getAllHttpSocketURI() }
def nonInputProps = nonInputProperties
def sharedRepoFolder = new File(buildDir, "cluster/shared/repo/${baseName}")
doFirst { doFirst {
delete("${buildDir}/cluster/shared/repo/${baseName}") delete(sharedRepoFolder)
// Getting the endpoints causes a wait for the cluster // Getting the endpoints causes a wait for the cluster
println "Test cluster endpoints are: ${-> baseCluster.get().allHttpSocketURI.join(",")}" println "Test cluster endpoints are: ${-> baseInfo.get().join(",")}"
println "Upgrading one node to create a mixed cluster" println "Upgrading one node to create a mixed cluster"
baseCluster.get().nextNodeToNextVersion() baseCluster.get().nextNodeToNextVersion()
// Getting the endpoints causes a wait for the cluster // Getting the endpoints causes a wait for the cluster
println "Upgrade complete, endpoints are: ${-> baseCluster.get().allHttpSocketURI.join(",")}" println "Upgrade complete, endpoints are: ${-> baseInfoAfterOneNodeUpdate.get().join(",")}"
println "Upgrading another node to create a mixed cluster" println "Upgrading another node to create a mixed cluster"
baseCluster.get().nextNodeToNextVersion() baseCluster.get().nextNodeToNextVersion()
nonInputProperties.systemProperty('tests.rest.cluster', baseCluster.map(c -> c.allHttpSocketURI.join(","))) nonInputProps.systemProperty('tests.rest.cluster', baseInfoAfterTwoNodesUpdate.map(c -> c.join(",")))
nonInputProperties.systemProperty('tests.clustername', baseName) nonInputProps.systemProperty('tests.clustername', baseName)
if (excludeList.isEmpty() == false) { if (excludeList.isEmpty() == false) {
systemProperty 'tests.rest.blacklist', excludeList.join(',') systemProperty 'tests.rest.blacklist', excludeList.join(',')
} }
@ -103,7 +133,7 @@ buildParams.bwcVersions.withWireCompatible { bwcVersion, baseName ->
systemProperty 'tests.path.repo', "${buildDir}/cluster/shared/repo/${baseName}" systemProperty 'tests.path.repo', "${buildDir}/cluster/shared/repo/${baseName}"
systemProperty 'tests.bwc_nodes_version', bwcVersion.toString().replace('-SNAPSHOT', '') systemProperty 'tests.bwc_nodes_version', bwcVersion.toString().replace('-SNAPSHOT', '')
systemProperty 'tests.new_nodes_version', project.version.toString().replace('-SNAPSHOT', '') systemProperty 'tests.new_nodes_version', project.version.toString().replace('-SNAPSHOT', '')
onlyIf("BWC tests disabled") { project.bwc_tests_enabled } // onlyIf("BWC tests disabled") { project.bwc_tests_enabled }
} }
tasks.register(bwcTaskName(bwcVersion)) { tasks.register(bwcTaskName(bwcVersion)) {

View file

@ -1,5 +1,9 @@
import org.elasticsearch.gradle.internal.info.BuildParams
import org.elasticsearch.gradle.internal.test.RestIntegTestTask import org.elasticsearch.gradle.internal.test.RestIntegTestTask
import org.elasticsearch.gradle.testclusters.TestClusterValueSource
import org.elasticsearch.gradle.testclusters.TestClustersPlugin
import org.elasticsearch.gradle.testclusters.TestClustersRegistry
import org.elasticsearch.gradle.util.GradleUtils
import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE
apply plugin: 'elasticsearch.internal-testclusters' apply plugin: 'elasticsearch.internal-testclusters'
@ -11,6 +15,8 @@ dependencies {
testImplementation project(':x-pack:plugin:ccr:qa') testImplementation project(':x-pack:plugin:ccr:qa')
} }
def clusterPath = getPath()
def leaderCluster = testClusters.register("leader-cluster") { def leaderCluster = testClusters.register("leader-cluster") {
testDistribution = 'DEFAULT' testDistribution = 'DEFAULT'
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
@ -24,7 +30,19 @@ def followCluster = testClusters.register("follow-cluster") {
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
setting 'xpack.security.enabled', 'true' setting 'xpack.security.enabled', 'true'
user username: 'admin', password: 'admin-password', role: 'superuser' user username: 'admin', password: 'admin-password', role: 'superuser'
setting 'cluster.remote.leader_cluster.seeds', { "\"${leaderCluster.get().getAllTransportPortURI().join(",")}\"" }, IGNORE_VALUE Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderInfo = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}
def leaderUris = leaderInfo.map { it.getAllTransportPortURI() }
setting 'cluster.remote.leader_cluster.seeds',
{ "\"${leaderUris.get().join(",")}\"" }, IGNORE_VALUE
} }
tasks.register("leader-cluster", RestIntegTestTask) { tasks.register("leader-cluster", RestIntegTestTask) {
@ -41,7 +59,7 @@ tasks.register("writeJavaPolicy") {
policyFile.write( policyFile.write(
[ [
"grant {", "grant {",
" permission java.io.FilePermission \"${-> testClusters."follow-cluster".getFirstNode().getServerLog()}\", \"read\";", " permission java.io.FilePermission \"${-> followCluster.map { it.getFirstNode().getServerLog() }.get()}\", \"read\";",
"};" "};"
].join("\n") ].join("\n")
) )
@ -50,11 +68,28 @@ tasks.register("writeJavaPolicy") {
tasks.register("follow-cluster", RestIntegTestTask) { tasks.register("follow-cluster", RestIntegTestTask) {
dependsOn 'writeJavaPolicy', "leader-cluster" dependsOn 'writeJavaPolicy', "leader-cluster"
useCluster leaderCluster useCluster leaderCluster
systemProperty 'tests.target_cluster', 'follow' systemProperty 'tests.target_cluster', 'follow'
nonInputProperties.systemProperty 'java.security.policy', "file://${policyFile}" nonInputProperties.systemProperty 'java.security.policy', "file://${policyFile}"
nonInputProperties.systemProperty 'tests.leader_host', leaderCluster.map(c -> c.allHttpSocketURI.get(0)) Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
nonInputProperties.systemProperty 'log', followCluster.map(c -> c.getFirstNode().getServerLog()) project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderInfo = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}
def followInfo = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("follow-cluster")
it.parameters.service = serviceProvider
}
def leaderUri = leaderInfo.map { it.getAllHttpSocketURI().get(0) }
def followerUri = followInfo.map { it.getAllHttpSocketURI().get(0) }
nonInputProperties.systemProperty 'tests.leader_host', leaderUri
nonInputProperties.systemProperty 'log', followCluster.map(c -> c.getFirstNode().getServerLog())
} }
tasks.named("check").configure { dependsOn "follow-cluster" } tasks.named("check").configure { dependsOn "follow-cluster" }

View file

@ -1,6 +1,10 @@
import org.elasticsearch.gradle.Version import org.elasticsearch.gradle.Version
import org.elasticsearch.gradle.internal.info.BuildParams
import org.elasticsearch.gradle.internal.test.RestIntegTestTask import org.elasticsearch.gradle.internal.test.RestIntegTestTask
import org.elasticsearch.gradle.testclusters.TestClusterValueSource
import org.elasticsearch.gradle.testclusters.TestClustersPlugin
import org.elasticsearch.gradle.testclusters.TestClustersRegistry
import org.elasticsearch.gradle.util.GradleUtils
import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE
apply plugin: 'elasticsearch.internal-testclusters' apply plugin: 'elasticsearch.internal-testclusters'
@ -12,6 +16,7 @@ dependencies {
testImplementation project(':x-pack:plugin:ccr:qa') testImplementation project(':x-pack:plugin:ccr:qa')
} }
def clusterPath = getPath()
def leaderCluster = testClusters.register('leader-cluster') { def leaderCluster = testClusters.register('leader-cluster') {
testDistribution = 'DEFAULT' testDistribution = 'DEFAULT'
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
@ -21,12 +26,23 @@ def leaderCluster = testClusters.register('leader-cluster') {
} }
def middleCluster = testClusters.register('middle-cluster') { def middleCluster = testClusters.register('middle-cluster') {
testDistribution = 'DEFAULT' testDistribution = 'DEFAULT'
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
setting 'xpack.security.enabled', 'true' setting 'xpack.security.enabled', 'true'
user username: 'admin', password: 'admin-password', role: 'superuser' user username: 'admin', password: 'admin-password', role: 'superuser'
setting 'cluster.remote.leader_cluster.seeds',
{ "\"${leaderCluster.get().getAllTransportPortURI().join(",")}\"" }, IGNORE_VALUE Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderInfo = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}
def leaderUris = leaderInfo.map { it.getAllTransportPortURI() }
setting 'cluster.remote.leader_cluster.seeds',
{ "\"${leaderUris.get().join(",")}\"" }, IGNORE_VALUE
} }
tasks.register("leader-cluster", RestIntegTestTask) { tasks.register("leader-cluster", RestIntegTestTask) {
@ -40,30 +56,74 @@ tasks.register("middle-cluster", RestIntegTestTask) {
useCluster testClusters.named("leader-cluster") useCluster testClusters.named("leader-cluster")
systemProperty 'tests.target_cluster', 'middle' systemProperty 'tests.target_cluster', 'middle'
systemProperty 'tests.leader_cluster_repository_path', "${buildDir}/cluster/shared/repo/leader-cluster" systemProperty 'tests.leader_cluster_repository_path', "${buildDir}/cluster/shared/repo/leader-cluster"
nonInputProperties.systemProperty 'tests.leader_host',leaderCluster.map(c -> c.allHttpSocketURI.get(0))
}
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderUri = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}.map { it.allHttpSocketURI.get(0) }
nonInputProperties.systemProperty 'tests.leader_host', leaderUri
}
tasks.register('follow-cluster', RestIntegTestTask) { tasks.register('follow-cluster', RestIntegTestTask) {
dependsOn "leader-cluster", "middle-cluster" dependsOn "leader-cluster", "middle-cluster"
useCluster leaderCluster useCluster leaderCluster
useCluster middleCluster useCluster middleCluster
systemProperty 'tests.target_cluster', 'follow' systemProperty 'tests.target_cluster', 'follow'
systemProperty 'tests.leader_cluster_repository_path', "${buildDir}/cluster/shared/repo/leader-cluster" systemProperty 'tests.leader_cluster_repository_path', "${buildDir}/cluster/shared/repo/leader-cluster"
nonInputProperties.systemProperty 'tests.leader_host', leaderCluster.map(c -> c.allHttpSocketURI.get(0))
nonInputProperties.systemProperty 'tests.middle_host', middleCluster.map(c -> c.allHttpSocketURI.get(0)) Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderUri = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}.map { it.allHttpSocketURI.get(0) }
def middleUri = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("middle-cluster")
it.parameters.service = serviceProvider
}.map { it.allHttpSocketURI.get(0) }
nonInputProperties.systemProperty 'tests.leader_host', leaderUri
nonInputProperties.systemProperty 'tests.middle_host', middleUri
} }
testClusters.matching {it.name == "follow-cluster" }.configureEach { testClusters.matching { it.name == "follow-cluster" }.configureEach {
testDistribution = 'DEFAULT' testDistribution = 'DEFAULT'
setting 'xpack.monitoring.collection.enabled', 'true' setting 'xpack.monitoring.collection.enabled', 'true'
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
setting 'xpack.security.enabled', 'true' setting 'xpack.security.enabled', 'true'
user username: 'admin', password: 'admin-password', role: 'superuser' user username: 'admin', password: 'admin-password', role: 'superuser'
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderUris = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}.map { it.getAllTransportPortURI() }
def middleUris = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("middle-cluster")
it.parameters.service = serviceProvider
}.map { it.getAllTransportPortURI() }
setting 'cluster.remote.leader_cluster.seeds', setting 'cluster.remote.leader_cluster.seeds',
{ "\"${leaderCluster.get().getAllTransportPortURI().join(",")}\"" }, IGNORE_VALUE { "\"${leaderUris.get().join(",")}\"" }, IGNORE_VALUE
setting 'cluster.remote.middle_cluster.seeds', setting 'cluster.remote.middle_cluster.seeds',
{ "\"${middleCluster.get().getAllTransportPortURI().join(",")}\"" }, IGNORE_VALUE { "\"${middleUris.get().join(",")}\"" }, IGNORE_VALUE
} }

View file

@ -1,5 +1,9 @@
import org.elasticsearch.gradle.internal.test.RestIntegTestTask import org.elasticsearch.gradle.internal.test.RestIntegTestTask
import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE
import org.elasticsearch.gradle.testclusters.TestClusterValueSource
import org.elasticsearch.gradle.testclusters.TestClustersPlugin
import org.elasticsearch.gradle.testclusters.TestClustersRegistry
import org.elasticsearch.gradle.util.GradleUtils
apply plugin: 'elasticsearch.internal-testclusters' apply plugin: 'elasticsearch.internal-testclusters'
apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.standalone-rest-test'
@ -10,6 +14,8 @@ dependencies {
testImplementation project(':x-pack:plugin:ccr:qa:') testImplementation project(':x-pack:plugin:ccr:qa:')
} }
def clusterPath = getPath()
def leaderCluster = testClusters.register('leader-cluster') { def leaderCluster = testClusters.register('leader-cluster') {
testDistribution = 'DEFAULT' testDistribution = 'DEFAULT'
setting 'xpack.security.enabled', 'true' setting 'xpack.security.enabled', 'true'
@ -21,8 +27,20 @@ def followerCluster = testClusters.register('follow-cluster') {
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
setting 'xpack.security.enabled', 'true' setting 'xpack.security.enabled', 'true'
user username: 'admin', password: 'admin-password', role: 'superuser' user username: 'admin', password: 'admin-password', role: 'superuser'
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderInfo = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}
def leaderUris = leaderInfo.map { it.getAllTransportPortURI() }
setting 'cluster.remote.leader_cluster.seeds', setting 'cluster.remote.leader_cluster.seeds',
{ "\"${leaderCluster.get().getAllTransportPortURI().join(",")}\"" }, IGNORE_VALUE { "\"${leaderUris.get().join(",")}\"" }, IGNORE_VALUE
} }
tasks.register('leader-cluster', RestIntegTestTask) { tasks.register('leader-cluster', RestIntegTestTask) {
@ -34,7 +52,19 @@ tasks.register('follow-cluster', RestIntegTestTask) {
dependsOn 'leader-cluster' dependsOn 'leader-cluster'
useCluster leaderCluster useCluster leaderCluster
systemProperty 'tests.target_cluster', 'follow' systemProperty 'tests.target_cluster', 'follow'
nonInputProperties.systemProperty 'tests.leader_host', followerCluster.map(c -> c.allHttpSocketURI.get(0))
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def followInfo = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("follow-cluster")
it.parameters.service = serviceProvider
}
def followUri = followInfo.map { it.allHttpSocketURI.get(0) }
nonInputProperties.systemProperty 'tests.leader_host', followUri
} }
tasks.named("check").configure { dependsOn "follow-cluster" } tasks.named("check").configure { dependsOn "follow-cluster" }

View file

@ -1,6 +1,10 @@
import org.elasticsearch.gradle.internal.test.RestIntegTestTask import org.elasticsearch.gradle.internal.test.RestIntegTestTask
import org.elasticsearch.gradle.testclusters.StandaloneRestIntegTestTask import org.elasticsearch.gradle.testclusters.StandaloneRestIntegTestTask
import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE
import org.elasticsearch.gradle.testclusters.TestClusterValueSource
import org.elasticsearch.gradle.testclusters.TestClustersPlugin
import org.elasticsearch.gradle.testclusters.TestClustersRegistry
import org.elasticsearch.gradle.util.GradleUtils
apply plugin: 'elasticsearch.internal-testclusters' apply plugin: 'elasticsearch.internal-testclusters'
apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.standalone-rest-test'
@ -9,6 +13,8 @@ dependencies {
testImplementation project(':x-pack:plugin:ccr:qa') testImplementation project(':x-pack:plugin:ccr:qa')
} }
def clusterPath = getPath()
def leaderCluster = testClusters.register('leader-cluster') { def leaderCluster = testClusters.register('leader-cluster') {
testDistribution = 'DEFAULT' testDistribution = 'DEFAULT'
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
@ -22,12 +28,23 @@ def followCluster = testClusters.register('follow-cluster') {
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
setting 'xpack.security.enabled', 'true' setting 'xpack.security.enabled', 'true'
user username: 'admin', password: 'admin-password', role: 'superuser' user username: 'admin', password: 'admin-password', role: 'superuser'
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderInfo = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}
def leaderUri = leaderInfo.map { it.getAllTransportPortURI().get(0) }
setting 'cluster.remote.leader_cluster.seeds', setting 'cluster.remote.leader_cluster.seeds',
{ "\"${leaderCluster.get().getAllTransportPortURI().get(0)}\"" }, IGNORE_VALUE { "\"${leaderUri.get()}\"" }, IGNORE_VALUE
nameCustomization = { 'follow' } nameCustomization = { 'follow' }
} }
tasks.register('leader-cluster', RestIntegTestTask) { tasks.register('leader-cluster', RestIntegTestTask) {
mustRunAfter("precommit") mustRunAfter("precommit")
systemProperty 'tests.target_cluster', 'leader' systemProperty 'tests.target_cluster', 'leader'
@ -37,8 +54,19 @@ tasks.register('follow-cluster', RestIntegTestTask) {
dependsOn 'leader-cluster' dependsOn 'leader-cluster'
useCluster leaderCluster useCluster leaderCluster
systemProperty 'tests.target_cluster', 'follow' systemProperty 'tests.target_cluster', 'follow'
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderUri = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}.map { it.allHttpSocketURI.get(0) }
nonInputProperties.systemProperty 'tests.leader_host', nonInputProperties.systemProperty 'tests.leader_host',
"${-> leaderCluster.get().getAllHttpSocketURI().get(0)}" "${-> leaderUri.get() }"
} }
tasks.register("followClusterRestartTest", StandaloneRestIntegTestTask) { tasks.register("followClusterRestartTest", StandaloneRestIntegTestTask) {
@ -48,10 +76,27 @@ tasks.register("followClusterRestartTest", StandaloneRestIntegTestTask) {
systemProperty 'tests.rest.load_packaged', 'false' systemProperty 'tests.rest.load_packaged', 'false'
systemProperty 'tests.target_cluster', 'follow-restart' systemProperty 'tests.target_cluster', 'follow-restart'
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderUri = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}.map { it.allHttpSocketURI.get(0) }
def followUris = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("follow-cluster")
it.parameters.service = serviceProvider
}.map { it.allHttpSocketURI.join(",") }
nonInputProperties.systemProperty 'tests.leader_host', leaderUri
nonInputProperties.systemProperty 'tests.rest.cluster', followUris
doFirst { doFirst {
followCluster.get().restart() serviceProvider.get().restart(clusterPath, "follow-cluster")
nonInputProperties.systemProperty 'tests.leader_host', leaderCluster.map(c-> c.getAllHttpSocketURI().get(0))
nonInputProperties.systemProperty 'tests.rest.cluster', followCluster.map(c -> c.getAllHttpSocketURI().join(","))
} }
} }

View file

@ -1,4 +1,9 @@
import org.elasticsearch.gradle.internal.test.RestIntegTestTask import org.elasticsearch.gradle.internal.test.RestIntegTestTask
import org.elasticsearch.gradle.testclusters.TestClusterValueSource
import org.elasticsearch.gradle.testclusters.TestClustersPlugin
import org.elasticsearch.gradle.testclusters.TestClustersRegistry
import org.elasticsearch.gradle.util.GradleUtils
import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE import static org.elasticsearch.gradle.PropertyNormalization.IGNORE_VALUE
apply plugin: 'elasticsearch.internal-testclusters' apply plugin: 'elasticsearch.internal-testclusters'
@ -10,26 +15,38 @@ dependencies {
testImplementation project(':x-pack:plugin:ccr:qa') testImplementation project(':x-pack:plugin:ccr:qa')
} }
def clusterPath = getPath()
def leadCluster = testClusters.register('leader-cluster') { def leadCluster = testClusters.register('leader-cluster') {
testDistribution = 'DEFAULT' testDistribution = 'DEFAULT'
setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.license.self_generated.type', 'trial'
setting 'xpack.security.enabled', 'true' setting 'xpack.security.enabled', 'true'
extraConfigFile 'roles.yml', file('leader-roles.yml') extraConfigFile 'roles.yml', file('leader-roles.yml')
user username: "test_admin", role: "superuser" user username: "test_admin", role: "superuser"
user username: "test_ccr", role: "ccruser" user username: "test_ccr", role: "ccruser"
} }
testClusters.register('follow-cluster') { testClusters.register('follow-cluster') {
testDistribution = 'DEFAULT' testDistribution = 'DEFAULT'
setting 'cluster.remote.leader_cluster.seeds', { Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
"\"${leadCluster.get().getAllTransportPortURI().join(",")}\"" project.gradle.sharedServices,
}, IGNORE_VALUE TestClustersPlugin.REGISTRY_SERVICE_NAME
setting 'xpack.license.self_generated.type', 'trial' )
setting 'xpack.security.enabled', 'true' def leaderUris = project.getProviders().of(TestClusterValueSource.class) {
setting 'xpack.monitoring.collection.enabled', 'false' // will be enabled by tests it.parameters.path.set(clusterPath)
extraConfigFile 'roles.yml', file('follower-roles.yml') it.parameters.clusterName.set("leader-cluster")
user username: "test_admin", role: "superuser" it.parameters.service = serviceProvider
user username: "test_ccr", role: "ccruser" }.map { it.AllTransportPortURI }
setting 'cluster.remote.leader_cluster.seeds', {
"\"${leaderUris.get().join(",")}\""
}, IGNORE_VALUE
setting 'xpack.license.self_generated.type', 'trial'
setting 'xpack.security.enabled', 'true'
setting 'xpack.monitoring.collection.enabled', 'false' // will be enabled by tests
extraConfigFile 'roles.yml', file('follower-roles.yml')
user username: "test_admin", role: "superuser"
user username: "test_ccr", role: "ccruser"
} }
tasks.register('leader-cluster', RestIntegTestTask) { tasks.register('leader-cluster', RestIntegTestTask) {
@ -41,7 +58,17 @@ def followerClusterTestTask = tasks.register('follow-cluster', RestIntegTestTask
dependsOn 'leader-cluster' dependsOn 'leader-cluster'
useCluster leadCluster useCluster leadCluster
systemProperty 'tests.target_cluster', 'follow' systemProperty 'tests.target_cluster', 'follow'
nonInputProperties.systemProperty 'tests.leader_host', leadCluster.map(c-> c.getAllHttpSocketURI().get(0)) Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
def leaderUri = project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("leader-cluster")
it.parameters.service = serviceProvider
}.map { it.allHttpSocketURI.get(0) }
nonInputProperties.systemProperty 'tests.leader_host', leaderUri
} }
tasks.named("check").configure { dependsOn(followerClusterTestTask) } tasks.named("check").configure { dependsOn(followerClusterTestTask) }

View file

@ -83,7 +83,6 @@ tasks.named("test").configure {
} }
} }
File functionsFolder = file("build/testrun/test/temp/esql/functions") File functionsFolder = file("build/testrun/test/temp/esql/functions")
File signatureFolder = file("build/testrun/test/temp/esql/functions/signature")
File typesFolder = file("build/testrun/test/temp/esql/functions/types") File typesFolder = file("build/testrun/test/temp/esql/functions/types")
def functionsDocFolder = file("${rootDir}/docs/reference/esql/functions") def functionsDocFolder = file("${rootDir}/docs/reference/esql/functions")
def effectiveProjectDir = projectDir def effectiveProjectDir = projectDir

View file

@ -1,4 +1,8 @@
import org.elasticsearch.gradle.internal.test.RestIntegTestTask import org.elasticsearch.gradle.internal.test.RestIntegTestTask
import org.elasticsearch.gradle.testclusters.TestClusterValueSource
import org.elasticsearch.gradle.testclusters.TestClustersPlugin
import org.elasticsearch.gradle.testclusters.TestClustersRegistry
import org.elasticsearch.gradle.util.GradleUtils
apply plugin: 'elasticsearch.internal-test-artifact' apply plugin: 'elasticsearch.internal-test-artifact'
@ -11,7 +15,10 @@ dependencies {
Project mainProject = project Project mainProject = project
subprojects { subprojects {
def clusterPath = getPath()
// Use tests from the root security qa project in subprojects // Use tests from the root security qa project in subprojects
configurations.create('testArtifacts').transitive(false) configurations.create('testArtifacts').transitive(false)
@ -46,6 +53,17 @@ subprojects {
dependsOn copyTestClasses dependsOn copyTestClasses
classpath += configurations.testArtifacts classpath += configurations.testArtifacts
testClassesDirs = project.files(testArtifactsDir) testClassesDirs = project.files(testArtifactsDir)
Provider<TestClustersRegistry> serviceProvider = GradleUtils.getBuildService(
project.gradle.sharedServices,
TestClustersPlugin.REGISTRY_SERVICE_NAME
)
project.getProviders().of(TestClusterValueSource.class) {
it.parameters.path.set(clusterPath)
it.parameters.clusterName.set("javaRestTest")
it.parameters.service = serviceProvider
}
nonInputProperties.systemProperty 'tests.audit.logfile', nonInputProperties.systemProperty 'tests.audit.logfile',
"${-> testClusters.javaRestTest.singleNode().getAuditLog()}" "${-> testClusters.javaRestTest.singleNode().getAuditLog()}"
nonInputProperties.systemProperty 'tests.audit.yesterday.logfile', nonInputProperties.systemProperty 'tests.audit.yesterday.logfile',

View file

@ -9,7 +9,6 @@ import org.elasticsearch.gradle.Architecture
import org.elasticsearch.gradle.OS import org.elasticsearch.gradle.OS
import org.elasticsearch.gradle.Version import org.elasticsearch.gradle.Version
import org.elasticsearch.gradle.internal.BwcVersions import org.elasticsearch.gradle.internal.BwcVersions
import org.elasticsearch.gradle.internal.info.BuildParams
import org.elasticsearch.gradle.internal.test.AntFixture import org.elasticsearch.gradle.internal.test.AntFixture
import org.elasticsearch.gradle.testclusters.StandaloneRestIntegTestTask import org.elasticsearch.gradle.testclusters.StandaloneRestIntegTestTask
import org.elasticsearch.gradle.transform.UnzipTransform import org.elasticsearch.gradle.transform.UnzipTransform