Added wait_for_metadata_version parameter to cluster state api. (#35535)

The `wait_for_metadata_version` parameter will instruct the cluster state
api to only return a cluster state until the metadata's version is equal or
greater than the version specified in `wait_for_metadata_version`. If  
the specified `wait_for_timeout` has expired then a timed out response 
is returned. (a response with no cluster state and wait for timed out flag set to true)
In  the case metadata's version is equal or higher than  `wait_for_metadata_version`
then the api will immediately return.

This feature is useful to avoid external components from constantly
polling the cluster state to whether somethings have changed in the
cluster state's metadata.
This commit is contained in:
Martijn van Groningen 2018-11-26 08:50:08 +01:00 committed by GitHub
parent 00e6fec718
commit 7624734f14
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 351 additions and 13 deletions

View File

@ -114,7 +114,7 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase {
builder.add(node);
}
ClusterState build = ClusterState.builder(clusterName).nodes(builder.build()).build();
channel.sendResponse(new ClusterStateResponse(clusterName, build, 0L));
channel.sendResponse(new ClusterStateResponse(clusterName, build, 0L, false));
});
newService.start();
newService.acceptIncomingRequests();

View File

@ -33,6 +33,14 @@
"type": "boolean",
"description": "Return settings in flat format (default: false)"
},
"wait_for_metadata_version": {
"type": "number",
"description": "Wait for the metadata version to be equal or greater than the specified metadata version"
},
"wait_for_timeout" : {
"type": "time",
"description": "The maximum time to wait for wait_for_metadata_version before timing out"
},
"ignore_unavailable": {
"type" : "boolean",
"description" : "Whether specified concrete indices should be ignored when unavailable (missing or closed)"

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.admin.cluster.state;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.support.IndicesOptions;
@ -26,16 +27,21 @@ import org.elasticsearch.action.support.master.MasterNodeReadRequest;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.TimeValue;
import java.io.IOException;
public class ClusterStateRequest extends MasterNodeReadRequest<ClusterStateRequest> implements IndicesRequest.Replaceable {
public static final TimeValue DEFAULT_WAIT_FOR_NODE_TIMEOUT = TimeValue.timeValueMinutes(1);
private boolean routingTable = true;
private boolean nodes = true;
private boolean metaData = true;
private boolean blocks = true;
private boolean customs = true;
private Long waitForMetaDataVersion;
private TimeValue waitForTimeout = DEFAULT_WAIT_FOR_NODE_TIMEOUT;
private String[] indices = Strings.EMPTY_ARRAY;
private IndicesOptions indicesOptions = IndicesOptions.lenientExpandOpen();
@ -51,6 +57,11 @@ public class ClusterStateRequest extends MasterNodeReadRequest<ClusterStateReque
customs = in.readBoolean();
indices = in.readStringArray();
indicesOptions = IndicesOptions.readIndicesOptions(in);
// TODO: change version to V_6_6_0 after backporting:
if (in.getVersion().onOrAfter(Version.V_7_0_0)) {
waitForTimeout = in.readTimeValue();
waitForMetaDataVersion = in.readOptionalLong();
}
}
@Override
@ -63,6 +74,11 @@ public class ClusterStateRequest extends MasterNodeReadRequest<ClusterStateReque
out.writeBoolean(customs);
out.writeStringArray(indices);
indicesOptions.writeIndicesOptions(out);
// TODO: change version to V_6_6_0 after backporting:
if (out.getVersion().onOrAfter(Version.V_7_0_0)) {
out.writeTimeValue(waitForTimeout);
out.writeOptionalLong(waitForMetaDataVersion);
}
}
@Override
@ -156,6 +172,28 @@ public class ClusterStateRequest extends MasterNodeReadRequest<ClusterStateReque
return customs;
}
public TimeValue waitForTimeout() {
return waitForTimeout;
}
public ClusterStateRequest waitForTimeout(TimeValue waitForTimeout) {
this.waitForTimeout = waitForTimeout;
return this;
}
public Long waitForMetaDataVersion() {
return waitForMetaDataVersion;
}
public ClusterStateRequest waitForMetaDataVersion(long waitForMetaDataVersion) {
if (waitForMetaDataVersion < 1) {
throw new IllegalArgumentException("provided waitForMetaDataVersion should be >= 1, but instead is [" +
waitForMetaDataVersion + "]");
}
this.waitForMetaDataVersion = waitForMetaDataVersion;
return this;
}
@Override
public void readFrom(StreamInput in) throws IOException {
throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable");

View File

@ -24,11 +24,13 @@ import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.ByteSizeValue;
import java.io.IOException;
import java.util.Objects;
/**
* The response for getting the cluster state.
@ -40,14 +42,16 @@ public class ClusterStateResponse extends ActionResponse {
// the total compressed size of the full cluster state, not just
// the parts included in this response
private ByteSizeValue totalCompressedSize;
private boolean waitForTimedOut = false;
public ClusterStateResponse() {
}
public ClusterStateResponse(ClusterName clusterName, ClusterState clusterState, long sizeInBytes) {
public ClusterStateResponse(ClusterName clusterName, ClusterState clusterState, long sizeInBytes, boolean waitForTimedOut) {
this.clusterName = clusterName;
this.clusterState = clusterState;
this.totalCompressedSize = new ByteSizeValue(sizeInBytes);
this.waitForTimedOut = waitForTimedOut;
}
/**
@ -75,11 +79,24 @@ public class ClusterStateResponse extends ActionResponse {
return totalCompressedSize;
}
/**
* Returns whether the request timed out waiting for a cluster state with a metadata version equal or
* higher than the specified metadata.
*/
public boolean isWaitForTimedOut() {
return waitForTimedOut;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
clusterName = new ClusterName(in);
clusterState = ClusterState.readFrom(in, null);
// TODO: change version to V_6_6_0 after backporting:
if (in.getVersion().onOrAfter(Version.V_7_0_0)) {
clusterState = in.readOptionalWriteable(innerIn -> ClusterState.readFrom(innerIn, null));
} else {
clusterState = ClusterState.readFrom(in, null);
}
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) {
totalCompressedSize = new ByteSizeValue(in);
} else {
@ -89,19 +106,80 @@ public class ClusterStateResponse extends ActionResponse {
// at which point the correct cluster state size will always be reported
totalCompressedSize = new ByteSizeValue(0L);
}
// TODO: change version to V_6_6_0 after backporting:
if (in.getVersion().onOrAfter(Version.V_7_0_0)) {
waitForTimedOut = in.readBoolean();
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
clusterName.writeTo(out);
if (out.getVersion().onOrAfter(Version.V_6_3_0)) {
clusterState.writeTo(out);
// TODO: change version to V_6_6_0 after backporting:
if (out.getVersion().onOrAfter(Version.V_7_0_0)) {
out.writeOptionalWriteable(clusterState);
} else {
ClusterModule.filterCustomsForPre63Clients(clusterState).writeTo(out);
if (out.getVersion().onOrAfter(Version.V_6_3_0)) {
clusterState.writeTo(out);
} else {
ClusterModule.filterCustomsForPre63Clients(clusterState).writeTo(out);
}
}
if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) {
totalCompressedSize.writeTo(out);
}
// TODO: change version to V_6_6_0 after backporting:
if (out.getVersion().onOrAfter(Version.V_7_0_0)) {
out.writeBoolean(waitForTimedOut);
}
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ClusterStateResponse response = (ClusterStateResponse) o;
return waitForTimedOut == response.waitForTimedOut &&
Objects.equals(clusterName, response.clusterName) &&
// Best effort. Only compare cluster state version and master node id,
// because cluster state doesn't implement equals()
Objects.equals(getVersion(clusterState), getVersion(response.clusterState)) &&
Objects.equals(getMasterNodeId(clusterState), getMasterNodeId(response.clusterState)) &&
Objects.equals(totalCompressedSize, response.totalCompressedSize);
}
@Override
public int hashCode() {
// Best effort. Only use cluster state version and master node id,
// because cluster state doesn't implement hashcode()
return Objects.hash(
clusterName,
getVersion(clusterState),
getMasterNodeId(clusterState),
totalCompressedSize,
waitForTimedOut
);
}
private static String getMasterNodeId(ClusterState clusterState) {
if (clusterState == null) {
return null;
}
DiscoveryNodes nodes = clusterState.getNodes();
if (nodes != null) {
return nodes.getMasterNodeId();
} else {
return null;
}
}
private static Long getVersion(ClusterState clusterState) {
if (clusterState != null) {
return clusterState.getVersion();
} else {
return null;
}
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
@ -33,10 +34,13 @@ import org.elasticsearch.cluster.metadata.MetaData.Custom;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.function.Predicate;
import static org.elasticsearch.discovery.zen.PublishClusterStateAction.serializeFullClusterState;
@ -74,7 +78,51 @@ public class TransportClusterStateAction extends TransportMasterNodeReadAction<C
@Override
protected void masterOperation(final ClusterStateRequest request, final ClusterState state,
final ActionListener<ClusterStateResponse> listener) throws IOException {
ClusterState currentState = clusterService.state();
if (request.waitForMetaDataVersion() != null) {
final Predicate<ClusterState> metadataVersionPredicate = clusterState -> {
return clusterState.metaData().version() >= request.waitForMetaDataVersion();
};
final ClusterStateObserver observer =
new ClusterStateObserver(clusterService, request.waitForTimeout(), logger, threadPool.getThreadContext());
final ClusterState clusterState = observer.setAndGetObservedState();
if (metadataVersionPredicate.test(clusterState)) {
buildResponse(request, clusterState, listener);
} else {
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
try {
buildResponse(request, state, listener);
} catch (Exception e) {
listener.onFailure(e);
}
}
@Override
public void onClusterServiceClose() {
listener.onFailure(new NodeClosedException(clusterService.localNode()));
}
@Override
public void onTimeout(TimeValue timeout) {
try {
listener.onResponse(new ClusterStateResponse(clusterState.getClusterName(), null, 0L, true));
} catch (Exception e) {
listener.onFailure(e);
}
}
}, metadataVersionPredicate);
}
} else {
ClusterState currentState = clusterService.state();
buildResponse(request, currentState, listener);
}
}
private void buildResponse(final ClusterStateRequest request,
final ClusterState currentState,
final ActionListener<ClusterStateResponse> listener) throws IOException {
logger.trace("Serving cluster state request using version {}", currentState.version());
ClusterState.Builder builder = ClusterState.builder(currentState.getClusterName());
builder.version(currentState.version());
@ -133,7 +181,7 @@ public class TransportClusterStateAction extends TransportMasterNodeReadAction<C
}
}
listener.onResponse(new ClusterStateResponse(currentState.getClusterName(), builder.build(),
serializeFullClusterState(currentState, Version.CURRENT).length()));
serializeFullClusterState(currentState, Version.CURRENT).length(), false));
}

