Don't start a new node in InternalTestCluster#getClient (#127318)

This method would default to starting a new node when the cluster was
empty. This is pretty trappy as `getClient()` (or things like
`getMaster()` that depend on `getClient()`) don't look at all like
something that would start a new node.

In any case, the intention of tests is much clearer when they explicitly
define a cluster configuration.
This commit is contained in:
Niels Bauman 2025-04-25 10:07:52 +02:00 committed by GitHub
parent 15b6e85400
commit c72d00fd39
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
47 changed files with 91 additions and 72 deletions

View file

@ -57,7 +57,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcke
/**
* An integration test that verifies how different paths/scenarios affect the APM metrics for failure stores.
*/
@ESIntegTestCase.ClusterScope(numDataNodes = 0, numClientNodes = 0, scope = ESIntegTestCase.Scope.SUITE)
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class IngestFailureStoreMetricsIT extends ESIntegTestCase {
private static final List<String> METRICS = List.of(

View file

@ -14,11 +14,9 @@ import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.http.HttpSmokeTestCase;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.snapshots.mockstore.MockRepository;
import org.elasticsearch.test.ESIntegTestCase;
import java.util.Collection;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0)
public abstract class AbstractSnapshotRestTestCase extends HttpSmokeTestCase {
/**

View file

@ -17,6 +17,7 @@ import org.elasticsearch.client.Request;
import org.elasticsearch.client.Response;
import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase;
import org.elasticsearch.snapshots.mockstore.MockRepository;
import org.elasticsearch.test.ESIntegTestCase;
import java.util.concurrent.CancellationException;
import java.util.concurrent.TimeUnit;
@ -27,6 +28,7 @@ import static org.elasticsearch.test.TaskAssertions.assertAllTasksHaveFinished;
import static org.elasticsearch.test.TaskAssertions.awaitTaskWithPrefix;
import static org.hamcrest.core.IsEqual.equalTo;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class RestGetSnapshotsCancellationIT extends AbstractSnapshotRestTestCase {
public void testGetSnapshotsCancellation() throws Exception {

View file

@ -17,6 +17,7 @@ import org.elasticsearch.client.Request;
import org.elasticsearch.client.Response;
import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase;
import org.elasticsearch.snapshots.mockstore.MockRepository;
import org.elasticsearch.test.ESIntegTestCase;
import java.util.Collection;
import java.util.concurrent.CancellationException;
@ -27,6 +28,7 @@ import static org.elasticsearch.test.TaskAssertions.assertAllCancellableTasksAre
import static org.elasticsearch.test.TaskAssertions.assertAllTasksHaveFinished;
import static org.elasticsearch.test.TaskAssertions.awaitTaskWithPrefix;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class RestSnapshotsStatusCancellationIT extends AbstractSnapshotRestTestCase {
public void testSnapshotStatusCancellation() throws Exception {

View file

@ -50,7 +50,6 @@ import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class AutoCreateSystemIndexIT extends ESIntegTestCase {
@Override

View file

@ -56,7 +56,6 @@ import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.nullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class CreateSystemIndicesIT extends ESIntegTestCase {
@Before
@ -104,7 +103,6 @@ public class CreateSystemIndicesIT extends ESIntegTestCase {
*/
public void testNonPrimarySystemIndexIsAutoCreatedViaConcreteName() throws Exception {
final String nonPrimarySystemIndex = INDEX_NAME + "-2";
internalCluster().startNodes(1);
// Trigger the creation of the system index
indexDoc(nonPrimarySystemIndex, "1", "foo", "bar");
@ -132,7 +130,6 @@ public class CreateSystemIndicesIT extends ESIntegTestCase {
*/
public void testNonPrimarySystemIndexCreationThrowsError() {
final String nonPrimarySystemIndex = INDEX_NAME + "-2";
internalCluster().startNodes(1);
// Create the system index
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createIndex(nonPrimarySystemIndex));
@ -269,8 +266,6 @@ public class CreateSystemIndicesIT extends ESIntegTestCase {
}
private void doCreateTest(Runnable runnable, String concreteIndex) {
internalCluster().startNodes(1);
// Trigger the creation of the system index
runnable.run();
ensureGreen(INDEX_NAME);
@ -293,8 +288,6 @@ public class CreateSystemIndicesIT extends ESIntegTestCase {
}
public void testConcurrentAutoCreates() throws InterruptedException {
internalCluster().startNodes(3);
final Client client = client();
final int count = randomIntBetween(5, 30);
final CountDownLatch latch = new CountDownLatch(count);

View file

@ -51,6 +51,7 @@ public class BulkProcessorClusterSettingsIT extends ESIntegTestCase {
}
public void testIndexWithDisabledAutoCreateIndex() {
internalCluster().startNode();
updateClusterSettings(Settings.builder().put(AutoCreateIndex.AUTO_CREATE_INDEX_SETTING.getKey(), randomFrom("-*", "+.*")));
final BulkItemResponse itemResponse = client().prepareBulk()
.add(prepareIndex("test-index").setSource("foo", "bar"))

View file

@ -251,6 +251,7 @@ public class AwarenessAllocationIT extends ESIntegTestCase {
}
public void testForceAwarenessSettingValidation() {
internalCluster().startNode();
final String prefix = AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING.getKey();
final IllegalArgumentException illegalArgumentException = expectThrows(

View file

@ -186,6 +186,8 @@ public class FilteringAllocationIT extends ESIntegTestCase {
}
public void testInvalidIPFilterClusterSettings() {
logger.info("--> starting 2 nodes");
internalCluster().startNodes(2);
String ipKey = randomFrom("_ip", "_host_ip", "_publish_ip");
var filterSetting = randomFrom(
FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING,

View file

@ -16,7 +16,6 @@ import org.elasticsearch.health.HealthService;
import org.elasticsearch.health.HealthStatus;
import org.elasticsearch.health.metadata.HealthMetadata;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.InternalTestCluster;
import org.junit.After;
import org.junit.Before;
@ -27,16 +26,14 @@ import static org.elasticsearch.indices.ShardLimitValidator.SETTING_CLUSTER_MAX_
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.hasSize;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class ShardsCapacityHealthIndicatorServiceIT extends ESIntegTestCase {
private static final String INDEX_NAME = "index-name";
private InternalTestCluster internalCluster;
@Before
public void setUp() throws Exception {
super.setUp();
internalCluster = internalCluster();
updateClusterSettings(Settings.builder().put(SETTING_CLUSTER_MAX_SHARDS_PER_NODE.getKey(), 30));
}
@ -53,7 +50,7 @@ public class ShardsCapacityHealthIndicatorServiceIT extends ESIntegTestCase {
// index: 4 shards + 1 replica = 8 shards used (30 - 8 = 22 > 10 available shards)
createIndex(4, 1);
var result = fetchShardsCapacityIndicatorResult(internalCluster);
var result = fetchShardsCapacityIndicatorResult();
assertEquals(result.status(), HealthStatus.GREEN);
assertEquals(result.symptom(), "The cluster has enough room to add new shards.");
assertThat(result.diagnosisList(), empty());
@ -64,7 +61,7 @@ public class ShardsCapacityHealthIndicatorServiceIT extends ESIntegTestCase {
// index: 11 shards + 1 replica = 22 shards used (30 - 22 < 10 available shards)
createIndex(10, 1);
var result = fetchShardsCapacityIndicatorResult(internalCluster);
var result = fetchShardsCapacityIndicatorResult();
assertEquals(result.status(), HealthStatus.YELLOW);
assertEquals(result.symptom(), "Cluster is close to reaching the configured maximum number of shards for data nodes.");
assertThat(result.diagnosisList(), hasSize(1));
@ -75,7 +72,7 @@ public class ShardsCapacityHealthIndicatorServiceIT extends ESIntegTestCase {
// index: 13 shards + 1 replica = 26 shards used (30 - 26 < 5 available shards)
createIndex(13, 1);
var result = fetchShardsCapacityIndicatorResult(internalCluster);
var result = fetchShardsCapacityIndicatorResult();
assertEquals(result.status(), HealthStatus.RED);
assertEquals(result.symptom(), "Cluster is close to reaching the configured maximum number of shards for data nodes.");
assertThat(result.diagnosisList(), hasSize(1));
@ -86,15 +83,15 @@ public class ShardsCapacityHealthIndicatorServiceIT extends ESIntegTestCase {
createIndex(INDEX_NAME, indexSettings(shards, replicas).build());
}
private HealthIndicatorResult fetchShardsCapacityIndicatorResult(InternalTestCluster internalCluster) throws Exception {
ensureStableCluster(internalCluster.getNodeNames().length);
var healthNode = ESIntegTestCase.waitAndGetHealthNode(internalCluster);
private HealthIndicatorResult fetchShardsCapacityIndicatorResult() throws Exception {
ensureStableCluster(internalCluster().getNodeNames().length);
var healthNode = ESIntegTestCase.waitAndGetHealthNode(internalCluster());
assertNotNull(healthNode);
var randomNode = internalCluster.getRandomNodeName();
var randomNode = internalCluster().getRandomNodeName();
waitForShardLimitsMetadata(randomNode);
var healthService = internalCluster.getInstance(HealthService.class, randomNode);
var healthService = internalCluster().getInstance(HealthService.class, randomNode);
var healthIndicatorResults = getHealthServiceResults(healthService, randomNode);
assertThat(healthIndicatorResults, hasSize(1));
return healthIndicatorResults.get(0);

View file

@ -36,7 +36,7 @@ import java.util.function.Function;
import static org.hamcrest.Matchers.equalTo;
@ESIntegTestCase.ClusterScope(numDataNodes = 0, numClientNodes = 0, scope = ESIntegTestCase.Scope.TEST)
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class IngestStatsNamesAndTypesIT extends ESIntegTestCase {
@Override

View file

@ -52,7 +52,7 @@ import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numClientNodes = 1, numDataNodes = 0)
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class RestControllerIT extends ESIntegTestCase {
@Override
protected boolean addMockHttpTransport() {

View file

@ -30,7 +30,6 @@ import java.util.List;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class RetrieverTelemetryIT extends ESIntegTestCase {
private static final String INDEX_NAME = "test_index";

View file

@ -173,6 +173,8 @@ public class BlobStoreIncrementalityIT extends AbstractSnapshotIntegTestCase {
}
public void testRecordCorrectSegmentCountsWithBackgroundMerges() throws Exception {
internalCluster().startMasterOnlyNode();
internalCluster().ensureAtLeastNumDataNodes(2);
final String repoName = "test-repo";
createRepository(repoName, "fs");

View file

@ -1480,6 +1480,7 @@ public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase {
}
public void testConcurrentRestoreDeleteAndClone() throws Exception {
internalCluster().startNode();
final String repository = "test-repo";
createRepository(logger, repository, "fs");
@ -2137,7 +2138,7 @@ public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase {
}
public void testDeleteIndexWithOutOfOrderFinalization() {
internalCluster().startNode();
final var indexToDelete = "index-to-delete";
final var indexNames = List.of(indexToDelete, "index-0", "index-1", "index-2");

View file

@ -685,6 +685,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
}
public void testSnapshotWithDateMath() {
internalCluster().startMasterOnlyNode();
final String repo = "repo";
final String snapshotName = "<snapshot-{now/d}>";
@ -712,6 +713,8 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
}
public void testSnapshotTotalAndIncrementalSizes() throws Exception {
internalCluster().startMasterOnlyNode();
internalCluster().startDataOnlyNode();
final String indexName = "test-blocks-1";
final String repositoryName = "repo-" + indexName;
final String snapshot0 = "snapshot-0";
@ -798,6 +801,8 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
}
public void testDeduplicateIndexMetadata() throws Exception {
internalCluster().startMasterOnlyNode();
internalCluster().startDataOnlyNode();
final String indexName = "test-blocks-1";
final String repositoryName = "repo-" + indexName;
final String snapshot0 = "snapshot-0";
@ -968,6 +973,8 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
}
public void testRetentionLeasesClearedOnRestore() throws Exception {
internalCluster().startMasterOnlyNode();
internalCluster().startDataOnlyNode();
final String repoName = "test-repo-retention-leases";
createRepository(repoName, "fs");

View file

@ -580,7 +580,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
internalCluster().clearDisruptionScheme();
}
try {
if (cluster() != null) {
if (cluster() != null && cluster().size() > 0) {
if (currentClusterScope != Scope.TEST) {
Metadata metadata = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState().getMetadata();

View file

@ -831,14 +831,11 @@ public final class InternalTestCluster extends TestCluster {
public Client client() {
/* Randomly return a client to one of the nodes in the cluster */
NodeAndClient c = getRandomNodeAndClient();
ensureOpen();
if (c == null) {
synchronized (this) {
return getOrBuildRandomNode().client();
}
} else {
return c.client();
throw new AssertionError("Unable to get client, no node found");
}
ensureOpen();
return c.client();
}
/**

View file

@ -10,7 +10,6 @@ package org.elasticsearch.xpack.autoscaling.action;
import org.apache.logging.log4j.Level;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.monitor.os.OsProbe;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.MockLog;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.xpack.autoscaling.AutoscalingIntegTestCase;
@ -28,7 +27,6 @@ import static org.hamcrest.Matchers.greaterThan;
value = "org.elasticsearch.xpack.autoscaling.action.TransportGetAutoscalingCapacityAction:debug",
reason = "to ensure we log autoscaling capacity response on DEBUG level"
)
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class TransportGetAutoscalingCapacityActionIT extends AutoscalingIntegTestCase {
public void testCurrentCapacity() throws Exception {

View file

@ -37,7 +37,6 @@ import static org.hamcrest.Matchers.endsWith;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0)
public class IndexTemplateRegistryRolloverIT extends ESIntegTestCase {
private ClusterService clusterService;

View file

@ -33,8 +33,6 @@ public class ClusterStateLicenseServiceClusterTests extends AbstractLicensesInte
}
public void testClusterRestartWithLicense() throws Exception {
wipeAllLicenses();
int numNodes = randomIntBetween(1, 5);
logger.info("--> starting {} node(s)", numNodes);
internalCluster().startNodes(numNodes);
@ -79,7 +77,6 @@ public class ClusterStateLicenseServiceClusterTests extends AbstractLicensesInte
}
public void testClusterRestartWhileEnabled() throws Exception {
wipeAllLicenses();
internalCluster().startNode();
ensureGreen();
assertLicenseActive(true);
@ -91,7 +88,6 @@ public class ClusterStateLicenseServiceClusterTests extends AbstractLicensesInte
}
public void testClusterRestartWhileExpired() throws Exception {
wipeAllLicenses();
internalCluster().startNode();
ensureGreen();
assertLicenseActive(true);
@ -106,7 +102,6 @@ public class ClusterStateLicenseServiceClusterTests extends AbstractLicensesInte
public void testClusterRestartWithOldSignature() throws Exception {
assumeFalse("Can't run in a FIPS JVM. We can't generate old licenses since PBEWithSHA1AndDESede is not available", inFipsJvm());
wipeAllLicenses();
internalCluster().startNode();
ensureGreen();
assertLicenseActive(true);

View file

@ -27,7 +27,6 @@ import org.elasticsearch.search.internal.ReaderContext;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskInfo;
import org.elasticsearch.test.ESIntegTestCase;
import java.io.IOException;
import java.nio.file.Path;
@ -40,7 +39,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.hasSize;
@ -48,7 +46,6 @@ import static org.hamcrest.Matchers.hasSize;
/**
* IT tests that can block EQL execution at different places
*/
@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 0, numClientNodes = 0, maxNumDataNodes = 0)
public abstract class AbstractEqlBlockingIntegTestCase extends AbstractEqlIntegTestCase {
protected List<SearchBlockPlugin> initBlockFactory(boolean searchBlock, boolean fieldCapsBlock) {

View file

@ -16,9 +16,6 @@ import org.elasticsearch.xpack.core.XPackSettings;
import java.util.Collection;
import java.util.Collections;
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 0, numClientNodes = 0, maxNumDataNodes = 0)
public abstract class AbstractEqlIntegTestCase extends ESIntegTestCase {
@Override

View file

@ -29,6 +29,7 @@ import org.elasticsearch.script.MockScriptPlugin;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xpack.core.XPackPlugin;
import org.elasticsearch.xpack.core.async.AsyncExecutionId;
import org.elasticsearch.xpack.core.async.DeleteAsyncResultRequest;
@ -64,6 +65,7 @@ import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 0)
public class AsyncEqlSearchActionIT extends AbstractEqlBlockingIntegTestCase {
private final ExecutorService executorService = Executors.newFixedThreadPool(1);
@ -81,6 +83,7 @@ public class AsyncEqlSearchActionIT extends AbstractEqlBlockingIntegTestCase {
}
private void prepareIndex() throws Exception {
internalCluster().startNode();
assertAcked(
indicesAdmin().prepareCreate("test")
.setMapping("val", "type=integer", "event_type", "type=keyword", "@timestamp", "type=date", "i", "type=integer")
@ -108,6 +111,7 @@ public class AsyncEqlSearchActionIT extends AbstractEqlBlockingIntegTestCase {
}
public void testBasicAsyncExecution() throws Exception {
internalCluster().startNode();
prepareIndex();
boolean success = randomBoolean();
@ -160,6 +164,7 @@ public class AsyncEqlSearchActionIT extends AbstractEqlBlockingIntegTestCase {
}
public void testGoingAsync() throws Exception {
internalCluster().startNode();
prepareIndex();
boolean success = randomBoolean();
@ -218,6 +223,7 @@ public class AsyncEqlSearchActionIT extends AbstractEqlBlockingIntegTestCase {
}
public void testAsyncCancellation() throws Exception {
internalCluster().startNode();
prepareIndex();
boolean success = randomBoolean();
@ -261,6 +267,7 @@ public class AsyncEqlSearchActionIT extends AbstractEqlBlockingIntegTestCase {
}
public void testFinishingBeforeTimeout() throws Exception {
internalCluster().startNode();
prepareIndex();
boolean success = randomBoolean();

View file

@ -17,6 +17,7 @@ import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.xpack.core.async.GetAsyncResultRequest;
import org.elasticsearch.xpack.eql.plugin.EqlAsyncGetResultAction;
@ -32,6 +33,7 @@ import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 0)
public class PartialSearchResultsIT extends AbstractEqlIntegTestCase {
@Override

View file

@ -34,7 +34,6 @@ import java.util.List;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class TextSimilarityRankRetrieverTelemetryTests extends ESIntegTestCase {
private static final String INDEX_NAME = "test_index";

View file

@ -26,6 +26,7 @@ import org.elasticsearch.persistent.PersistentTasksCustomMetadata;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xcontent.XContentType;
import org.elasticsearch.xpack.core.XPackField;
import org.elasticsearch.xpack.core.ml.MachineLearningField;
@ -74,6 +75,7 @@ import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.nullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class MachineLearningLicensingIT extends BaseMlIntegTestCase {
public static final Set<String> RELATED_TASKS = Set.of(MlTasks.DATAFEED_TASK_NAME, MlTasks.JOB_TASK_NAME);

View file

@ -22,6 +22,7 @@ import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xcontent.XContentType;
import org.elasticsearch.xpack.core.ml.MlTasks;
import org.elasticsearch.xpack.core.ml.action.CloseJobAction;
@ -63,6 +64,7 @@ import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasEntry;
import static org.hamcrest.Matchers.notNullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class BasicDistributedJobsIT extends BaseMlIntegTestCase {
public void testFailOverBasics() throws Exception {

View file

@ -15,6 +15,7 @@ import org.elasticsearch.core.TimeValue;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.metrics.Max;
import org.elasticsearch.search.aggregations.metrics.Min;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xpack.ml.aggs.categorization.CategorizeTextAggregationBuilder;
import org.elasticsearch.xpack.ml.aggs.categorization.InternalCategorizationAggregation;
import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase;
@ -27,13 +28,13 @@ import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notANumber;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 3)
public class CategorizeTextAggregationIT extends BaseMlIntegTestCase {
private static final String DATA_INDEX = "categorization-agg-data";
@Before
public void setupCluster() {
internalCluster().ensureAtLeastNumDataNodes(3);
ensureStableCluster();
createSourceData();
}

View file

@ -15,6 +15,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xpack.ml.aggs.categorization.CategorizeTextAggregationBuilder;
import org.elasticsearch.xpack.ml.aggs.categorization.InternalCategorizationAggregation;
import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase;
@ -32,6 +33,7 @@ import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class CategorizeTextDistributedIT extends BaseMlIntegTestCase {
/**

View file

@ -14,6 +14,7 @@ import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.indices.TestIndexNameExpressionResolver;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.ClientHelper;
import org.elasticsearch.xpack.core.ml.action.PutJobAction;
@ -36,6 +37,7 @@ import java.util.concurrent.atomic.AtomicReference;
import static org.hamcrest.CoreMatchers.equalTo;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class EstablishedMemUsageIT extends BaseMlIntegTestCase {
private final long bucketSpan = AnalysisConfig.Builder.DEFAULT_BUCKET_SPAN.getMillis();

View file

@ -20,6 +20,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.TestIndexNameExpressionResolver;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.ClientHelper;
import org.elasticsearch.xpack.core.action.util.QueryPage;
@ -51,6 +52,7 @@ import static org.hamcrest.Matchers.nullValue;
/**
* Test that ML does not touch unnecessary indices when removing job index aliases
*/
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class JobStorageDeletionTaskIT extends BaseMlIntegTestCase {
private static final long bucketSpan = AnalysisConfig.Builder.DEFAULT_BUCKET_SPAN.getMillis();

View file

@ -13,6 +13,7 @@ import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.cluster.node.DiscoveryNodeRole;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xcontent.XContentBuilder;
import org.elasticsearch.xcontent.json.JsonXContent;
import org.elasticsearch.xpack.core.ml.MlTasks;
@ -55,6 +56,7 @@ import static org.hamcrest.Matchers.nullValue;
* Tests that involve interactions of ML jobs that are persistent tasks
* and trained models.
*/
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class JobsAndModelsIT extends BaseMlIntegTestCase {
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/103588")

View file

@ -17,6 +17,7 @@ import org.elasticsearch.core.Predicates;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xcontent.XContentParser;
import org.elasticsearch.xcontent.json.JsonXContent;
import org.elasticsearch.xpack.core.ml.action.PutTrainedModelAction;
@ -42,6 +43,7 @@ import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class LearningToRankExplainIT extends BaseMlIntegTestCase {
private static final String LTR_SEARCH_INDEX = "ltr-search-index";

View file

@ -29,6 +29,8 @@ import org.elasticsearch.persistent.PersistentTaskResponse;
import org.elasticsearch.persistent.PersistentTasksClusterService;
import org.elasticsearch.persistent.PersistentTasksCustomMetadata;
import org.elasticsearch.persistent.UpdatePersistentTaskStatusAction;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentBuilder;
import org.elasticsearch.xcontent.XContentParser;
@ -64,8 +66,6 @@ import org.elasticsearch.xpack.ml.MachineLearning;
import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister;
import org.elasticsearch.xpack.ml.job.process.autodetect.BlackHoleAutodetectProcess;
import org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase;
import org.junit.After;
import org.junit.Before;
import java.io.IOException;
import java.util.Collections;
@ -91,6 +91,8 @@ import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
@TestLogging(value = "org.elasticsearch.xpack.ml.utils.persistence:TRACE", reason = "")
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class MlDistributedFailureIT extends BaseMlIntegTestCase {
@Override
@ -113,16 +115,6 @@ public class MlDistributedFailureIT extends BaseMlIntegTestCase {
});
}
@Before
public void setLogging() {
updateClusterSettings(Settings.builder().put("logger.org.elasticsearch.xpack.ml.utils.persistence", "TRACE"));
}
@After
public void unsetLogging() {
updateClusterSettings(Settings.builder().putNull("logger.org.elasticsearch.xpack.ml.utils.persistence"));
}
public void testLoseDedicatedMasterNode() throws Exception {
internalCluster().ensureAtMostNumDataNodes(0);
logger.info("Starting dedicated master node...");

View file

@ -12,6 +12,7 @@ import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xpack.core.action.util.QueryPage;
import org.elasticsearch.xpack.core.ml.action.CloseJobAction;
import org.elasticsearch.xpack.core.ml.action.GetJobsStatsAction;
@ -33,6 +34,7 @@ import static org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata.Type
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class MlNodeShutdownIT extends BaseMlIntegTestCase {
public void testJobsVacateShuttingDownNode() throws Exception {

View file

@ -12,6 +12,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.disruption.NetworkDisruption;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.xpack.core.ml.action.CloseJobAction;
@ -30,6 +31,7 @@ import java.util.Set;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class NetworkDisruptionIT extends BaseMlIntegTestCase {
@Override

View file

@ -15,6 +15,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.persistent.PersistentTasksCustomMetadata;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.ml.MachineLearningField;
import org.elasticsearch.xpack.core.ml.MlTasks;
@ -31,6 +32,7 @@ import org.elasticsearch.xpack.ml.utils.NativeMemoryCalculator;
import java.util.List;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class TooManyJobsIT extends BaseMlIntegTestCase {
public void testCloseFailedJob() throws Exception {

View file

@ -122,7 +122,6 @@ import static org.mockito.Mockito.when;
* Note for other type of integration tests you should use the external test cluster created by the Gradle integTest task.
* For example tests extending this base class test with the non native autodetect process.
*/
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0, supportsDedicatedMasters = false)
public abstract class BaseMlIntegTestCase extends ESIntegTestCase {
// The ML jobs can trigger many tasks that are not easily tracked. For this reason, here we list
@ -171,7 +170,9 @@ public abstract class BaseMlIntegTestCase extends ESIntegTestCase {
@Before
public void ensureTemplatesArePresent() throws Exception {
awaitClusterState(logger, MachineLearning::criticalTemplatesInstalled);
if (cluster().size() > 0) {
awaitClusterState(logger, MachineLearning::criticalTemplatesInstalled);
}
}
protected Job.Builder createJob(String id) {

View file

@ -36,7 +36,7 @@ import java.util.List;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class RRFRetrieverTelemetryIT extends ESIntegTestCase {
private static final String INDEX_NAME = "test_index";

View file

@ -36,7 +36,7 @@ import static org.hamcrest.Matchers.iterableWithSize;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.nullValue;
@ESIntegTestCase.ClusterScope(numDataNodes = 0, numClientNodes = 0, scope = ESIntegTestCase.Scope.TEST)
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public class ApiKeyOwnerProfileIntegTests extends SecurityIntegTestCase {
public static final SecureString FILE_USER_TEST_PASSWORD = new SecureString("file-user-test-password".toCharArray());

View file

@ -125,6 +125,7 @@ public class CleanupRoleMappingDuplicatesMigrationIT extends SecurityIntegTestCa
public void testMigrationSuccessful() throws Exception {
internalCluster().setBootstrapMasterNodeIndex(0);
internalCluster().startNode();
ensureGreen();
final String masterNode = internalCluster().getMasterName();
@ -158,6 +159,7 @@ public class CleanupRoleMappingDuplicatesMigrationIT extends SecurityIntegTestCa
public void testMigrationSuccessfulNoOverlap() throws Exception {
internalCluster().setBootstrapMasterNodeIndex(0);
internalCluster().startNode();
ensureGreen();
final String masterNode = internalCluster().getMasterName();
@ -191,6 +193,7 @@ public class CleanupRoleMappingDuplicatesMigrationIT extends SecurityIntegTestCa
public void testMigrationSuccessfulNoNative() throws Exception {
internalCluster().setBootstrapMasterNodeIndex(0);
internalCluster().startNode();
ensureGreen();
final String masterNode = internalCluster().getMasterName();
@ -221,6 +224,7 @@ public class CleanupRoleMappingDuplicatesMigrationIT extends SecurityIntegTestCa
public void testMigrationFallbackNamePreCondition() throws Exception {
internalCluster().setBootstrapMasterNodeIndex(0);
internalCluster().startNode();
ensureGreen();
final String masterNode = internalCluster().getMasterName();
// Wait for file watcher to start
@ -256,6 +260,7 @@ public class CleanupRoleMappingDuplicatesMigrationIT extends SecurityIntegTestCa
public void testSkipMigrationNoFileBasedMappings() throws Exception {
internalCluster().setBootstrapMasterNodeIndex(0);
internalCluster().startNode();
ensureGreen();
// Create a native role mapping to create security index and trigger migration (skipped initially)
createNativeRoleMapping("everyone_kibana_alone");
@ -275,6 +280,7 @@ public class CleanupRoleMappingDuplicatesMigrationIT extends SecurityIntegTestCa
public void testSkipMigrationEmptyFileBasedMappings() throws Exception {
internalCluster().setBootstrapMasterNodeIndex(0);
internalCluster().startNode();
ensureGreen();
final String masterNode = internalCluster().getMasterName();
@ -304,6 +310,7 @@ public class CleanupRoleMappingDuplicatesMigrationIT extends SecurityIntegTestCa
public void testNewIndexSkipMigration() {
internalCluster().setBootstrapMasterNodeIndex(0);
internalCluster().startNode();
final String masterNode = internalCluster().getMasterName();
ensureGreen();
deleteSecurityIndex(); // hack to force a new security index to be created

View file

@ -70,6 +70,7 @@ public class IpFilteringIntegrationTests extends SecurityIntegTestCase {
}
public void testThatIpFilteringIsIntegratedIntoNettyPipelineViaHttp() throws Exception {
internalCluster().startNode();
TransportAddress transportAddress = randomFrom(
internalCluster().getDataNodeInstance(HttpServerTransport.class).boundAddress().boundAddresses()
);
@ -80,6 +81,7 @@ public class IpFilteringIntegrationTests extends SecurityIntegTestCase {
}
public void testThatIpFilteringIsAppliedForProfile() throws Exception {
internalCluster().startNode();
try (Socket socket = new Socket()) {
trySocketConnection(socket, getProfileAddress("client"));
assertThat(socket.isClosed(), is(true));

View file

@ -179,7 +179,9 @@ public abstract class SecurityIntegTestCase extends ESIntegTestCase {
@Before
// before methods from the superclass are run before this, which means that the current cluster is ready to go
public void assertXPackIsInstalled() {
doAssertXPackIsInstalled();
if (cluster().size() > 0) {
doAssertXPackIsInstalled();
}
}
protected void doAssertXPackIsInstalled() {

View file

@ -42,7 +42,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.hasSize;
@ -50,7 +49,7 @@ import static org.hamcrest.Matchers.hasSize;
/**
* IT tests that can block SQL execution at different places
*/
@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 0, numClientNodes = 0, maxNumDataNodes = 0)
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false)
public abstract class AbstractSqlBlockingIntegTestCase extends ESIntegTestCase {
@Override

View file

@ -14,6 +14,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.search.SearchContextMissingException;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.test.ESIntegTestCase;
import java.util.Arrays;
@ -21,6 +22,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcke
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.instanceOf;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 0)
public class SqlSearchPageTimeoutIT extends AbstractSqlIntegTestCase {
@Override
@ -40,6 +42,7 @@ public class SqlSearchPageTimeoutIT extends AbstractSqlIntegTestCase {
}
public void testSearchContextIsCleanedUpAfterPageTimeout(String query) throws Exception {
internalCluster().startNode();
setupTestIndex();
SqlQueryResponse response = new SqlQueryRequestBuilder(client()).query(query)

View file

@ -15,9 +15,6 @@ import org.elasticsearch.xpack.core.XPackSettings;
import java.util.Collection;
import java.util.Collections;
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 0, numClientNodes = 0, maxNumDataNodes = 0)
public abstract class AbstractSqlIntegTestCase extends ESIntegTestCase {
@Override