1
0
Эх сурвалжийг харах

Fix up misc master-node timeouts in xpack/plugin (#109232)

More simple cases of #107984.
David Turner 1 жил өмнө
parent
commit
2c0ad093ef
35 өөрчлөгдсөн 216 нэмэгдсэн , 104 устгасан
  1. 1 0
      test/external-modules/latency-simulating-directory/src/internalClusterTest/java/org/elasticsearch/test/simulatedlatencyrepo/LatencySimulatingBlobStoreRepositoryTests.java
  2. 1 0
      x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/AbstractFrozenAutoscalingIntegTestCase.java
  3. 3 2
      x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderIT.java
  4. 1 1
      x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/core/action/DataStreamLifecycleUsageTransportActionIT.java
  5. 3 2
      x-pack/plugin/core/src/main/java/org/elasticsearch/protocol/xpack/XPackUsageRequest.java
  6. 3 2
      x-pack/plugin/core/src/main/java/org/elasticsearch/protocol/xpack/frozen/FreezeRequest.java
  7. 1 1
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportXPackUsageAction.java
  8. 0 26
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageRequestBuilder.java
  9. 1 1
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MountSnapshotStep.java
  10. 17 17
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rest/action/RestXPackUsageAction.java
  11. 5 1
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/MountSearchableSnapshotRequest.java
  12. 35 7
      x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java
  13. 4 1
      x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexRecoveryTests.java
  14. 72 22
      x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java
  15. 5 3
      x-pack/plugin/frozen-indices/src/main/java/org/elasticsearch/xpack/frozen/rest/action/RestFreezeIndexAction.java
  16. 6 3
      x-pack/plugin/monitoring/src/internalClusterTest/java/org/elasticsearch/xpack/monitoring/integration/MonitoringIT.java
  17. 7 4
      x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsCollector.java
  18. 2 2
      x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsCollectorTests.java
  19. 4 2
      x-pack/plugin/old-lucene-versions/src/internalClusterTest/java/org/elasticsearch/xpack/lucene/bwc/ArchiveLicenseIntegTests.java
  20. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java
  21. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/ClusterStateApplierOrderingTests.java
  22. 3 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java
  23. 3 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java
  24. 7 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java
  25. 2 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsLicenseIntegTests.java
  26. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsSettingValidationIntegTests.java
  27. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsSystemIndicesIntegTests.java
  28. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsUuidValidationIntegTests.java
  29. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/allocation/AllocationFilteringIntegTests.java
  30. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotDiskThresholdIntegTests.java
  31. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/full/SearchableSnapshotsPrewarmingIntegTests.java
  32. 1 0
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/PartiallyCachedShardAllocationIntegTests.java
  33. 12 0
      x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/MountSearchableSnapshotRequestTests.java
  34. 3 2
      x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java
  35. 6 5
      x-pack/qa/freeze-plugin/src/main/java/org/elasticsearch/plugin/freeze/FreezeIndexPlugin.java

+ 1 - 0
test/external-modules/latency-simulating-directory/src/internalClusterTest/java/org/elasticsearch/test/simulatedlatencyrepo/LatencySimulatingBlobStoreRepositoryTests.java

@@ -123,6 +123,7 @@ public class LatencySimulatingBlobStoreRepositoryTests extends AbstractSnapshotI
 
         logger.info("--> mount snapshot");
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             "test-idx",
             repositoryName,
             si.snapshotId().getName(),

+ 1 - 0
x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/AbstractFrozenAutoscalingIntegTestCase.java

@@ -97,6 +97,7 @@ public abstract class AbstractFrozenAutoscalingIntegTestCase extends AbstractSna
         assertThat(total.storage(), equalTo(ByteSizeValue.ZERO));
 
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             fsRepoName,
             snapshotInfo.snapshotId().getName(),

+ 3 - 2
x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderIT.java

@@ -25,8 +25,9 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.protocol.xpack.XPackUsageRequest;
 import org.elasticsearch.test.ESIntegTestCase;
-import org.elasticsearch.xpack.core.action.XPackUsageRequestBuilder;
+import org.elasticsearch.xpack.core.action.XPackUsageAction;
 import org.elasticsearch.xpack.core.action.XPackUsageResponse;
 import org.elasticsearch.xpack.core.datatiers.DataTiersFeatureSetUsage;
 import org.junit.Before;
@@ -417,7 +418,7 @@ public class DataTierAllocationDeciderIT extends ESIntegTestCase {
     }
 
     private DataTiersFeatureSetUsage getUsage() {
-        XPackUsageResponse usages = new XPackUsageRequestBuilder(client()).get();
+        XPackUsageResponse usages = safeGet(client().execute(XPackUsageAction.INSTANCE, new XPackUsageRequest(SAFE_AWAIT_TIMEOUT)));
         return usages.getUsages()
             .stream()
             .filter(u -> u instanceof DataTiersFeatureSetUsage)

+ 1 - 1
x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/core/action/DataStreamLifecycleUsageTransportActionIT.java

@@ -169,7 +169,7 @@ public class DataStreamLifecycleUsageTransportActionIT extends ESIntegTestCase {
         double averageRetention,
         boolean defaultRolloverUsed
     ) throws Exception {
-        XPackUsageFeatureResponse response = client().execute(DATA_STREAM_LIFECYCLE, new XPackUsageRequest()).get();
+        XPackUsageFeatureResponse response = safeGet(client().execute(DATA_STREAM_LIFECYCLE, new XPackUsageRequest(SAFE_AWAIT_TIMEOUT)));
         XContentBuilder builder = XContentFactory.jsonBuilder();
         builder = response.getUsage().toXContent(builder, ToXContent.EMPTY_PARAMS);
         Tuple<XContentType, Map<String, Object>> tuple = XContentHelper.convertToMap(

+ 3 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/protocol/xpack/XPackUsageRequest.java

@@ -9,6 +9,7 @@ package org.elasticsearch.protocol.xpack;
 import org.elasticsearch.action.ActionRequestValidationException;
 import org.elasticsearch.action.support.master.MasterNodeRequest;
 import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.tasks.CancellableTask;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskId;
@@ -18,8 +19,8 @@ import java.util.Map;
 
 public class XPackUsageRequest extends MasterNodeRequest<XPackUsageRequest> {
 
-    public XPackUsageRequest() {
-        super(TRAPPY_IMPLICIT_DEFAULT_MASTER_NODE_TIMEOUT);
+    public XPackUsageRequest(TimeValue masterNodeTimeout) {
+        super(masterNodeTimeout);
     }
 
     public XPackUsageRequest(StreamInput in) throws IOException {

+ 3 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/protocol/xpack/frozen/FreezeRequest.java

@@ -15,6 +15,7 @@ import org.elasticsearch.action.support.master.AcknowledgedRequest;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.util.CollectionUtils;
+import org.elasticsearch.core.TimeValue;
 
 import java.io.IOException;
 
@@ -26,8 +27,8 @@ public class FreezeRequest extends AcknowledgedRequest<FreezeRequest> implements
     private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpen();
     private ActiveShardCount waitForActiveShards = ActiveShardCount.DEFAULT;
 
-    public FreezeRequest(String... indices) {
-        super(TRAPPY_IMPLICIT_DEFAULT_MASTER_NODE_TIMEOUT, DEFAULT_ACK_TIMEOUT);
+    public FreezeRequest(TimeValue masterNodeTimeout, TimeValue ackTimeout, String... indices) {
+        super(masterNodeTimeout, ackTimeout);
         this.indices = indices;
     }
 

+ 1 - 1
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportXPackUsageAction.java

@@ -69,7 +69,7 @@ public class TransportXPackUsageAction extends TransportMasterNodeAction<XPackUs
             @Override
             protected void doRun() {
                 if (responses.size() < usageActions().size()) {
-                    final var childRequest = new XPackUsageRequest();
+                    final var childRequest = new XPackUsageRequest(request.masterNodeTimeout());
                     childRequest.setParentTask(request.getParentTask());
                     client.executeLocally(
                         usageActions.get(responses.size()),

+ 0 - 26
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageRequestBuilder.java

@@ -1,26 +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; you may not use this file except in compliance with the Elastic License
- * 2.0.
- */
-package org.elasticsearch.xpack.core.action;
-
-import org.elasticsearch.action.ActionType;
-import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder;
-import org.elasticsearch.client.internal.ElasticsearchClient;
-import org.elasticsearch.protocol.xpack.XPackUsageRequest;
-
-public class XPackUsageRequestBuilder extends MasterNodeOperationRequestBuilder<
-    XPackUsageRequest,
-    XPackUsageResponse,
-    XPackUsageRequestBuilder> {
-
-    public XPackUsageRequestBuilder(ElasticsearchClient client) {
-        this(client, XPackUsageAction.INSTANCE);
-    }
-
-    public XPackUsageRequestBuilder(ElasticsearchClient client, ActionType<XPackUsageResponse> action) {
-        super(client, action, new XPackUsageRequest());
-    }
-}

+ 1 - 1
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/MountSnapshotStep.java

@@ -142,6 +142,7 @@ public class MountSnapshotStep extends AsyncRetryDuringSnapshotActionStep {
         overrideTierPreference(this.getKey().phase()).ifPresent(override -> settingsBuilder.put(DataTier.TIER_PREFERENCE, override));
 
         final MountSearchableSnapshotRequest mountSearchableSnapshotRequest = new MountSearchableSnapshotRequest(
+            TimeValue.MAX_VALUE,
             mountedIndexName,
             snapshotRepository,
             snapshotName,
@@ -153,7 +154,6 @@ public class MountSnapshotStep extends AsyncRetryDuringSnapshotActionStep {
             false,
             storageType
         );
-        mountSearchableSnapshotRequest.masterNodeTimeout(TimeValue.MAX_VALUE);
         getClient().execute(
             MountSearchableSnapshotAction.INSTANCE,
             mountSearchableSnapshotRequest,

+ 17 - 17
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rest/action/RestXPackUsageAction.java

@@ -7,18 +7,18 @@
 package org.elasticsearch.xpack.core.rest.action;
 
 import org.elasticsearch.client.internal.node.NodeClient;
-import org.elasticsearch.core.TimeValue;
-import org.elasticsearch.http.HttpChannel;
+import org.elasticsearch.protocol.xpack.XPackUsageRequest;
 import org.elasticsearch.rest.BaseRestHandler;
 import org.elasticsearch.rest.RestRequest;
 import org.elasticsearch.rest.RestResponse;
+import org.elasticsearch.rest.RestUtils;
 import org.elasticsearch.rest.Scope;
 import org.elasticsearch.rest.ServerlessScope;
 import org.elasticsearch.rest.action.RestBuilderListener;
 import org.elasticsearch.rest.action.RestCancellableNodeClient;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xpack.core.XPackFeatureSet;
-import org.elasticsearch.xpack.core.action.XPackUsageRequestBuilder;
+import org.elasticsearch.xpack.core.action.XPackUsageAction;
 import org.elasticsearch.xpack.core.action.XPackUsageResponse;
 
 import java.io.IOException;
@@ -26,7 +26,6 @@ import java.util.List;
 
 import static org.elasticsearch.rest.RestRequest.Method.GET;
 import static org.elasticsearch.rest.RestStatus.OK;
-import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout;
 
 @ServerlessScope(Scope.INTERNAL)
 public class RestXPackUsageAction extends BaseRestHandler {
@@ -43,20 +42,21 @@ public class RestXPackUsageAction extends BaseRestHandler {
 
     @Override
     public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
-        final TimeValue masterTimeout = getMasterNodeTimeout(request);
-        final HttpChannel httpChannel = request.getHttpChannel();
-        return channel -> new XPackUsageRequestBuilder(new RestCancellableNodeClient(client, httpChannel)).setMasterNodeTimeout(
-            masterTimeout
-        ).execute(new RestBuilderListener<>(channel) {
-            @Override
-            public RestResponse buildResponse(XPackUsageResponse response, XContentBuilder builder) throws Exception {
-                builder.startObject();
-                for (XPackFeatureSet.Usage usage : response.getUsages()) {
-                    builder.field(usage.name(), usage);
+        final var usageRequest = new XPackUsageRequest(RestUtils.getMasterNodeTimeout(request));
+        return channel -> new RestCancellableNodeClient(client, request.getHttpChannel()).execute(
+            XPackUsageAction.INSTANCE,
+            usageRequest,
+            new RestBuilderListener<>(channel) {
+                @Override
+                public RestResponse buildResponse(XPackUsageResponse response, XContentBuilder builder) throws Exception {
+                    builder.startObject();
+                    for (XPackFeatureSet.Usage usage : response.getUsages()) {
+                        builder.field(usage.name(), usage);
+                    }
+                    builder.endObject();
+                    return new RestResponse(OK, builder);
                 }
-                builder.endObject();
-                return new RestResponse(OK, builder);
             }
-        });
+        );
     }
 }

+ 5 - 1
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/MountSearchableSnapshotRequest.java

@@ -16,7 +16,9 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.rest.RestRequest;
+import org.elasticsearch.rest.RestUtils;
 import org.elasticsearch.xcontent.ConstructingObjectParser;
 import org.elasticsearch.xcontent.ObjectParser;
 import org.elasticsearch.xcontent.ParseField;
@@ -38,6 +40,7 @@ public class MountSearchableSnapshotRequest extends MasterNodeRequest<MountSearc
         "mount_searchable_snapshot",
         false,
         (a, request) -> new MountSearchableSnapshotRequest(
+            RestUtils.getMasterNodeTimeout(request),
             Objects.requireNonNullElse((String) a[1], (String) a[0]),
             Objects.requireNonNull(request.param("repository")),
             Objects.requireNonNull(request.param("snapshot")),
@@ -92,6 +95,7 @@ public class MountSearchableSnapshotRequest extends MasterNodeRequest<MountSearc
      * Constructs a new mount searchable snapshot request, restoring an index with the settings needed to make it a searchable snapshot.
      */
     public MountSearchableSnapshotRequest(
+        TimeValue masterNodeTimeout,
         String mountedIndexName,
         String repositoryName,
         String snapshotName,
@@ -101,7 +105,7 @@ public class MountSearchableSnapshotRequest extends MasterNodeRequest<MountSearc
         boolean waitForCompletion,
         Storage storage
     ) {
-        super(TRAPPY_IMPLICIT_DEFAULT_MASTER_NODE_TIMEOUT);
+        super(masterNodeTimeout);
         this.mountedIndexName = Objects.requireNonNull(mountedIndexName);
         this.repositoryName = Objects.requireNonNull(repositoryName);
         this.snapshotName = Objects.requireNonNull(snapshotName);

+ 35 - 7
x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java

@@ -90,7 +90,12 @@ public class FrozenIndexIT extends ESIntegTestCase {
 
         assertThat(client().prepareDelete("index", indexResponse.getId()).get().status(), equalTo(RestStatus.OK));
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index").waitForActiveShards(ActiveShardCount.ONE)));
+        assertAcked(
+            client().execute(
+                FreezeIndexAction.INSTANCE,
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index").waitForActiveShards(ActiveShardCount.ONE)
+            )
+        );
 
         assertThat(
             clusterAdmin().prepareState().get().getState().metadata().index("index").getTimestampRange(),
@@ -179,7 +184,9 @@ public class FrozenIndexIT extends ESIntegTestCase {
             assertNull(indicesService.getTimestampFieldType(index));
         }
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index")).actionGet()
+        );
         ensureGreen("index");
         for (final IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) {
             final PlainActionFuture<DateFieldMapper.DateFieldType> timestampFieldTypeFuture = new PlainActionFuture<>();
@@ -193,7 +200,12 @@ public class FrozenIndexIT extends ESIntegTestCase {
             assertThat(timestampFieldTypeFuture.get().dateTimeFormatter().parseMillis(date), equalTo(1580817683000L));
         }
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index").setFreeze(false)).actionGet());
+        assertAcked(
+            client().execute(
+                FreezeIndexAction.INSTANCE,
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index").setFreeze(false)
+            ).actionGet()
+        );
         ensureGreen("index");
         for (final IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) {
             assertNull(indicesService.getTimestampFieldType(index));
@@ -227,7 +239,10 @@ public class FrozenIndexIT extends ESIntegTestCase {
         for (int i = 0; i < numDocs; i++) {
             prepareIndex(indexName).setSource("created_date", "2011-02-02").get();
         }
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName))
+                .actionGet()
+        );
         final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest(indexName).indicesOptions(
             IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED
         ).keepAlive(TimeValue.timeValueMinutes(2));
@@ -254,7 +269,12 @@ public class FrozenIndexIT extends ESIntegTestCase {
                 }
             );
         } finally {
-            assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName).setFreeze(false)).actionGet());
+            assertAcked(
+                client().execute(
+                    FreezeIndexAction.INSTANCE,
+                    new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName).setFreeze(false)
+                ).actionGet()
+            );
             client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
         }
     }
@@ -275,7 +295,12 @@ public class FrozenIndexIT extends ESIntegTestCase {
             prepareIndex("index-2").setId(id).setSource("value", i).get();
         }
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index-1", "index-2")).actionGet());
+        assertAcked(
+            client().execute(
+                FreezeIndexAction.INSTANCE,
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index-1", "index-2")
+            ).actionGet()
+        );
         final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("index-*").indicesOptions(
             IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED
         ).keepAlive(TimeValue.timeValueMinutes(2));
@@ -311,7 +336,10 @@ public class FrozenIndexIT extends ESIntegTestCase {
             String id = Integer.toString(i);
             prepareIndex("test-index").setId(id).setSource("value", i).get();
         }
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("test-index")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "test-index"))
+                .actionGet()
+        );
         // include the frozen indices
         {
             final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("test-*").indicesOptions(

+ 4 - 1
x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexRecoveryTests.java

@@ -74,7 +74,10 @@ public class FrozenIndexRecoveryTests extends ESIntegTestCase {
                 for (int i = 0; i < moreDocs; i++) {
                     client.prepareIndex(indexName).setSource("num", i).get();
                 }
-                assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
+                assertAcked(
+                    client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName))
+                        .actionGet()
+                );
                 return super.onNodeStopped(nodeName);
             }
         });

+ 72 - 22
x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java

@@ -97,7 +97,10 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         prepareIndex(indexName).setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
         prepareIndex(indexName).setId("2").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
         prepareIndex(indexName).setId("3").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName))
+                .actionGet()
+        );
         expectThrows(
             ClusterBlockException.class,
             prepareIndex(indexName).setId("4").setSource("field", "value").setRefreshPolicy(IMMEDIATE)
@@ -187,7 +190,10 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         for (int i = 0; i < 10; i++) {
             prepareIndex(indexName).setId("" + i).setSource("field", "foo bar baz").get();
         }
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName))
+                .actionGet()
+        );
         int numRequests = randomIntBetween(20, 50);
         int numRefreshes = 0;
         int numSearches = 0;
@@ -231,7 +237,9 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
             // sometimes close it
             assertAcked(indicesAdmin().prepareClose("index").get());
         }
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index")).actionGet()
+        );
         {
             IndicesService indexServices = getInstanceFromNode(IndicesService.class);
             Index index = resolveIndex("index");
@@ -243,7 +251,12 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
             assertEquals(0, shard.refreshStats().getTotal());
             assertThat(indexService.getMetadata().getTimestampRange(), sameInstance(IndexLongFieldRange.UNKNOWN));
         }
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index").setFreeze(false)).actionGet());
+        assertAcked(
+            client().execute(
+                FreezeIndexAction.INSTANCE,
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index").setFreeze(false)
+            ).actionGet()
+        );
         {
             IndicesService indexServices = getInstanceFromNode(IndicesService.class);
             Index index = resolveIndex("index");
@@ -269,12 +282,15 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
 
     public void testDoubleFreeze() {
         createIndex("test-idx", Settings.builder().put("index.number_of_shards", 2).build());
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("test-idx")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "test-idx"))
+                .actionGet()
+        );
         ResourceNotFoundException exception = expectThrows(
             ResourceNotFoundException.class,
             client().execute(
                 FreezeIndexAction.INSTANCE,
-                new FreezeRequest("test-idx").indicesOptions(
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "test-idx").indicesOptions(
                     IndicesOptions.builder().wildcardOptions(IndicesOptions.WildcardOptions.builder().allowEmptyExpressions(false)).build()
                 )
             )
@@ -287,12 +303,15 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         prepareIndex("idx").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
         createIndex("idx-closed", Settings.builder().put("index.number_of_shards", 1).build());
         prepareIndex("idx-closed").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("idx")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "idx")).actionGet()
