mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-04-24 15:17:30 -04:00
Fix support for infinite ?master_timeout
(#107050)
Specifying `?master_timeout=-1` on an API which performs a cluster state update means that the cluster state update task will never time out while waiting in the pending tasks queue. However this parameter is also re-used in a few places where a timeout of `-1` means something else, typically to timeout immediately. This commit fixes those places so that `?master_timeout=-1` consistently means to wait forever.
This commit is contained in:
parent
1587dada58
commit
ccbb5badce
32 changed files with 133 additions and 97 deletions
5
docs/changelog/107050.yaml
Normal file
5
docs/changelog/107050.yaml
Normal file
|
@ -0,0 +1,5 @@
|
|||
pr: 107050
|
||||
summary: Fix support for infinite `?master_timeout`
|
||||
area: Cluster Coordination
|
||||
type: bug
|
||||
issues: []
|
|
@ -57,10 +57,7 @@ This API deletes a configured collection of
|
|||
[[ccr-delete-auto-follow-pattern-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-delete-auto-follow-pattern-examples]]
|
||||
==== {api-examples-title}
|
||||
|
|
|
@ -75,10 +75,7 @@ This API will return the specified auto-follow pattern collection.
|
|||
[[ccr-get-auto-follow-pattern-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-get-auto-follow-pattern-examples]]
|
||||
==== {api-examples-title}
|
||||
|
|
|
@ -43,10 +43,7 @@ meantime.
|
|||
[[ccr-pause-auto-follow-pattern-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-pause-auto-follow-pattern-examples]]
|
||||
==== {api-examples-title}
|
||||
|
|
|
@ -74,10 +74,7 @@ the new patterns.
|
|||
[[ccr-put-auto-follow-pattern-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-put-auto-follow-pattern-request-body]]
|
||||
==== {api-request-body-title}
|
||||
|
|
|
@ -38,10 +38,7 @@ have been deleted or closed in the meantime.
|
|||
[[ccr-resume-auto-follow-pattern-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-resume-auto-follow-pattern-examples]]
|
||||
==== {api-examples-title}
|
||||
|
|
|
@ -52,10 +52,7 @@ replication options and whether the follower indices are active or paused.
|
|||
[[ccr-get-follow-info-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[role="child_attributes"]
|
||||
[[ccr-get-follow-info-response-body]]
|
||||
|
|
|
@ -56,10 +56,7 @@ following task.
|
|||
[[ccr-post-pause-follow-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-post-pause-follow-examples]]
|
||||
==== {api-examples-title}
|
||||
|
|
|
@ -69,10 +69,7 @@ returns, the follower index will resume fetching operations from the leader inde
|
|||
[[ccr-post-resume-follow-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-post-resume-follow-request-body]]
|
||||
==== {api-request-body-title}
|
||||
|
|
|
@ -63,10 +63,7 @@ irreversible operation.
|
|||
[[ccr-post-unfollow-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-post-unfollow-examples]]
|
||||
==== {api-examples-title}
|
||||
|
|
|
@ -65,11 +65,7 @@ referenced leader index. When this API returns, the follower index exists, and
|
|||
follower shard requires transferring all the remote Lucene segment files to
|
||||
the follower index.
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[[ccr-put-follow-request-body]]
|
||||
==== {api-request-body-title}
|
||||
|
|
|
@ -56,10 +56,7 @@ shard-level stats as in the <<ccr-get-follow-stats,get follower stats API>>.
|
|||
`timeout`::
|
||||
(Optional, time) Controls the amount of time to wait for results. Defaults to unlimited.
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[role="child_attributes"]
|
||||
[[ccr-get-stats-response-body]]
|
||||
|
|
|
@ -1218,9 +1218,9 @@ tag::timeoutparms[]
|
|||
tag::master-timeout[]
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>)
|
||||
Period to wait for a connection to the master node. If no response is received
|
||||
before the timeout expires, the request fails and returns an error. Defaults to
|
||||
`30s`.
|
||||
Period to wait for the master node. If the master node is not available before
|
||||
the timeout expires, the request fails and returns an error. Defaults to `30s`.
|
||||
Can also be set to `-1` to indicate that the request should never timeout.
|
||||
end::master-timeout[]
|
||||
|
||||
tag::timeout[]
|
||||
|
|
|
@ -27,10 +27,7 @@ information, see <<security-privileges>>.
|
|||
[[watcher-api-start-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
//[[watcher-api-start-request-body]]
|
||||
//==== {api-request-body-title}
|
||||
|
|
|
@ -27,10 +27,7 @@ information, see <<security-privileges>>.
|
|||
[[watcher-api-stop-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
//[[watcher-api-stop-request-body]]
|
||||
//==== {api-request-body-title}
|
||||
|
|
|
@ -42,10 +42,7 @@ Name of the snapshot repository that both source and target snapshot belong to.
|
|||
[[clone-snapshot-api-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
`timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
|
@ -55,4 +52,4 @@ fails and returns an error. Defaults to `30s`.
|
|||
`indices`::
|
||||
(Required, string)
|
||||
A comma-separated list of indices to include in the snapshot.
|
||||
<<api-multi-index,multi-target syntax>> is supported.
|
||||
<<api-multi-index,multi-target syntax>> is supported.
|
||||
|
|
|
@ -51,10 +51,7 @@ supported.
|
|||
[[delete-snapshot-repo-api-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
`timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
|
|
|
@ -59,10 +59,7 @@ cluster, omit this parameter or use `*` or `_all`.
|
|||
only. If `false`, the request gets information from the master node. Defaults to
|
||||
`false`.
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
[role="child_attributes"]
|
||||
[[get-snapshot-repo-api-response-body]]
|
||||
|
|
|
@ -52,10 +52,7 @@ IMPORTANT: Several options for this API can be specified using a query parameter
|
|||
or a request body parameter. If both parameters are specified, only the query
|
||||
parameter is used.
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
`timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
|
|
|
@ -47,10 +47,7 @@ Name of the snapshot repository to verify.
|
|||
[[verify-snapshot-repo-api-query-params]]
|
||||
==== {api-query-parms-title}
|
||||
|
||||
`master_timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
a connection to the master node. If no response is received before the timeout
|
||||
expires, the request fails and returns an error. Defaults to `30s`.
|
||||
include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=master-timeout]
|
||||
|
||||
`timeout`::
|
||||
(Optional, <<time-units, time units>>) Specifies the period of time to wait for
|
||||
|
|
|
@ -81,9 +81,12 @@ public class UpdateDataStreamGlobalRetentionService {
|
|||
List<UpdateDataStreamGlobalRetentionResponse.AffectedDataStream> affectedDataStreams,
|
||||
final ActionListener<UpdateDataStreamGlobalRetentionResponse> listener
|
||||
) {
|
||||
final var ackTimeout = request.masterNodeTimeout().millis() < 0 ? TimeValue.MAX_VALUE : request.masterNodeTimeout();
|
||||
// NB a negative master node timeout means never to time out, but a negative ack timeout means to time out immediately.
|
||||
// TODO when https://github.com/elastic/elasticsearch/issues/107044 is fixed, we can just use request.masterNodeTimeout() directly
|
||||
taskQueue.submitTask(
|
||||
"remove-data-stream-global-retention",
|
||||
new UpsertGlobalDataStreamMetadataTask(null, affectedDataStreams, listener, request.masterNodeTimeout()),
|
||||
new UpsertGlobalDataStreamMetadataTask(null, affectedDataStreams, listener, ackTimeout),
|
||||
request.masterNodeTimeout()
|
||||
);
|
||||
}
|
||||
|
@ -137,7 +140,7 @@ public class UpdateDataStreamGlobalRetentionService {
|
|||
@Nullable DataStreamGlobalRetention globalRetention,
|
||||
List<UpdateDataStreamGlobalRetentionResponse.AffectedDataStream> affectedDataStreams,
|
||||
ActionListener<UpdateDataStreamGlobalRetentionResponse> listener,
|
||||
TimeValue masterTimeout
|
||||
TimeValue ackTimeout
|
||||
) implements ClusterStateTaskListener, ClusterStateAckListener {
|
||||
|
||||
@Override
|
||||
|
@ -166,10 +169,5 @@ public class UpdateDataStreamGlobalRetentionService {
|
|||
logger.debug("Failed to update global retention [{}] because timeout was reached", globalRetention);
|
||||
listener.onResponse(UpdateDataStreamGlobalRetentionResponse.FAILED);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimeValue ackTimeout() {
|
||||
return masterTimeout;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -121,6 +121,14 @@ public class RolloverIT extends ESIntegTestCase {
|
|||
);
|
||||
}
|
||||
|
||||
public void testInfiniteMasterNodeTimeout() {
|
||||
assertAcked(prepareCreate("test_index-2").addAlias(new Alias("test_alias")).get());
|
||||
indexDoc("test_index-2", "1", "field", "value");
|
||||
flush("test_index-2");
|
||||
final RolloverResponse response = indicesAdmin().prepareRolloverIndex("test_alias").setMasterNodeTimeout(TimeValue.MINUS_ONE).get();
|
||||
assertTrue(response.isShardsAcknowledged());
|
||||
}
|
||||
|
||||
public void testRolloverWithExplicitWriteIndex() throws Exception {
|
||||
long beforeTime = client().threadPool().absoluteTimeInMillis() - 1000L;
|
||||
assertAcked(prepareCreate("test_index-2").addAlias(new Alias("test_alias").writeIndex(true)).get());
|
||||
|
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* 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 and the Server Side Public License, v 1; you may not use this file except
|
||||
* in compliance with, at your election, the Elastic License 2.0 or the Server
|
||||
* Side Public License, v 1.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.rest.action.admin.cluster;
|
||||
|
||||
import org.elasticsearch.client.Request;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class RestClusterStateActionIT extends ESIntegTestCase {
|
||||
|
||||
@Override
|
||||
protected boolean addMockHttpTransport() {
|
||||
return false;
|
||||
}
|
||||
|
||||
public void testInfiniteTimeOut() throws IOException {
|
||||
final var request = new Request("GET", "/_cluster/state/none");
|
||||
request.addParameter("master_timeout", "-1");
|
||||
getRestClient().performRequest(request);
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.core.IOUtils;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.index.IndexVersion;
|
||||
import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -688,6 +689,32 @@ public class SnapshotStatusApisIT extends AbstractSnapshotIntegTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testInfiniteTimeout() throws Exception {
|
||||
createRepository("test-repo", "mock");
|
||||
createIndex("test-idx", 1, 0);
|
||||
indexRandomDocs("test-idx", 10);
|
||||
ensureGreen();
|
||||
blockAllDataNodes("test-repo");
|
||||
final ActionFuture<CreateSnapshotResponse> snapshotResponseFuture = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap")
|
||||
.setWaitForCompletion(true)
|
||||
.execute();
|
||||
try {
|
||||
waitForBlockOnAnyDataNode("test-repo");
|
||||
final List<SnapshotStatus> snapshotStatus = clusterAdmin().prepareSnapshotStatus("test-repo")
|
||||
.setMasterNodeTimeout(TimeValue.MINUS_ONE)
|
||||
.get()
|
||||
.getSnapshots();
|
||||
assertThat(snapshotStatus, hasSize(1));
|
||||
assertEquals("test-snap", snapshotStatus.get(0).getSnapshot().getSnapshotId().getName());
|
||||
// a timeout of a node-level request results in a successful response but without node-level details, so this checks no timeout:
|
||||
assertThat(snapshotStatus.get(0).getShards().get(0).getStats().getTotalFileCount(), greaterThan(0));
|
||||
assertFalse(snapshotResponseFuture.isDone());
|
||||
} finally {
|
||||
unblockAllDataNodes("test-repo");
|
||||
snapshotResponseFuture.get(10, TimeUnit.SECONDS);
|
||||
}
|
||||
}
|
||||
|
||||
private static SnapshotIndexShardStatus stateFirstShard(SnapshotStatus snapshotStatus, String indexName) {
|
||||
return snapshotStatus.getIndices().get(indexName).getShards().get(0);
|
||||
}
|
||||
|
|
|
@ -141,7 +141,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
|
|||
client.executeLocally(
|
||||
TransportNodesSnapshotsStatus.TYPE,
|
||||
new TransportNodesSnapshotsStatus.Request(nodesIds.toArray(Strings.EMPTY_ARRAY)).snapshots(snapshots)
|
||||
.timeout(request.masterNodeTimeout()),
|
||||
.timeout(request.masterNodeTimeout().millis() < 0 ? null : request.masterNodeTimeout()),
|
||||
// fork to snapshot meta since building the response is expensive for large snapshots
|
||||
new RefCountAwareThreadedActionListener<>(
|
||||
threadPool.executor(ThreadPool.Names.SNAPSHOT_META),
|
||||
|
|
|
@ -514,6 +514,10 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
|
|||
final var rolloverIndexName = rolloverResult.rolloverIndexName();
|
||||
final var sourceIndexName = rolloverResult.sourceIndexName();
|
||||
|
||||
final var waitForActiveShardsTimeout = rolloverRequest.masterNodeTimeout().millis() < 0
|
||||
? null
|
||||
: rolloverRequest.masterNodeTimeout();
|
||||
|
||||
rolloverTaskContext.success(() -> {
|
||||
// Now assuming we have a new state and the name of the rolled over index, we need to wait for the configured number of
|
||||
// active shards, as well as return the names of the indices that were rolled/created
|
||||
|
@ -521,7 +525,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
|
|||
clusterService,
|
||||
new String[] { rolloverIndexName },
|
||||
rolloverRequest.getCreateIndexRequest().waitForActiveShards(),
|
||||
rolloverRequest.masterNodeTimeout(),
|
||||
waitForActiveShardsTimeout,
|
||||
allocationActionMultiListener.delay(rolloverTask.listener())
|
||||
.map(
|
||||
isShardsAcknowledged -> new RolloverResponse(
|
||||
|
|
|
@ -14,6 +14,7 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateObserver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.node.NodeClosedException;
|
||||
|
||||
|
@ -43,7 +44,7 @@ public enum ActiveShardsObserver {
|
|||
ClusterService clusterService,
|
||||
final String[] indexNames,
|
||||
final ActiveShardCount activeShardCount,
|
||||
final TimeValue timeout,
|
||||
@Nullable final TimeValue timeout,
|
||||
final ActionListener<Boolean> listener
|
||||
) {
|
||||
if (activeShardCount == ActiveShardCount.NONE) {
|
||||
|
|
|
@ -39,7 +39,8 @@ public abstract class MasterNodeRequest<Request extends MasterNodeRequest<Reques
|
|||
}
|
||||
|
||||
/**
|
||||
* A timeout value in case the master has not been discovered yet or disconnected.
|
||||
* Specifies how long to wait when the master has not been discovered yet, or is disconnected, or is busy processing other tasks. The
|
||||
* value {@link TimeValue#MINUS_ONE} means to wait forever.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public final Request masterNodeTimeout(TimeValue timeout) {
|
||||
|
@ -48,12 +49,17 @@ public abstract class MasterNodeRequest<Request extends MasterNodeRequest<Reques
|
|||
}
|
||||
|
||||
/**
|
||||
* A timeout value in case the master has not been discovered yet or disconnected.
|
||||
* Specifies how long to wait when the master has not been discovered yet, or is disconnected, or is busy processing other tasks. The
|
||||
* value {@link TimeValue#MINUS_ONE} means to wait forever.
|
||||
*/
|
||||
public final Request masterNodeTimeout(String timeout) {
|
||||
return masterNodeTimeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".masterNodeTimeout"));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return how long to wait when the master has not been discovered yet, or is disconnected, or is busy processing other tasks. The
|
||||
* value {@link TimeValue#MINUS_ONE} means to wait forever.
|
||||
*/
|
||||
public final TimeValue masterNodeTimeout() {
|
||||
return this.masterNodeTimeout;
|
||||
}
|
||||
|
|
|
@ -285,16 +285,22 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
|
|||
|
||||
private void retry(long currentStateVersion, final Throwable failure, final Predicate<ClusterState> statePredicate) {
|
||||
if (observer == null) {
|
||||
final long remainingTimeoutMS = request.masterNodeTimeout().millis() - (threadPool.relativeTimeInMillis() - startTime);
|
||||
if (remainingTimeoutMS <= 0) {
|
||||
logger.debug(() -> "timed out before retrying [" + actionName + "] after failure", failure);
|
||||
listener.onFailure(new MasterNotDiscoveredException(failure));
|
||||
return;
|
||||
final TimeValue timeout;
|
||||
if (request.masterNodeTimeout().millis() < 0) {
|
||||
timeout = null;
|
||||
} else {
|
||||
final long remainingTimeoutMS = request.masterNodeTimeout().millis() - (threadPool.relativeTimeInMillis() - startTime);
|
||||
if (remainingTimeoutMS <= 0) {
|
||||
logger.debug(() -> "timed out before retrying [" + actionName + "] after failure", failure);
|
||||
listener.onFailure(new MasterNotDiscoveredException(failure));
|
||||
return;
|
||||
}
|
||||
timeout = TimeValue.timeValueMillis(remainingTimeoutMS);
|
||||
}
|
||||
this.observer = new ClusterStateObserver(
|
||||
currentStateVersion,
|
||||
clusterService.getClusterApplierService(),
|
||||
TimeValue.timeValueMillis(remainingTimeoutMS),
|
||||
timeout,
|
||||
logger,
|
||||
threadPool.getThreadContext()
|
||||
);
|
||||
|
|
|
@ -152,6 +152,7 @@ public class RestClusterStateAction extends BaseRestHandler {
|
|||
@Override
|
||||
public Iterator<? extends ToXContent> toXContentChunked(ToXContent.Params outerParams) {
|
||||
if (request.local() == false
|
||||
&& request.masterNodeTimeout().millis() >= 0
|
||||
&& currentTimeMillisSupplier.getAsLong() - startTimeMillis > request.masterNodeTimeout().millis()) {
|
||||
throw new ElasticsearchTimeoutException("Timed out getting cluster state");
|
||||
}
|
||||
|
|
|
@ -477,6 +477,9 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
|
||||
public void testMasterBecomesAvailable() throws ExecutionException, InterruptedException {
|
||||
Request request = new Request();
|
||||
if (randomBoolean()) {
|
||||
request.masterNodeTimeout(TimeValue.MINUS_ONE);
|
||||
}
|
||||
setState(clusterService, ClusterStateCreationUtils.state(localNode, null, allNodes));
|
||||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
ActionTestUtils.execute(new Action("internal:testAction", transportService, clusterService, threadPool), null, request, listener);
|
||||
|
|
|
@ -252,10 +252,14 @@ public class TransportPutDataFrameAnalyticsAction extends TransportMasterNodeAct
|
|||
delegate.onResponse(finalConfig);
|
||||
});
|
||||
|
||||
final var deleterTimeout = masterNodeTimeout.millis() < 0 ? TimeValue.MAX_VALUE : masterNodeTimeout;
|
||||
// NB a negative masterNodeTimeout means never to time out, but recording dataframe analytics configs does not support infinite
|
||||
// timeouts so we just use a very long timeout here instead
|
||||
|
||||
ClusterState clusterState = clusterService.state();
|
||||
if (clusterState == null) {
|
||||
logger.warn("Cannot update doc mapping because clusterState == null");
|
||||
configProvider.put(config, headers, masterNodeTimeout, auditingListener);
|
||||
configProvider.put(config, headers, deleterTimeout, auditingListener);
|
||||
return;
|
||||
}
|
||||
ElasticsearchMappings.addDocMappingIfMissing(
|
||||
|
@ -264,7 +268,7 @@ public class TransportPutDataFrameAnalyticsAction extends TransportMasterNodeAct
|
|||
client,
|
||||
clusterState,
|
||||
masterNodeTimeout,
|
||||
auditingListener.delegateFailureAndWrap((l, unused) -> configProvider.put(config, headers, masterNodeTimeout, l)),
|
||||
auditingListener.delegateFailureAndWrap((l, unused) -> configProvider.put(config, headers, deleterTimeout, l)),
|
||||
MlConfigIndex.CONFIG_INDEX_MAPPINGS_VERSION
|
||||
);
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue