Browse Source

Merge remote-tracking branch 'upstream/master' into index-lifecycle

Tal Levy 7 years ago
parent
commit
c6c01425bb

+ 18 - 37
buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java

@@ -25,16 +25,13 @@ import org.gradle.api.logging.Logger;
 import org.gradle.api.logging.Logging;
 
 import java.util.Objects;
-import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 
 public class ElasticsearchNode {
 
     private final String name;
     private final GradleServicesAdapter services;
-    private final AtomicInteger noOfClaims = new AtomicInteger();
-    private final AtomicBoolean started = new AtomicBoolean(false);
+    private final AtomicBoolean configurationFrozen = new AtomicBoolean(false);
     private final Logger logger = Logging.getLogger(ElasticsearchNode.class);
 
     private Distribution distribution;
@@ -54,7 +51,7 @@ public class ElasticsearchNode {
     }
 
     public void setVersion(Version version) {
-        checkNotRunning();
+        checkFrozen();
         this.version = version;
     }
 
@@ -63,47 +60,26 @@ public class ElasticsearchNode {
     }
 
     public void setDistribution(Distribution distribution) {
-        checkNotRunning();
+        checkFrozen();
         this.distribution = distribution;
     }
 
-    public void claim() {
-        noOfClaims.incrementAndGet();
+    void start() {
+        logger.info("Starting `{}`", this);
     }
 
-    /**
-     * Start the cluster if not running. Does nothing if the cluster is already running.
-     *
-     * @return future of thread running in the background
-     */
-    public Future<Void> start() {
-        if (started.getAndSet(true)) {
-            logger.lifecycle("Already started cluster: {}", name);
-        } else {
-            logger.lifecycle("Starting cluster: {}", name);
-        }
-        return null;
+    void stop(boolean tailLogs) {
+        logger.info("Stopping `{}`, tailLogs: {}", this, tailLogs);
     }
 
-    /**
-     * Stops a running cluster if it's not claimed. Does nothing otherwise.
-     */
-    public void unClaimAndStop() {
-        int decrementedClaims = noOfClaims.decrementAndGet();
-        if (decrementedClaims > 0) {
-            logger.lifecycle("Not stopping {}, since cluster still has {} claim(s)", name, decrementedClaims);
-            return;
-        }
-        if (started.get() == false) {
-            logger.lifecycle("Asked to unClaimAndStop, but cluster was not running: {}", name);
-            return;
-        }
-        logger.lifecycle("Stopping {}, number of claims is {}", name, decrementedClaims);
+    public void freeze() {
+        logger.info("Locking configuration of `{}`", this);
+        configurationFrozen.set(true);
     }
 
-    private void checkNotRunning() {
-        if (started.get()) {
-            throw new IllegalStateException("Configuration can not be altered while running ");
+    private void checkFrozen() {
+        if (configurationFrozen.get()) {
+            throw new IllegalStateException("Configuration can not be altered, already locked");
         }
     }
 
@@ -119,4 +95,9 @@ public class ElasticsearchNode {
     public int hashCode() {
         return Objects.hash(name);
     }
+
+    @Override
+    public String toString() {
+        return "ElasticsearchNode{name='" + name + "'}";
+    }
 }

+ 116 - 18
buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java

@@ -33,73 +33,171 @@ import org.gradle.api.tasks.TaskState;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
 
 public class TestClustersPlugin implements Plugin<Project> {
 
     private static final String LIST_TASK_NAME = "listTestClusters";
     private static final String NODE_EXTENSION_NAME = "testClusters";
+    public static final String PROPERTY_TESTCLUSTERS_RUN_ONCE = "_testclusters_run_once";
 
     private final Logger logger =  Logging.getLogger(TestClustersPlugin.class);
 
+    // this is static because we need a single mapping across multi project builds, as some of the listeners we use,
+    // like task graph are singletons across multi project builds.
+    private static final Map<Task, List<ElasticsearchNode>> usedClusters = new ConcurrentHashMap<>();
+    private static final Map<ElasticsearchNode, Integer> claimsInventory = new ConcurrentHashMap<>();
+    private static final Set<ElasticsearchNode> runningClusters = Collections.synchronizedSet(new HashSet<>());
+
     @Override
     public void apply(Project project) {
-        NamedDomainObjectContainer<? extends ElasticsearchNode> container = project.container(
+        // enable the DSL to describe clusters
+        NamedDomainObjectContainer<ElasticsearchNode> container = createTestClustersContainerExtension(project);
+
+        // provide a task to be able to list defined clusters.
+        createListClustersTask(project, container);
+
+        // create DSL for tasks to mark clusters these use
+        createUseClusterTaskExtension(project);
+
+        // There's a single Gradle instance for multi project builds, this means that some configuration needs to be
+        // done only once even if the plugin is applied multiple times as a part of multi project build
+        ExtraPropertiesExtension rootProperties = project.getRootProject().getExtensions().getExtraProperties();
+        if (rootProperties.has(PROPERTY_TESTCLUSTERS_RUN_ONCE) == false) {
+            rootProperties.set(PROPERTY_TESTCLUSTERS_RUN_ONCE, true);
+            // When running in the Daemon it's possible for this to hold references to past
+            usedClusters.clear();
+            claimsInventory.clear();
+            runningClusters.clear();
+
+            // When we know what tasks will run, we claim the clusters of those task to differentiate between clusters
+            // that are defined in the build script and the ones that will actually be used in this invocation of gradle
+            // we use this information to determine when the last task that required the cluster executed so that we can
+            // terminate the cluster right away and free up resources.
+            configureClaimClustersHook(project);
+
+            // Before each task, we determine if a cluster needs to be started for that task.
+            configureStartClustersHook(project);
+
+            // After each task we determine if there are clusters that are no longer needed.
+            configureStopClustersHook(project);
+        }
+    }
+
+    private NamedDomainObjectContainer<ElasticsearchNode> createTestClustersContainerExtension(Project project) {
+        // Create an extensions that allows describing clusters
+        NamedDomainObjectContainer<ElasticsearchNode> container = project.container(
             ElasticsearchNode.class,
-            (name) -> new ElasticsearchNode(name, GradleServicesAdapter.getInstance(project))
+            name -> new ElasticsearchNode(
+                name,
+                GradleServicesAdapter.getInstance(project)
+            )
         );
         project.getExtensions().add(NODE_EXTENSION_NAME, container);
+        return container;
+    }
+
 
+    private void createListClustersTask(Project project, NamedDomainObjectContainer<ElasticsearchNode> container) {
         Task listTask = project.getTasks().create(LIST_TASK_NAME);
         listTask.setGroup("ES cluster formation");
         listTask.setDescription("Lists all ES clusters configured for this project");
         listTask.doLast((Task task) ->
-            container.forEach((ElasticsearchNode cluster) ->
+            container.forEach(cluster ->
                 logger.lifecycle("   * {}: {}", cluster.getName(), cluster.getDistribution())
             )
         );
+    }
 
-        Map<Task, List<ElasticsearchNode>> taskToCluster = new HashMap<>();
-
+    private void createUseClusterTaskExtension(Project project) {
         // register an extension for all current and future tasks, so that any task can declare that it wants to use a
         // specific cluster.
         project.getTasks().all((Task task) ->
             task.getExtensions().findByType(ExtraPropertiesExtension.class)
-            .set(
-                "useCluster",
-                new Closure<Void>(this, this) {
-                    public void doCall(ElasticsearchNode conf) {
-                        taskToCluster.computeIfAbsent(task, k -> new ArrayList<>()).add(conf);
-                    }
-                })
+                .set(
+                    "useCluster",
+                    new Closure<Void>(this, task) {
+                        public void doCall(ElasticsearchNode node) {
+                            usedClusters.computeIfAbsent(task, k -> new ArrayList<>()).add(node);
+                        }
+                    })
         );
+    }
 
+    private void configureClaimClustersHook(Project project) {
         project.getGradle().getTaskGraph().whenReady(taskExecutionGraph ->
             taskExecutionGraph.getAllTasks()
                 .forEach(task ->
-                    taskToCluster.getOrDefault(task, Collections.emptyList()).forEach(ElasticsearchNode::claim)
+                    usedClusters.getOrDefault(task, Collections.emptyList()).forEach(each -> {
+                        synchronized (claimsInventory) {
+                            claimsInventory.put(each, claimsInventory.getOrDefault(each, 0) + 1);
+                        }
+                        each.freeze();
+                    })
                 )
         );
+    }
+
+    private void configureStartClustersHook(Project project) {
         project.getGradle().addListener(
             new TaskActionListener() {
                 @Override
                 public void beforeActions(Task task) {
                     // we only start the cluster before the actions, so we'll not start it if the task is up-to-date
-                    taskToCluster.getOrDefault(task, new ArrayList<>()).forEach(ElasticsearchNode::start);
+                    final List<ElasticsearchNode> clustersToStart;
+                    synchronized (runningClusters) {
+                        clustersToStart = usedClusters.getOrDefault(task,Collections.emptyList()).stream()
+                            .filter(each -> runningClusters.contains(each) == false)
+                            .collect(Collectors.toList());
+                        runningClusters.addAll(clustersToStart);
+                    }
+                    clustersToStart.forEach(ElasticsearchNode::start);
+
                 }
                 @Override
                 public void afterActions(Task task) {}
             }
         );
+    }
+
+    private void configureStopClustersHook(Project project) {
         project.getGradle().addListener(
             new TaskExecutionListener() {
                 @Override
                 public void afterExecute(Task task, TaskState state) {
-                    // always un-claim the cluster, even if _this_ task is up-to-date, as others might not have been and caused the
-                    // cluster to start.
-                    taskToCluster.getOrDefault(task, new ArrayList<>()).forEach(ElasticsearchNode::unClaimAndStop);
+                    // always unclaim the cluster, even if _this_ task is up-to-date, as others might not have been
+                    // and caused the cluster to start.
+                    List<ElasticsearchNode> clustersUsedByTask = usedClusters.getOrDefault(
+                        task,
+                        Collections.emptyList()
+                    );
+                    if (state.getFailure() != null) {
+                        // If the task fails, and other tasks use this cluster, the other task will likely never be
+                        // executed at all, so we will never get to un-claim and terminate it.
+                        // The downside is that with multi project builds if that other  task is in a different
+                        // project and executing right now, we may terminate the cluster while it's running it.
+                        clustersUsedByTask.forEach(each -> each.stop(true));
+                    } else {
+                        clustersUsedByTask.forEach(each -> {
+                            synchronized (claimsInventory) {
+                                claimsInventory.put(each, claimsInventory.get(each) - 1);
+                            }
+                        });
+                        final List<ElasticsearchNode> stoppable;
+                        synchronized (runningClusters) {
+                            stoppable = claimsInventory.entrySet().stream()
+                                .filter(entry -> entry.getValue() == 0)
+                                .filter(entry -> runningClusters.contains(entry.getKey()))
+                                .map(Map.Entry::getKey)
+                                .collect(Collectors.toList());
+                        }
+                        stoppable.forEach(each -> each.stop(false));
+                    }
                 }
                 @Override
                 public void beforeExecute(Task task) {}

+ 2 - 0
buildSrc/src/test/java/org/elasticsearch/gradle/BuildExamplePluginsIT.java

@@ -23,6 +23,7 @@ import org.apache.commons.io.FileUtils;
 import org.elasticsearch.gradle.test.GradleIntegrationTestCase;
 import org.gradle.testkit.runner.GradleRunner;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 
@@ -38,6 +39,7 @@ import java.util.List;
 import java.util.Objects;
 import java.util.stream.Collectors;
 
+@Ignore
 public class BuildExamplePluginsIT extends GradleIntegrationTestCase {
 
     private static List<File> EXAMPLE_PLUGINS = Collections.unmodifiableList(

+ 34 - 12
buildSrc/src/test/java/org/elasticsearch/gradle/test/GradleIntegrationTestCase.java

@@ -78,8 +78,16 @@ public abstract class GradleIntegrationTestCase extends GradleUnitTestCase {
         assertTaskOutcome(result, taskName, TaskOutcome.FAILED);
     }
 
-    protected void assertTaskSuccessful(BuildResult result, String taskName) {
-        assertTaskOutcome(result, taskName, TaskOutcome.SUCCESS);
+    protected void assertTaskSuccessful(BuildResult result, String... taskNames) {
+        for (String taskName : taskNames) {
+            assertTaskOutcome(result, taskName, TaskOutcome.SUCCESS);
+        }
+    }
+
+    protected void assertTaskSkipped(BuildResult result, String... taskNames) {
+        for (String taskName : taskNames) {
+            assertTaskOutcome(result, taskName, TaskOutcome.SKIPPED);
+        }
     }
 
     private void assertTaskOutcome(BuildResult result, String taskName, TaskOutcome taskOutcome) {
@@ -96,17 +104,19 @@ public abstract class GradleIntegrationTestCase extends GradleUnitTestCase {
         );
     }
 
-    protected void assertTaskUpToDate(BuildResult result, String taskName) {
-        BuildTask task = result.task(taskName);
-        if (task == null) {
-            fail("Expected task `" + taskName + "` to be up-to-date, but it did not run");
+    protected void assertTaskUpToDate(BuildResult result, String... taskNames) {
+        for (String taskName : taskNames) {
+            BuildTask task = result.task(taskName);
+            if (task == null) {
+                fail("Expected task `" + taskName + "` to be up-to-date, but it did not run");
+            }
+            assertEquals(
+                "Expected task to be up to date but it was: " + task.getOutcome() +
+                    "\n\nOutput is:\n" + result.getOutput(),
+                TaskOutcome.UP_TO_DATE,
+                task.getOutcome()
+            );
         }
-        assertEquals(
-            "Expected task to be up to date but it was: " + task.getOutcome() +
-                "\n\nOutput is:\n" + result.getOutput() ,
-            TaskOutcome.UP_TO_DATE,
-            task.getOutcome()
-        );
     }
 
     protected void assertBuildFileExists(BuildResult result, String projectName, String path) {
@@ -139,4 +149,16 @@ public abstract class GradleIntegrationTestCase extends GradleUnitTestCase {
             return file.getAbsolutePath();
         }
     }
+
+    public void assertOutputOnlyOnce(String output, String... text) {
+        for (String each : text) {
+            int i = output.indexOf(each);
+            if (i == -1 ) {
+                fail("Expected `" + text + "` to appear at most once, but it didn't at all.\n\nOutout is:\n"+ output);
+            }
+            if(output.indexOf(each) !=  output.lastIndexOf(each)) {
+                fail("Expected `" + text + "` to appear at most once, but it did multiple times.\n\nOutout is:\n"+ output);
+            }
+        }
+    }
 }

+ 81 - 75
buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java

@@ -21,124 +21,130 @@ package org.elasticsearch.gradle.testclusters;
 import org.elasticsearch.gradle.test.GradleIntegrationTestCase;
 import org.gradle.testkit.runner.BuildResult;
 import org.gradle.testkit.runner.GradleRunner;
-import org.gradle.testkit.runner.TaskOutcome;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
+import java.util.Arrays;
 
 public class TestClustersPluginIT extends GradleIntegrationTestCase {
 
     public void testListClusters() {
-        BuildResult result = GradleRunner.create()
-            .withProjectDir(getProjectDir("testclusters"))
-            .withArguments("listTestClusters", "-s")
-            .withPluginClasspath()
-            .build();
+        BuildResult result = getTestClustersRunner("listTestClusters").build();
 
-        assertEquals(TaskOutcome.SUCCESS, result.task(":listTestClusters").getOutcome());
+        assertTaskSuccessful(result, ":listTestClusters");
         assertOutputContains(
             result.getOutput(),
-                "   * myTestCluster:"
+            "   * myTestCluster:"
         );
-
     }
 
     public void testUseClusterByOne() {
-        BuildResult result = GradleRunner.create()
-            .withProjectDir(getProjectDir("testclusters"))
-            .withArguments("user1", "-s")
-            .withPluginClasspath()
-            .build();
-
-        assertEquals(TaskOutcome.SUCCESS, result.task(":user1").getOutcome());
-        assertOutputContains(
-            result.getOutput(),
-                "Starting cluster: myTestCluster",
-                "Stopping myTestCluster, number of claims is 0"
-        );
+        BuildResult result = getTestClustersRunner("user1").build();
+        assertTaskSuccessful(result, ":user1");
+        assertStartedAndStoppedOnce(result);
     }
 
     public void testUseClusterByOneWithDryRun() {
-        BuildResult result = GradleRunner.create()
-            .withProjectDir(getProjectDir("testclusters"))
-            .withArguments("user1", "-s", "--dry-run")
-            .withPluginClasspath()
-            .build();
-
+        BuildResult result = getTestClustersRunner("--dry-run", "user1").build();
         assertNull(result.task(":user1"));
-        assertOutputDoesNotContain(
-            result.getOutput(),
-            "Starting cluster: myTestCluster",
-            "Stopping myTestCluster, number of claims is 0"
-        );
+        assertNotStarted(result);
     }
 
     public void testUseClusterByTwo() {
-        BuildResult result = GradleRunner.create()
-            .withProjectDir(getProjectDir("testclusters"))
-            .withArguments("user1", "user2", "-s")
-            .withPluginClasspath()
-            .build();
+        BuildResult result = getTestClustersRunner("user1", "user2").build();
+        assertTaskSuccessful(result, ":user1", ":user2");
+        assertStartedAndStoppedOnce(result);
+    }
 
-        assertEquals(TaskOutcome.SUCCESS, result.task(":user1").getOutcome());
-        assertEquals(TaskOutcome.SUCCESS, result.task(":user2").getOutcome());
+    public void testUseClusterByUpToDateTask() {
+        BuildResult result = getTestClustersRunner("upToDate1", "upToDate2").build();
+        assertTaskUpToDate(result, ":upToDate1", ":upToDate2");
+        assertNotStarted(result);
+    }
+
+    public void testUseClusterBySkippedTask() {
+        BuildResult result = getTestClustersRunner("skipped1", "skipped2").build();
+        assertTaskSkipped(result, ":skipped1", ":skipped2");
+        assertNotStarted(result);
+    }
+
+    public void testUseClusterBySkippedAndWorkingTask() {
+        BuildResult result = getTestClustersRunner("skipped1", "user1").build();
+        assertTaskSkipped(result, ":skipped1");
+        assertTaskSuccessful(result, ":user1");
         assertOutputContains(
             result.getOutput(),
-            "Starting cluster: myTestCluster",
-            "Not stopping myTestCluster, since cluster still has 1 claim(s)",
-            "Stopping myTestCluster, number of claims is 0"
+            "> Task :user1",
+            "Starting `ElasticsearchNode{name='myTestCluster'}`",
+            "Stopping `ElasticsearchNode{name='myTestCluster'}`"
         );
     }
 
-    public void testUseClusterByUpToDateTask() {
+    public void testMultiProject() {
         BuildResult result = GradleRunner.create()
-            .withProjectDir(getProjectDir("testclusters"))
-            .withArguments("upToDate1", "upToDate2", "-s")
+            .withProjectDir(getProjectDir("testclusters_multiproject"))
+            .withArguments("user1", "user2", "-s", "-i", "--parallel")
             .withPluginClasspath()
             .build();
+        assertTaskSuccessful(result, ":user1", ":user2");
+
+        assertStartedAndStoppedOnce(result);
+    }
 
-        assertEquals(TaskOutcome.UP_TO_DATE, result.task(":upToDate1").getOutcome());
-        assertEquals(TaskOutcome.UP_TO_DATE, result.task(":upToDate2").getOutcome());
+    public void testUseClusterByFailingOne() {
+        BuildResult result = getTestClustersRunner("itAlwaysFails").buildAndFail();
+        assertTaskFailed(result, ":itAlwaysFails");
+        assertStartedAndStoppedOnce(result);
         assertOutputContains(
             result.getOutput(),
-            "Not stopping myTestCluster, since cluster still has 1 claim(s)",
-            "cluster was not running: myTestCluster"
+            "Stopping `ElasticsearchNode{name='myTestCluster'}`, tailLogs: true",
+            "Execution failed for task ':itAlwaysFails'."
         );
-        assertOutputDoesNotContain(result.getOutput(), "Starting cluster: myTestCluster");
     }
 
-    public void testUseClusterBySkippedTask() {
-        BuildResult result = GradleRunner.create()
-            .withProjectDir(getProjectDir("testclusters"))
-            .withArguments("skipped1", "skipped2", "-s")
-            .withPluginClasspath()
-            .build();
+    public void testUseClusterByFailingDependency() {
+        BuildResult result = getTestClustersRunner("dependsOnFailed").buildAndFail();
+        assertTaskFailed(result, ":itAlwaysFails");
+        assertNull(result.task(":dependsOnFailed"));
+        assertStartedAndStoppedOnce(result);
+        assertOutputContains(
+            result.getOutput(),
+            "Stopping `ElasticsearchNode{name='myTestCluster'}`, tailLogs: true",
+            "Execution failed for task ':itAlwaysFails'."
+        );
+    }
 
-        assertEquals(TaskOutcome.SKIPPED, result.task(":skipped1").getOutcome());
-        assertEquals(TaskOutcome.SKIPPED, result.task(":skipped2").getOutcome());
+    public void testConfigurationLocked() {
+        BuildResult result = getTestClustersRunner("illegalConfigAlter").buildAndFail();
+        assertTaskFailed(result, ":illegalConfigAlter");
         assertOutputContains(
             result.getOutput(),
-            "Not stopping myTestCluster, since cluster still has 1 claim(s)",
-            "cluster was not running: myTestCluster"
+            "Configuration can not be altered, already locked"
         );
-        assertOutputDoesNotContain(result.getOutput(), "Starting cluster: myTestCluster");
     }
 
-    public void tetUseClusterBySkippedAndWorkingTask() {
-        BuildResult result = GradleRunner.create()
+    private void assertNotStarted(BuildResult result) {
+        assertOutputDoesNotContain(
+            result.getOutput(),
+            "Starting ",
+            "Stopping "
+        );
+    }
+
+    private GradleRunner getTestClustersRunner(String... tasks) {
+        String[] arguments = Arrays.copyOf(tasks, tasks.length + 2);
+        arguments[tasks.length] = "-s";
+        arguments[tasks.length + 1] = "-i";
+        return GradleRunner.create()
             .withProjectDir(getProjectDir("testclusters"))
-            .withArguments("skipped1", "user1", "-s")
-            .withPluginClasspath()
-            .build();
+            .withArguments(arguments)
+            .withPluginClasspath();
+    }
 
-        assertEquals(TaskOutcome.SKIPPED, result.task(":skipped1").getOutcome());
-        assertEquals(TaskOutcome.SUCCESS, result.task(":user1").getOutcome());
-        assertOutputContains(
+
+    private void assertStartedAndStoppedOnce(BuildResult result) {
+        assertOutputOnlyOnce(
             result.getOutput(),
-            "> Task :user1",
-            "Starting cluster: myTestCluster",
-            "Stopping myTestCluster, number of claims is 0"
+            "Starting `ElasticsearchNode{name='myTestCluster'}`",
+            "Stopping `ElasticsearchNode{name='myTestCluster'}`"
         );
     }
-
 }

+ 20 - 0
buildSrc/src/testKit/testclusters/build.gradle

@@ -39,3 +39,23 @@ task skipped2 {
     enabled = false
     useCluster testClusters.myTestCluster
 }
+
+task itAlwaysFails {
+    doLast {
+        throw new GradleException("Task 1 failed!")
+    }
+    useCluster testClusters.myTestCluster
+}
+
+task dependsOnFailed {
+    dependsOn itAlwaysFails
+    useCluster testClusters.myTestCluster
+}
+
+task illegalConfigAlter {
+    useCluster testClusters.myTestCluster
+    doFirst {
+        println "Going to alter configuration after use"
+        testClusters.myTestCluster.distribution = 'ZIP_OSS'
+    }
+}

+ 18 - 0
buildSrc/src/testKit/testclusters_multiproject/alpha/build.gradle

@@ -0,0 +1,18 @@
+plugins {
+    id 'elasticsearch.testclusters'
+}
+testClusters {
+    myTestCluster
+}
+task user1 {
+    useCluster testClusters.myTestCluster
+    doFirst {
+        println "$path"
+    }
+}
+task user2 {
+    useCluster testClusters.myTestCluster
+    doFirst {
+        println "$path"
+    }
+}

+ 21 - 0
buildSrc/src/testKit/testclusters_multiproject/bravo/build.gradle

@@ -0,0 +1,21 @@
+plugins {
+    id 'elasticsearch.testclusters'
+}
+
+testClusters {
+    myTestCluster
+}
+
+task user1 {
+    useCluster testClusters.myTestCluster
+    doFirst {
+        println "$path"
+    }
+}
+
+task user2 {
+    useCluster testClusters.myTestCluster
+    doFirst {
+        println "$path"
+    }
+}

+ 21 - 0
buildSrc/src/testKit/testclusters_multiproject/build.gradle

@@ -0,0 +1,21 @@
+plugins {
+    id 'elasticsearch.testclusters'
+}
+
+testClusters {
+    myTestCluster
+}
+
+task user1 {
+    useCluster testClusters.myTestCluster
+    doFirst {
+        println "$path"
+    }
+}
+
+task user2 {
+    useCluster testClusters.myTestCluster
+    doFirst {
+        println "$path"
+    }
+}

+ 5 - 0
buildSrc/src/testKit/testclusters_multiproject/charlie/build.gradle

@@ -0,0 +1,5 @@
+task hello() {
+    doLast {
+        println "This task does not use the testclusters plugin. So it will have no extension."
+    }
+}

+ 3 - 0
buildSrc/src/testKit/testclusters_multiproject/settings.gradle

@@ -0,0 +1,3 @@
+include ':alpha'
+include ':bravo'
+include ':charlie'

+ 5 - 0
docs/reference/ingest/ingest-node-common-processor.asciidoc

@@ -0,0 +1,5 @@
+| `if`             | no       | -        | Conditionally execute this processor.
+| `on_failure`     | no       | -        | Handle failures for this processor. See <<handling-failure-in-pipelines>>.
+| `ignore_failure` | no       | `false`  | Ignore failures for this processor. See <<handling-failure-in-pipelines>>.
+| `tag`            | no       | -        | An identifier for this processor. Useful for debugging and metrics.
+// TODO: See <<ingest-conditionals>>. <-- for the if description once PR 35044 is merged

+ 33 - 2
docs/reference/ingest/ingest-node.asciidoc

@@ -778,6 +778,7 @@ Accepts a single value or an array of values.
 | Name      | Required  | Default  | Description
 | `field`  | yes       | -        | The field to be appended to. Supports <<accessing-template-fields,template snippets>>.
 | `value`  | yes       | -        | The value to be appended. Supports <<accessing-template-fields,template snippets>>.
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -806,6 +807,7 @@ the field is not a supported format or resultant value exceeds 2^63.
 | `field`          | yes       | -        | The field to convert
 | `target_field`   | no        | `field`  | The field to assign the converted value to, by default `field` is updated in-place
 | `ignore_missing` | no        | `false`  | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -844,6 +846,7 @@ still be updated with the unconverted field value.
 | `target_field`   | no        | `field`  | The field to assign the converted value to, by default `field` is updated in-place
 | `type`           | yes       | -        | The type to convert the existing value to
 | `ignore_missing` | no        | `false`  | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -876,6 +879,7 @@ in the same order they were defined as part of the processor definition.
 | `formats`              | yes       | -                   | An array of the expected date formats. Can be a Joda pattern or one of the following formats: ISO8601, UNIX, UNIX_MS, or TAI64N.
 | `timezone`        | no        | UTC                 | The timezone to use when parsing the date. Supports <<accessing-template-fields,template snippets>>.
 | `locale`          | no        | ENGLISH             | The locale to use when parsing the date, relevant when parsing month names or week days. Supports <<accessing-template-fields,template snippets>>.
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 Here is an example that adds the parsed date to the `timestamp` field based on the `initial_date` field:
@@ -1065,6 +1069,7 @@ understands this to mean `2016-04-01` as is explained in the <<date-math-index-n
 | `timezone`             | no        | UTC                          | The timezone to use when parsing the date and when date math index supports resolves expressions into concrete index names.
 | `locale`               | no        | ENGLISH                      | The locale to use when parsing the date from the document being preprocessed, relevant when parsing month names or week days.
 | `index_name_format`    | no        | yyyy-MM-dd                   | The format to be used when printing the parsed date into the index name. An valid Joda pattern is expected here. Supports <<accessing-template-fields,template snippets>>.
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [[dissect-processor]]
@@ -1134,7 +1139,7 @@ See <<dissect-key-modifiers, below>> for more information.
 | `pattern`         | yes       | -        | The pattern to apply to the field
 | `append_separator`| no        | "" (empty string) | The character(s) that separate the appended fields.
 | `ignore_missing`  | no        | false             | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
-| `
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -1162,7 +1167,6 @@ modifiers.
 | `+` with `/n` | Append with order  | left and right | `%{+keyname/2} %{+keyname/1}` | Appends two or more fields together in the order specified   | <<dissect-modifier-append-key-with-order,link>>
 | `?`           | Named skip key     | left           | `%{?ignoreme}`  | Skips the matched value in the output. Same behavior as `%{}`| <<dissect-modifier-named-skip-key,link>>
 | `*` and `&`   | Reference keys     | left           | `%{*r1} %{&r1}`    | Sets the output key as value of `*` and output value of `&`  | <<dissect-modifier-reference-keys,link>>
-| `
 |======
 
 [[dissect-modifier-skip-right-padding]]
@@ -1265,6 +1269,14 @@ Reference key modifier example
 Drops the document without raising any errors. This is useful to prevent the document from
 getting indexed based on some condition.
 
+[[drop-options]]
+.Drop Options
+[options="header"]
+|======
+| Name              | Required  | Default  | Description
+include::ingest-node-common-processor.asciidoc[]
+|======
+
 [source,js]
 --------------------------------------------------
 {
@@ -1289,6 +1301,7 @@ Otherwise these <<accessing-data-in-pipelines,fields>> can't be accessed by any
 | Name     | Required  | Default  | Description
 | `field`  | yes       | -        | The field to expand into an object field
 | `path`   | no        | -        | The field that contains the field to expand. Only required if the field to expand is part another object field, because the `field` option can only understand leaf fields.
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -1406,6 +1419,7 @@ to the requester.
 |======
 | Name       | Required  | Default  | Description
 | `message`  | yes       | -        | The error message thrown by the processor. Supports <<accessing-template-fields,template snippets>>.
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -1452,6 +1466,7 @@ then it aborts the execution and leaves the array unmodified.
 | `field`          | yes       | -        | The array field
 | `processor`      | yes       | -        | The processor to execute against each field
 | `ignore_missing` | no        | false    | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 Assume the following document:
@@ -1645,6 +1660,7 @@ Grok expression.
 | `pattern_definitions`  | no        | -                   | A map of pattern-name and pattern tuples defining custom patterns to be used by the current processor. Patterns matching existing names will override the pre-existing definition.
 | `trace_match`          | no        | false               | when true, `_ingest._grok_match_index` will be inserted into your matched document's metadata with the index into the pattern found in `patterns` that matched.
 | `ignore_missing`       | no        | false               | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 Here is an example of using the provided patterns to extract out and name structured fields from a string field in
@@ -1919,6 +1935,7 @@ If the field is not a string, the processor will throw an exception.
 | `replacement`     | yes       | -        | The string to replace the matching patterns with
 | `target_field`    | no        | `field`  | The field to assign the converted value to, by default `field` is updated in-place
 | `ignore_missing`  | no        | `false`  | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -1946,6 +1963,7 @@ Throws an error when the field is not an array.
 | `field`           | yes       | -        | The field to be separated
 | `separator`       | yes       | -        | The separator character
 | `target_field`    | no        | `field`  | The field to assign the joined value to, by default `field` is updated in-place
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -1971,6 +1989,7 @@ Converts a JSON string into a structured JSON object.
 | `field`        | yes       | -        | The field to be parsed
 | `target_field` | no        | `field`  | The field to insert the converted structured object into
 | `add_to_root`  | no        | false    | Flag that forces the serialized json to be injected into the top level of the document. `target_field` must not be set when this option is chosen.
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 All JSON-supported types will be parsed (null, boolean, number, array, object, string).
@@ -2082,6 +2101,7 @@ For example, if you have a log message which contains `ip=1.2.3.4 error=REFUSED`
 | `trim_key`       | no        | `null`   | String of characters to trim from extracted keys
 | `trim_value`     | no        | `null`   | String of characters to trim from extracted values
 | `strip_brackets` | no        | `false`  | If `true` strip brackets `()`, `<>`, `[]` as well as quotes `'` and `"` from extracted values
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 
@@ -2097,6 +2117,7 @@ Converts a string to its lowercase equivalent.
 | `field`          | yes       | -        | The field to make lowercase
 | `target_field`   | no        | `field`  | The field to assign the converted value to, by default `field` is updated in-place
 | `ignore_missing` | no        | `false`  | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -2119,6 +2140,7 @@ Executes another pipeline.
 |======
 | Name             | Required  | Default  | Description
 | `name`           | yes       | -        | The name of the pipeline to execute
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -2234,6 +2256,7 @@ Removes existing fields. If one field doesn't exist, an exception will be thrown
 | Name             | Required  | Default  | Description
 | `field`          | yes       | -        | Fields to be removed. Supports <<accessing-template-fields,template snippets>>.
 | `ignore_missing` | no        | `false`  | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 Here is an example to remove a single field:
@@ -2272,6 +2295,7 @@ Renames an existing field. If the field doesn't exist or the new name is already
 | `field`           | yes       | -        | The field to be renamed. Supports <<accessing-template-fields,template snippets>>.
 | `target_field`    | yes       | -        | The new name of the field. Supports <<accessing-template-fields,template snippets>>.
 | `ignore_missing`  | no        | `false`  | If `true` and `field` does not exist, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -2305,6 +2329,7 @@ caching see <<modules-scripting-using-caching, Script Caching>>.
 | `id`                   | no        | -          | The stored script id to refer to
 | `source`               | no        | -          | An inline script to be executed
 | `params`               | no        | -          | Script Parameters
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 One of `id` or `source` options must be provided in order to properly reference a script to execute.
@@ -2401,6 +2426,7 @@ its value will be replaced with the provided one.
 | `field` | yes       | -        | The field to insert, upsert, or update. Supports <<accessing-template-fields,template snippets>>.
 | `value` | yes       | -        | The value to be set for the field. Supports <<accessing-template-fields,template snippets>>.
 | `override` | no        | true     | If processor will update fields with pre-existing non-null-valued field. When set to `false`, such fields will not be touched.
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -2427,6 +2453,7 @@ Splits a field into an array using a separator character. Only works on string f
 | `separator`       | yes       | -        | A regex which matches the separator, eg `,` or `\s+`
 | `target_field`    | no        | `field`  | The field to assign the split value to, by default `field` is updated in-place
 | `ignore_missing`  | no        | `false`  | If `true` and `field` does not exist, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -2455,6 +2482,7 @@ Throws an error when the field is not an array.
 | `field`           | yes       | -        | The field to be sorted
 | `order`           | no        | `"asc"`  | The sort order to use. Accepts `"asc"` or `"desc"`.
 | `target_field`    | no        | `field`  | The field to assign the sorted value to, by default `field` is updated in-place
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -2482,6 +2510,7 @@ NOTE: This only works on leading and trailing whitespace.
 | `field`           | yes       | -        | The string-valued field to trim whitespace from
 | `target_field`    | no        | `field`  | The field to assign the trimmed value to, by default `field` is updated in-place
 | `ignore_missing`  | no        | `false`  | If `true` and `field` does not exist, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -2506,6 +2535,7 @@ Converts a string to its uppercase equivalent.
 | `field`          | yes       | -        | The field to make uppercase
 | `target_field`   | no        | `field`  | The field to assign the converted value to, by default `field` is updated in-place
 | `ignore_missing` | no        | `false`  | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]
@@ -2530,6 +2560,7 @@ URL-decodes a string
 | `field`          | yes       | -        | The field to decode
 | `target_field`   | no        | `field`  | The field to assign the converted value to, by default `field` is updated in-place
 | `ignore_missing` | no        | `false`  | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::ingest-node-common-processor.asciidoc[]
 |======
 
 [source,js]

+ 7 - 0
plugins/analysis-icu/build.gradle

@@ -39,3 +39,10 @@ dependencies {
 dependencyLicenses {
   mapping from: /lucene-.*/, to: 'lucene'
 }
+
+// Muted: https://github.com/elastic/elasticsearch/issues/35173
+integTestRunner {
+    systemProperty 'tests.rest.blacklist', 
+        'analysis_icu/10_basic/Normalization with a UnicodeSet Filter,' +
+          'analysis_icu/10_basic/Normalization with a CamcelCase UnicodeSet Filter'
+}

+ 5 - 2
server/src/main/java/org/elasticsearch/common/lucene/search/function/FunctionScoreQuery.java

@@ -123,8 +123,7 @@ public class FunctionScoreQuery extends Query {
     final ScoreMode scoreMode;
     final float maxBoost;
     private final Float minScore;
-
-    protected final CombineFunction combineFunction;
+    private final CombineFunction combineFunction;
 
     /**
      * Creates a FunctionScoreQuery without function.
@@ -192,6 +191,10 @@ public class FunctionScoreQuery extends Query {
         return minScore;
     }
 
+    public CombineFunction getCombineFunction() {
+        return combineFunction;
+    }
+
     @Override
     public Query rewrite(IndexReader reader) throws IOException {
         Query rewritten = super.rewrite(reader);

+ 2 - 6
server/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilder.java

@@ -309,18 +309,14 @@ public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScor
             query = new MatchAllDocsQuery();
         }
 
+        CombineFunction boostMode = this.boostMode == null ? DEFAULT_BOOST_MODE : this.boostMode;
         // handle cases where only one score function and no filter was provided. In this case we create a FunctionScoreQuery.
         if (filterFunctions.length == 0) {
             return new FunctionScoreQuery(query, minScore, maxBoost);
         } else if (filterFunctions.length == 1 && filterFunctions[0] instanceof FunctionScoreQuery.FilterScoreFunction == false) {
-            CombineFunction combineFunction = this.boostMode;
-            if (combineFunction == null) {
-                combineFunction = filterFunctions[0].getDefaultScoreCombiner();
-            }
-            return new FunctionScoreQuery(query, filterFunctions[0], combineFunction, minScore, maxBoost);
+            return new FunctionScoreQuery(query, filterFunctions[0], boostMode, minScore, maxBoost);
         }
         // in all other cases we create a FunctionScoreQuery with filters
-        CombineFunction boostMode = this.boostMode == null ? DEFAULT_BOOST_MODE : this.boostMode;
         return new FunctionScoreQuery(query, scoreMode, filterFunctions, boostMode, minScore, maxBoost);
     }
 

+ 23 - 1
server/src/test/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryBuilderTests.java

@@ -20,7 +20,6 @@
 package org.elasticsearch.index.query.functionscore;
 
 import com.fasterxml.jackson.core.JsonParseException;
-
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -672,6 +671,29 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<Functi
         assertSame(rewrite.filterFunctionBuilders()[1].getFilter(), secondFunction);
     }
 
+    /**
+     * Please see https://github.com/elastic/elasticsearch/issues/35123 for context.
+     */
+    public void testSingleScriptFunction() throws IOException {
+        QueryBuilder queryBuilder = RandomQueryBuilder.createQuery(random());
+        ScoreFunctionBuilder functionBuilder = new ScriptScoreFunctionBuilder(
+            new Script(ScriptType.INLINE, MockScriptEngine.NAME, "1", Collections.emptyMap()));
+
+        FunctionScoreQueryBuilder builder = functionScoreQuery(queryBuilder, functionBuilder);
+        if (randomBoolean()) {
+            builder.boostMode(randomFrom(CombineFunction.values()));
+        }
+
+        Query query = builder.toQuery(createShardContext());
+        assertThat(query, instanceOf(FunctionScoreQuery.class));
+
+        CombineFunction expectedBoostMode = builder.boostMode() != null
+            ? builder.boostMode()
+            : FunctionScoreQueryBuilder.DEFAULT_BOOST_MODE;
+        CombineFunction actualBoostMode = ((FunctionScoreQuery) query).getCombineFunction();
+        assertEquals(expectedBoostMode, actualBoostMode);
+    }
+
     public void testQueryMalformedArrayNotSupported() throws IOException {
         String json =
             "{\n" +