+        );
         assertAcked(indicesAdmin().prepareClose("idx-closed").get());
         assertAcked(
             client().execute(
                 FreezeIndexAction.INSTANCE,
-                new FreezeRequest("idx*").setFreeze(false).indicesOptions(IndicesOptions.strictExpand())
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "idx*").setFreeze(false)
+                    .indicesOptions(IndicesOptions.strictExpand())
             )
         );
         ClusterStateResponse stateResponse = clusterAdmin().prepareState().get();
@@ -307,7 +326,10 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         prepareIndex(indexName).setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
         createIndex("test-idx-1", Settings.builder().put("index.number_of_shards", 1).build());
         prepareIndex("test-idx-1").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName))
+                .actionGet()
+        );
         assertIndexFrozen(indexName);
 
         IndicesStatsResponse index = indicesAdmin().prepareStats(indexName).clear().setRefresh(true).get();
@@ -316,7 +338,9 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         index = indicesAdmin().prepareStats(indexName).clear().setRefresh(true).get();
         assertEquals(1, index.getTotal().refresh.getTotal());
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("test*")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "test*")).actionGet()
+        );
         assertIndexFrozen(indexName);
         assertIndexFrozen("test-idx-1");
         index = indicesAdmin().prepareStats(indexName).clear().setRefresh(true).get();