View File

@ -67,6 +67,10 @@ public class RestClusterStateAction extends BaseRestHandler {
clusterStateRequest.indicesOptions(IndicesOptions.fromRequest(request, clusterStateRequest.indicesOptions()));
clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local()));
clusterStateRequest.masterNodeTimeout(request.paramAsTime("master_timeout", clusterStateRequest.masterNodeTimeout()));
if (request.hasParam("wait_for_metadata_version")) {
clusterStateRequest.waitForMetaDataVersion(request.paramAsLong("wait_for_metadata_version", 0));
}
clusterStateRequest.waitForTimeout(request.paramAsTime("wait_for_timeout", ClusterStateRequest.DEFAULT_WAIT_FOR_NODE_TIMEOUT));
final String[] indices = Strings.splitStringByCommaToArray(request.param("indices", "_all"));
boolean isAllIndicesOnly = indices.length == 1 && "_all".equals(indices[0]);
@ -94,6 +98,9 @@ public class RestClusterStateAction extends BaseRestHandler {
@Override
public RestResponse buildResponse(ClusterStateResponse response, XContentBuilder builder) throws Exception {
builder.startObject();
if (clusterStateRequest.waitForMetaDataVersion() != null) {
builder.field(Fields.WAIT_FOR_TIMED_OUT, response.isWaitForTimedOut());
}
builder.field(Fields.CLUSTER_NAME, response.getClusterName().value());
builder.humanReadableField(Fields.CLUSTER_STATE_SIZE_IN_BYTES, Fields.CLUSTER_STATE_SIZE,
response.getTotalCompressedSize());
@ -124,6 +131,7 @@ public class RestClusterStateAction extends BaseRestHandler {
}
static final class Fields {
static final String WAIT_FOR_TIMED_OUT = "wait_for_timed_out";
static final String CLUSTER_NAME = "cluster_name";
static final String CLUSTER_STATE_SIZE = "compressed_size";
static final String CLUSTER_STATE_SIZE_IN_BYTES = "compressed_size_in_bytes";

View File

@ -0,0 +1,80 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.cluster.state;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESSingleNodeTestCase;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.nullValue;
public class ClusterStateApiTests extends ESSingleNodeTestCase {
public void testWaitForMetaDataVersion() throws Exception {
ClusterStateRequest clusterStateRequest = new ClusterStateRequest();
clusterStateRequest.waitForTimeout(TimeValue.timeValueHours(1));
ActionFuture<ClusterStateResponse> future1 = client().admin().cluster().state(clusterStateRequest);
assertThat(future1.isDone(), is(true));
assertThat(future1.actionGet().isWaitForTimedOut(), is(false));
long metadataVersion = future1.actionGet().getState().getMetaData().version();
// Verify that cluster state api returns after the cluster settings have been updated:
clusterStateRequest = new ClusterStateRequest();
clusterStateRequest.waitForMetaDataVersion(metadataVersion + 1);
ActionFuture<ClusterStateResponse> future2 = client().admin().cluster().state(clusterStateRequest);
assertThat(future2.isDone(), is(false));
ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest();
// Pick an arbitrary dynamic cluster setting and change it. Just to get metadata version incremented:
updateSettingsRequest.transientSettings(Settings.builder().put("cluster.max_shards_per_node", 999));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
assertBusy(() -> {
assertThat(future2.isDone(), is(true));
});
ClusterStateResponse response = future2.actionGet();
assertThat(response.isWaitForTimedOut(), is(false));
assertThat(response.getState().metaData().version(), equalTo(metadataVersion + 1));
// Verify that the timed out property has been set"
metadataVersion = response.getState().getMetaData().version();
clusterStateRequest.waitForMetaDataVersion(metadataVersion + 1);
clusterStateRequest.waitForTimeout(TimeValue.timeValueSeconds(1)); // Fail fast
ActionFuture<ClusterStateResponse> future3 = client().admin().cluster().state(clusterStateRequest);
assertBusy(() -> {
assertThat(future3.isDone(), is(true));
});
response = future3.actionGet();
assertThat(response.isWaitForTimedOut(), is(true));
assertThat(response.getState(), nullValue());
// Remove transient setting, otherwise test fails with the reason that this test leaves state behind:
updateSettingsRequest = new ClusterUpdateSettingsRequest();
updateSettingsRequest.transientSettings(Settings.builder().put("cluster.max_shards_per_node", (String) null));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
}
}

View File

@ -23,6 +23,7 @@ import org.elasticsearch.Version;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.VersionUtils;
@ -42,6 +43,16 @@ public class ClusterStateRequestTests extends ESTestCase {
Version testVersion = VersionUtils.randomVersionBetween(random(),
Version.CURRENT.minimumCompatibilityVersion(), Version.CURRENT);
// TODO: change version to V_6_6_0 after backporting:
if (testVersion.onOrAfter(Version.V_7_0_0)) {
if (randomBoolean()) {
clusterStateRequest.waitForMetaDataVersion(randomLongBetween(1, Long.MAX_VALUE));
}
if (randomBoolean()) {
clusterStateRequest.waitForTimeout(new TimeValue(randomNonNegativeLong()));
}
}
BytesStreamOutput output = new BytesStreamOutput();
output.setVersion(testVersion);
clusterStateRequest.writeTo(output);
@ -56,9 +67,21 @@ public class ClusterStateRequestTests extends ESTestCase {
assertThat(deserializedCSRequest.blocks(), equalTo(clusterStateRequest.blocks()));
assertThat(deserializedCSRequest.indices(), equalTo(clusterStateRequest.indices()));
assertOptionsMatch(deserializedCSRequest.indicesOptions(), clusterStateRequest.indicesOptions());
// TODO: change version to V_6_6_0 after backporting:
if (testVersion.onOrAfter(Version.V_7_0_0)) {
assertThat(deserializedCSRequest.waitForMetaDataVersion(), equalTo(clusterStateRequest.waitForMetaDataVersion()));
assertThat(deserializedCSRequest.waitForTimeout(), equalTo(clusterStateRequest.waitForTimeout()));
}
}
}
public void testWaitForMetaDataVersion() {
ClusterStateRequest clusterStateRequest = new ClusterStateRequest();
expectThrows(IllegalArgumentException.class,
() -> clusterStateRequest.waitForMetaDataVersion(randomLongBetween(Long.MIN_VALUE, 0)));
clusterStateRequest.waitForMetaDataVersion(randomLongBetween(1, Long.MAX_VALUE));
}
private static void assertOptionsMatch(IndicesOptions in, IndicesOptions out) {
assertThat(in.ignoreUnavailable(), equalTo(out.ignoreUnavailable()));
assertThat(in.expandWildcardsClosed(), equalTo(out.expandWildcardsClosed()));

View File

@ -0,0 +1,55 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.cluster.state;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.test.AbstractStreamableTestCase;
public class ClusterStateResponseTests extends AbstractStreamableTestCase<ClusterStateResponse> {
@Override
protected ClusterStateResponse createBlankInstance() {
return new ClusterStateResponse();
}
@Override
protected ClusterStateResponse createTestInstance() {
ClusterName clusterName = new ClusterName(randomAlphaOfLength(4));
ClusterState clusterState = null;
if (randomBoolean()) {
ClusterState.Builder clusterStateBuilder = ClusterState.builder(clusterName)
.version(randomNonNegativeLong());
if (randomBoolean()) {
clusterStateBuilder.nodes(DiscoveryNodes.builder().masterNodeId(randomAlphaOfLength(4)).build());
}
clusterState = clusterStateBuilder.build();
}
return new ClusterStateResponse(clusterName, clusterState, randomNonNegativeLong(), randomBoolean());
}
@Override
protected NamedWriteableRegistry getNamedWriteableRegistry() {
return new NamedWriteableRegistry(ClusterModule.getNamedWriteables());
}
}

View File

@ -99,7 +99,7 @@ abstract class FailAndRetryMockTransport<Response extends TransportResponse> imp
} else if (ClusterStateAction.NAME.equals(action)) {
TransportResponseHandler transportResponseHandler = responseHandlers.onResponseReceived(requestId, listener);
ClusterState clusterState = getMockClusterState(node);
transportResponseHandler.handleResponse(new ClusterStateResponse(clusterName, clusterState, 0L));
transportResponseHandler.handleResponse(new ClusterStateResponse(clusterName, clusterState, 0L, false));
} else if (TransportService.HANDSHAKE_ACTION_NAME.equals(action)) {
TransportResponseHandler transportResponseHandler = responseHandlers.onResponseReceived(requestId, listener);
Version version = node.getVersion();

View File

@ -175,7 +175,7 @@ public class TransportClientHeadersTests extends AbstractClientHeadersTestCase {
address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(),
Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT)));
((TransportResponseHandler<ClusterStateResponse>) handler)
.handleResponse(new ClusterStateResponse(cluster1, builder.build(), 0L));
.handleResponse(new ClusterStateResponse(cluster1, builder.build(), 0L, false));
clusterStateLatch.countDown();
} else if (TransportService.HANDSHAKE_ACTION_NAME .equals(action)) {
((TransportResponseHandler<TransportService.HandshakeResponse>) handler).handleResponse(

View File

@ -416,7 +416,7 @@ public class TransportClientNodesServiceTests extends ESTestCase {
}
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(transportService.getLocalDiscoNode()).build();
ClusterState build = ClusterState.builder(ClusterName.DEFAULT).nodes(discoveryNodes).build();
channel.sendResponse(new ClusterStateResponse(ClusterName.DEFAULT, build, 0L));
channel.sendResponse(new ClusterStateResponse(ClusterName.DEFAULT, build, 0L, false));
}
void blockRequest() {

View File

@ -60,7 +60,7 @@ public class RestNodesActionTests extends ESTestCase {
ClusterState clusterState = mock(ClusterState.class);
when(clusterState.nodes()).thenReturn(discoveryNodes);
ClusterStateResponse clusterStateResponse = new ClusterStateResponse(clusterName, clusterState, randomNonNegativeLong());
ClusterStateResponse clusterStateResponse = new ClusterStateResponse(clusterName, clusterState, randomNonNegativeLong(), false);
NodesInfoResponse nodesInfoResponse = new NodesInfoResponse(clusterName, Collections.emptyList(), Collections.emptyList());
NodesStatsResponse nodesStatsResponse = new NodesStatsResponse(clusterName, Collections.emptyList(), Collections.emptyList());

View File

@ -136,7 +136,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
builder.add(node);
}
ClusterState build = ClusterState.builder(clusterName).nodes(builder.build()).build();
channel.sendResponse(new ClusterStateResponse(clusterName, build, 0L));
channel.sendResponse(new ClusterStateResponse(clusterName, build, 0L, false));
});
newService.start();
newService.acceptIncomingRequests();