@@ -361,7 +385,9 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
             );
         }
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index")).actionGet()
+        );
         {
 
             IndicesService indexServices = getInstanceFromNode(IndicesService.class);
@@ -467,7 +493,9 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
     public void testWriteToFrozenIndex() {
         createIndex("idx", Settings.builder().put("index.number_of_shards", 1).build());
         prepareIndex("idx").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("idx")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "idx")).actionGet()
+        );
         assertIndexFrozen("idx");
         expectThrows(ClusterBlockException.class, prepareIndex("idx").setId("2").setSource("field", "value").setRefreshPolicy(IMMEDIATE));
     }
@@ -479,7 +507,7 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         assertAcked(
             client().execute(
                 FreezeIndexAction.INSTANCE,
-                new FreezeRequest("idx*", "not_available").indicesOptions(
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "idx*", "not_available").indicesOptions(
                     IndicesOptions.fromParameters(null, "true", null, null, IndicesOptions.strictExpandOpen())
                 )
             )
@@ -490,14 +518,16 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
 
     public void testUnfreezeClosedIndex() {
         createIndex("idx", Settings.builder().put("index.number_of_shards", 1).build());
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("idx")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "idx")).actionGet()
+        );
         assertAcked(indicesAdmin().prepareClose("idx"));
         assertEquals(IndexMetadata.State.CLOSE, clusterAdmin().prepareState().get().getState().metadata().index("idx").getState());
         expectThrows(
             IndexNotFoundException.class,
             client().execute(
                 FreezeIndexAction.INSTANCE,
-                new FreezeRequest("id*").setFreeze(false)
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "id*").setFreeze(false)
                     .indicesOptions(
                         IndicesOptions.builder()
                             .wildcardOptions(IndicesOptions.WildcardOptions.builder().allowEmptyExpressions(false))
@@ -506,7 +536,12 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
             )
         );
         // we don't resolve to closed indices
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("idx").setFreeze(false)).actionGet());
+        assertAcked(
+            client().execute(
+                FreezeIndexAction.INSTANCE,
+                new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "idx").setFreeze(false)
+            ).actionGet()
+        );
         assertEquals(IndexMetadata.State.OPEN, clusterAdmin().prepareState().get().getState().metadata().index("idx").getState());
     }
 
@@ -517,7 +552,9 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
 
         final long settingsVersion = clusterAdmin().prepareState().get().getState().metadata().index(index).getSettingsVersion();
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(index)).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, index)).actionGet()
+        );
         assertIndexFrozen(index);
         assertThat(
             clusterAdmin().prepareState().get().getState().metadata().index(index).getSettingsVersion(),
@@ -543,7 +580,10 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
             assertThat(indexService.getShard(0).getLastKnownGlobalCheckpoint(), greaterThanOrEqualTo(nbNoOps - 1L));
         });
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName))
+                .actionGet()
+        );
         assertIndexFrozen(indexName);
     }
 
@@ -557,7 +597,10 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
             assertThat(indexResponse.status(), is(RestStatus.CREATED));
         }
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName))
+                .actionGet()
+        );
         assertIndexFrozen(indexName);
 
         final IndexMetadata indexMetadata = clusterAdmin().prepareState().get().getState().metadata().index(indexName);
@@ -602,7 +645,10 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         );
         assertThat(stats.getIndex(indexName).getPrimaries().getTranslog().getUncommittedOperations(), equalTo(uncommittedOps));
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, indexName))
+                .actionGet()
+        );
         assertIndexFrozen(indexName);
 
         IndicesOptions indicesOptions = IndicesOptions.STRICT_EXPAND_OPEN;
@@ -621,7 +667,9 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         prepareIndex("index").setSource(DataStream.TIMESTAMP_FIELD_NAME, "2010-01-05T01:02:03.456Z").get();
         prepareIndex("index").setSource(DataStream.TIMESTAMP_FIELD_NAME, "2010-01-06T02:03:04.567Z").get();
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index")).actionGet()
+        );
 
         final IndexLongFieldRange timestampFieldRange = clusterAdmin().prepareState()
             .get()
@@ -653,7 +701,9 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
         prepareIndex("index").setSource(DataStream.TIMESTAMP_FIELD_NAME, "2010-01-05T01:02:03.456789012Z").get();
         prepareIndex("index").setSource(DataStream.TIMESTAMP_FIELD_NAME, "2010-01-06T02:03:04.567890123Z").get();
 
-        assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest("index")).actionGet());
+        assertAcked(
+            client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, "index")).actionGet()
+        );
 
         final IndexLongFieldRange timestampFieldRange = clusterAdmin().prepareState()
             .get()

+ 5 - 3
x-pack/plugin/frozen-indices/src/main/java/org/elasticsearch/xpack/frozen/rest/action/RestFreezeIndexAction.java

@@ -63,9 +63,11 @@ public final class RestFreezeIndexAction extends BaseRestHandler {
             });
         }
 
-        FreezeRequest freezeRequest = new FreezeRequest(Strings.splitStringByCommaToArray(request.param("index")));
-        freezeRequest.ackTimeout(getAckTimeout(request));
-        freezeRequest.masterNodeTimeout(getMasterNodeTimeout(request));
+        final var freezeRequest = new FreezeRequest(
+            getMasterNodeTimeout(request),
+            getAckTimeout(request),
+            Strings.splitStringByCommaToArray(request.param("index"))
+        );
         freezeRequest.indicesOptions(IndicesOptions.fromRequest(request, freezeRequest.indicesOptions()));
         String waitForActiveShards = request.param("wait_for_active_shards");
         if (waitForActiveShards != null) {

+ 6 - 3
x-pack/plugin/monitoring/src/internalClusterTest/java/org/elasticsearch/xpack/monitoring/integration/MonitoringIT.java

@@ -21,6 +21,7 @@ import org.elasticsearch.core.CheckedRunnable;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.mapper.extras.MapperExtrasPlugin;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.protocol.xpack.XPackUsageRequest;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchHits;
@@ -33,7 +34,7 @@ import org.elasticsearch.xcontent.ToXContentObject;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.core.XPackSettings;
-import org.elasticsearch.xpack.core.action.XPackUsageRequestBuilder;
+import org.elasticsearch.xpack.core.action.XPackUsageAction;
 import org.elasticsearch.xpack.core.action.XPackUsageResponse;
 import org.elasticsearch.xpack.core.monitoring.MonitoredSystem;
 import org.elasticsearch.xpack.core.monitoring.MonitoringFeatureSetUsage;
@@ -427,8 +428,10 @@ public class MonitoringIT extends ESSingleNodeTestCase {
         }, 30L, TimeUnit.SECONDS);
     }
 
-    private boolean getMonitoringUsageExportersDefined() throws Exception {
-        final XPackUsageResponse usageResponse = new XPackUsageRequestBuilder(client()).execute().get();
+    private boolean getMonitoringUsageExportersDefined() {
+        final XPackUsageResponse usageResponse = safeGet(
+            client().execute(XPackUsageAction.INSTANCE, new XPackUsageRequest(SAFE_AWAIT_TIMEOUT))
+        );
         final Optional<MonitoringFeatureSetUsage> monitoringUsage = usageResponse.getUsages()
             .stream()
             .filter(usage -> usage instanceof MonitoringFeatureSetUsage)

+ 7 - 4
x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsCollector.java

@@ -25,8 +25,9 @@ import org.elasticsearch.license.License;
 import org.elasticsearch.license.LicenseService;
 import org.elasticsearch.license.LicenseUtils;
 import org.elasticsearch.license.XPackLicenseState;
+import org.elasticsearch.protocol.xpack.XPackUsageRequest;
 import org.elasticsearch.xpack.core.XPackFeatureSet;
-import org.elasticsearch.xpack.core.action.XPackUsageRequestBuilder;
+import org.elasticsearch.xpack.core.action.XPackUsageAction;
 import org.elasticsearch.xpack.core.monitoring.exporter.MonitoringDoc;
 import org.elasticsearch.xpack.monitoring.collector.Collector;
 
@@ -85,8 +86,6 @@ public class ClusterStatsCollector extends Collector {
 
     @Override
     protected Collection<MonitoringDoc> doCollect(final MonitoringDoc.Node node, final long interval, final ClusterState clusterState) {
-        final Supplier<List<XPackFeatureSet.Usage>> usageSupplier = () -> new XPackUsageRequestBuilder(client).get().getUsages();
-
         final ClusterStatsResponse clusterStats = client.admin().cluster().prepareClusterStats().setTimeout(getCollectionTimeout()).get();
         ensureNoTimeouts(getCollectionTimeout(), clusterStats);
 
@@ -94,7 +93,11 @@ public class ClusterStatsCollector extends Collector {
         final String clusterUuid = clusterUuid(clusterState);
         final String version = Build.current().version();
         final License license = licenseService.getLicense();
-        final List<XPackFeatureSet.Usage> xpackUsage = collect(usageSupplier);
+        final List<XPackFeatureSet.Usage> xpackUsage = collect(
+            () -> client.execute(XPackUsageAction.INSTANCE, new XPackUsageRequest(getCollectionTimeout()))
+                .actionGet(getCollectionTimeout())
+                .getUsages()
+        );
         final boolean apmIndicesExist = doAPMIndicesExist(clusterState);
         // if they have any other type of license, then they are either okay or already know
         final boolean clusterNeedsTLSEnabled = license != null

+ 2 - 2
x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsCollectorTests.java

@@ -235,7 +235,7 @@ public class ClusterStatsCollectorTests extends BaseCollectorTestCase {
         @SuppressWarnings("unchecked")
         final ActionFuture<XPackUsageResponse> xPackUsageFuture = (ActionFuture<XPackUsageResponse>) mock(ActionFuture.class);
         when(client.execute(same(XPackUsageAction.INSTANCE), any(XPackUsageRequest.class))).thenReturn(xPackUsageFuture);
-        when(xPackUsageFuture.actionGet()).thenReturn(xPackUsageResponse);
+        when(xPackUsageFuture.actionGet(any(TimeValue.class))).thenReturn(xPackUsageResponse);
 
         final ClusterStatsCollector collector = new ClusterStatsCollector(
             settings.build(),
@@ -345,7 +345,7 @@ public class ClusterStatsCollectorTests extends BaseCollectorTestCase {
             @SuppressWarnings("unchecked")
             final ActionFuture<XPackUsageResponse> xPackUsageFuture = (ActionFuture<XPackUsageResponse>) mock(ActionFuture.class);
             when(client.execute(same(XPackUsageAction.INSTANCE), any(XPackUsageRequest.class))).thenReturn(xPackUsageFuture);
-            when(xPackUsageFuture.actionGet()).thenReturn(xPackUsageResponse);
+            when(xPackUsageFuture.actionGet(any(TimeValue.class))).thenReturn(xPackUsageResponse);
         }
 
         final long interval = randomNonNegativeLong();

+ 4 - 2
x-pack/plugin/old-lucene-versions/src/internalClusterTest/java/org/elasticsearch/xpack/lucene/bwc/ArchiveLicenseIntegTests.java

@@ -39,7 +39,9 @@ import static org.hamcrest.Matchers.oneOf;
 public class ArchiveLicenseIntegTests extends AbstractArchiveTestCase {
 
     public void testFeatureUsage() throws Exception {
-        XPackUsageFeatureResponse usage = client().execute(XPackUsageFeatureAction.ARCHIVE, new XPackUsageRequest()).get();
+        XPackUsageFeatureResponse usage = safeGet(
+            client().execute(XPackUsageFeatureAction.ARCHIVE, new XPackUsageRequest(SAFE_AWAIT_TIMEOUT))
+        );
         assertThat(usage.getUsage(), instanceOf(ArchiveFeatureSetUsage.class));
         ArchiveFeatureSetUsage archiveUsage = (ArchiveFeatureSetUsage) usage.getUsage();
         assertEquals(0, archiveUsage.getNumberOfArchiveIndices());
@@ -50,7 +52,7 @@ public class ArchiveLicenseIntegTests extends AbstractArchiveTestCase {
         assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0));
         ensureGreen(indexName);
 
-        usage = client().execute(XPackUsageFeatureAction.ARCHIVE, new XPackUsageRequest()).get();
+        usage = safeGet(client().execute(XPackUsageFeatureAction.ARCHIVE, new XPackUsageRequest(SAFE_AWAIT_TIMEOUT)));
         assertThat(usage.getUsage(), instanceOf(ArchiveFeatureSetUsage.class));
         archiveUsage = (ArchiveFeatureSetUsage) usage.getUsage();
         assertEquals(1, archiveUsage.getNumberOfArchiveIndices());

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java

@@ -184,6 +184,7 @@ public abstract class BaseSearchableSnapshotsIntegTestCase extends AbstractSnaps
         final Storage storage
     ) throws Exception {
         final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             repositoryName,
             snapshotName,

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/ClusterStateApplierOrderingTests.java

@@ -65,6 +65,7 @@ public class ClusterStateApplierOrderingTests extends BaseSearchableSnapshotsInt
             .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0);
 
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             fsRepoName,
             snapshotInfo.snapshotId().getName(),

+ 3 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java

@@ -225,6 +225,7 @@ public class FrozenSearchableSnapshotsIntegTests extends BaseFrozenSearchableSna
         statsWatcher.start();
 
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             fsRepoName,
             snapshotInfo.snapshotId().getName(),
@@ -453,6 +454,7 @@ public class FrozenSearchableSnapshotsIntegTests extends BaseFrozenSearchableSna
 
         Settings.Builder indexSettingsBuilder = Settings.builder().put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true);
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             "test-index",
             "repo",
             "snap",
@@ -565,6 +567,7 @@ public class FrozenSearchableSnapshotsIntegTests extends BaseFrozenSearchableSna
             .putNull(DataTier.TIER_PREFERENCE);
 
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             fsRepoName,
             snapshotInfo.snapshotId().getName(),

+ 3 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java

@@ -152,6 +152,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
             .build();
 
         final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             searchableSnapshotIndexOutsideSearchRange,
             repositoryName,
             snapshotId.getName(),
@@ -422,6 +423,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
             .build();
 
         final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             searchableSnapshotIndexOutsideSearchRange,
             repositoryName,
             snapshotId.getName(),
@@ -605,6 +607,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz
             .build();
 
         final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             searchableSnapshotIndexWithinSearchRange,
             repositoryName,
             snapshotId.getName(),

+ 7 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java

@@ -209,6 +209,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
         }
 
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             fsRepoName,
             snapshotInfo.snapshotId().getName(),
@@ -398,6 +399,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
 
         Settings.Builder indexSettingsBuilder = Settings.builder().put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true);
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             fsRepoName,
             snapshotName,
@@ -466,6 +468,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
 
         logger.info("--> restoring index [{}] using rate limits [{}]", restoredIndexName, useRateLimits);
         final MountSearchableSnapshotRequest mount = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             repositoryName,
             snapshotName,
@@ -550,6 +553,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
             Settings.Builder indexSettingsBuilder = Settings.builder()
                 .put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true);
             final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 restoredIndexName,
                 fsRepoName,
                 snapshotName,
@@ -582,6 +586,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
                 .put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true)
                 .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, replicaCount);
             final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 restoredIndexName,
                 fsRepoName,
                 snapshotName,
@@ -613,6 +618,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
                 .put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true)
                 .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, replicaLimit == dataNodesCount ? "0-all" : "0-" + replicaLimit);
             final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 restoredIndexName,
                 fsRepoName,
                 snapshotName,
@@ -1191,6 +1197,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
         ActionFuture<RestoreSnapshotResponse> response = null;
         try {
             final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 indexName,
                 repositoryName,
                 snapshotOne.getName(),

+ 2 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsLicenseIntegTests.java

@@ -64,6 +64,7 @@ public class SearchableSnapshotsLicenseIntegTests extends BaseFrozenSearchableSn
         assertAcked(indicesAdmin().prepareDelete(indexName));
 
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             indexName,
             repoName,
             snapshotName,
@@ -92,6 +93,7 @@ public class SearchableSnapshotsLicenseIntegTests extends BaseFrozenSearchableSn
 
     public void testMountRequiresLicense() {
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             indexName + "-extra",
             repoName,
             snapshotName,

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsSettingValidationIntegTests.java

@@ -36,6 +36,7 @@ public class SearchableSnapshotsSettingValidationIntegTests extends BaseFrozenSe
         assertAcked(indicesAdmin().prepareDelete(indexName));
 
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             indexName,
             repoName,
             snapshotName,

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsSystemIndicesIntegTests.java

@@ -76,6 +76,7 @@ public class SearchableSnapshotsSystemIndicesIntegTests extends BaseFrozenSearch
         }
 
         final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             indexName,
             repositoryName,
             snapshotName,

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsUuidValidationIntegTests.java

@@ -97,6 +97,7 @@ public class SearchableSnapshotsUuidValidationIntegTests extends BaseFrozenSearc
         createFullSnapshot(fsRepoName, snapshotName);
 
         final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             restoredIndexName,
             fsRepoName,
             snapshotName,

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/allocation/AllocationFilteringIntegTests.java

@@ -62,6 +62,7 @@ public class AllocationFilteringIntegTests extends BaseSearchableSnapshotsIntegT
             .put(mountedIndexSettings.build());
 
         return new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             indexName,
             fsRepoName,
             snapshotName,

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotDiskThresholdIntegTests.java

@@ -177,6 +177,7 @@ public class SearchableSnapshotDiskThresholdIntegTests extends DiskUsageIntegTes
             client().execute(
                 MountSearchableSnapshotAction.INSTANCE,
                 new MountSearchableSnapshotRequest(
+                    TEST_REQUEST_TIMEOUT,
                     prefix + index,
                     repositoryName,
                     snapshotName,

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/full/SearchableSnapshotsPrewarmingIntegTests.java

@@ -215,6 +215,7 @@ public class SearchableSnapshotsPrewarmingIntegTests extends ESSingleNodeTestCas
                     final RestoreSnapshotResponse restoreSnapshotResponse = client().execute(
                         MountSearchableSnapshotAction.INSTANCE,
                         new MountSearchableSnapshotRequest(
+                            TEST_REQUEST_TIMEOUT,
                             indexName,
                             "repository",
                             "snapshot",

+ 1 - 0
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/PartiallyCachedShardAllocationIntegTests.java

@@ -91,6 +91,7 @@ public class PartiallyCachedShardAllocationIntegTests extends BaseFrozenSearchab
             .put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true);
 
         return new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             indexName,
             fsRepoName,
             snapshotName,

+ 12 - 0
x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/MountSearchableSnapshotRequestTests.java

@@ -33,6 +33,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
 
     private MountSearchableSnapshotRequest randomState(MountSearchableSnapshotRequest instance) {
         return new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             randomBoolean() ? instance.mountedIndexName() : mutateString(instance.mountedIndexName()),
             randomBoolean() ? instance.repositoryName() : mutateString(instance.repositoryName()),
             randomBoolean() ? instance.snapshotName() : mutateString(instance.snapshotName()),
@@ -48,6 +49,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
     protected MountSearchableSnapshotRequest createTestInstance() {
         return randomState(
             new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 randomAlphaOfLength(5),
                 randomAlphaOfLength(5),
                 randomAlphaOfLength(5),
@@ -69,6 +71,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
     protected MountSearchableSnapshotRequest mutateInstance(MountSearchableSnapshotRequest req) {
         return switch (randomInt(8)) {
             case 0 -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 mutateString(req.mountedIndexName()),
                 req.repositoryName(),
                 req.snapshotName(),
@@ -79,6 +82,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
                 req.storage()
             ).masterNodeTimeout(req.masterNodeTimeout());
             case 1 -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 req.mountedIndexName(),
                 mutateString(req.repositoryName()),
                 req.snapshotName(),
@@ -89,6 +93,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
                 req.storage()
             ).masterNodeTimeout(req.masterNodeTimeout());
             case 2 -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 req.mountedIndexName(),
                 req.repositoryName(),
                 mutateString(req.snapshotName()),
@@ -99,6 +104,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
                 req.storage()
             ).masterNodeTimeout(req.masterNodeTimeout());
             case 3 -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 req.mountedIndexName(),
                 req.repositoryName(),
                 req.snapshotName(),
@@ -109,6 +115,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
                 req.storage()
             ).masterNodeTimeout(req.masterNodeTimeout());
             case 4 -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 req.mountedIndexName(),
                 req.repositoryName(),
                 req.snapshotName(),
@@ -119,6 +126,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
                 req.storage()
             ).masterNodeTimeout(req.masterNodeTimeout());
             case 5 -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 req.mountedIndexName(),
                 req.repositoryName(),
                 req.snapshotName(),
@@ -129,6 +137,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
                 req.storage()
             ).masterNodeTimeout(req.masterNodeTimeout());
             case 6 -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 req.mountedIndexName(),
                 req.repositoryName(),
                 req.snapshotName(),
@@ -139,6 +148,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
                 req.storage()
             ).masterNodeTimeout(req.masterNodeTimeout());
             case 7 -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 req.mountedIndexName(),
                 req.repositoryName(),
                 req.snapshotName(),
@@ -149,6 +159,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
                 randomValueOtherThan(req.storage(), () -> randomFrom(MountSearchableSnapshotRequest.Storage.values()))
             ).masterNodeTimeout(req.masterNodeTimeout());
             default -> new MountSearchableSnapshotRequest(
+                TEST_REQUEST_TIMEOUT,
                 req.mountedIndexName(),
                 req.repositoryName(),
                 req.snapshotName(),
@@ -196,6 +207,7 @@ public class MountSearchableSnapshotRequestTests extends AbstractWireSerializing
 
     public void testForbidsCustomDataPath() {
         final ActionRequestValidationException validationException = new MountSearchableSnapshotRequest(
+            TEST_REQUEST_TIMEOUT,
             randomAlphaOfLength(5),
             randomAlphaOfLength(5),
             randomAlphaOfLength(5),

+ 3 - 2
x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java

@@ -20,6 +20,7 @@ import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.settings.SecureString;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
+import org.elasticsearch.protocol.xpack.XPackUsageRequest;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.snapshots.SnapshotInfo;
 import org.elasticsearch.snapshots.SnapshotState;
@@ -29,7 +30,7 @@ import org.elasticsearch.test.SecuritySettingsSourceField;
 import org.elasticsearch.transport.TransportRequest;
 import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.core.XPackFeatureSet;
-import org.elasticsearch.xpack.core.action.XPackUsageRequestBuilder;
+import org.elasticsearch.xpack.core.action.XPackUsageAction;
 import org.elasticsearch.xpack.core.action.XPackUsageResponse;
 import org.elasticsearch.xpack.core.security.SecurityFeatureSetUsage;
 import org.elasticsearch.xpack.core.security.action.realm.ClearRealmCacheRequestBuilder;
@@ -901,7 +902,7 @@ public class NativeRealmIntegTests extends NativeRealmIntegTestCase {
             preparePutUser("joe" + i, "s3krit-password", hasher, "superuser").get();
         }
 
-        XPackUsageResponse response = new XPackUsageRequestBuilder(client()).get();
+        XPackUsageResponse response = safeGet(client().execute(XPackUsageAction.INSTANCE, new XPackUsageRequest(SAFE_AWAIT_TIMEOUT)));
         Optional<XPackFeatureSet.Usage> securityUsage = response.getUsages()
             .stream()
             .filter(usage -> usage instanceof SecurityFeatureSetUsage)

+ 6 - 5
x-pack/qa/freeze-plugin/src/main/java/org/elasticsearch/plugin/freeze/FreezeIndexPlugin.java

@@ -29,7 +29,6 @@ import org.elasticsearch.rest.RestRequest;
 import org.elasticsearch.rest.action.RestToXContentListener;
 import org.elasticsearch.xpack.core.frozen.action.FreezeIndexAction;
 
-import java.io.IOException;
 import java.util.List;
 import java.util.function.Predicate;
 import java.util.function.Supplier;
@@ -74,11 +73,13 @@ public class FreezeIndexPlugin extends Plugin implements ActionPlugin {
         }
 
         @Override
-        protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
+        protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) {
             boolean freeze = request.path().endsWith("/_freeze");
-            FreezeRequest freezeRequest = new FreezeRequest(Strings.splitStringByCommaToArray(request.param("index")));
-            freezeRequest.ackTimeout(getAckTimeout(request));
-            freezeRequest.masterNodeTimeout(getMasterNodeTimeout(request));
+            FreezeRequest freezeRequest = new FreezeRequest(
+                getMasterNodeTimeout(request),
+                getAckTimeout(request),
+                Strings.splitStringByCommaToArray(request.param("index"))
+            );
             freezeRequest.indicesOptions(IndicesOptions.fromRequest(request, freezeRequest.indicesOptions()));
             String waitForActiveShards = request.param("wait_for_active_shards");
             if (waitForActiveShards != null) {