Discovery: Add two phased commit to Cluster State publishing
When publishing a new cluster state, the master will send it to all the node of the cluster, noting down how many *master* nodes responded successfully. The nodes do not yet process the new cluster state, but rather park it in memory. As soon as at least minimum master nodes have ack-ed the cluster state change, it is committed and a commit request is sent to all the node that responded so far (and will respond in the future). Once receiving the commit requests the nodes continue to process the cluster state change as they did before this change. A few notable comments: 1. For this change to have effect, min master nodes must be configured. 2. All basic cluster state validation is done in the first phase of publish and is thus now part of `ShardOperationResult` 3. A new `COMMIT_TIMEOUT` settings is introduced, dictating how long a master should wait for nodes to ack the first phase. Unlike `PUBLISH_TIMEOUT`, if waiting for a commit times out, the cluster state change will be rejected. 4. Failing to achieve a min master node of acks, will cause the master to step down as it clearly doesn't have enough active followers. 5. Previously there was a short window between the moment a master lost it's followers and it stepping down because of node fault detection failures. In this short window, the master could process any change (but fail to publish it). This PR closes this gap to 0. 6. A dedicated pending cluster states queue was added to keep pending non-comitted cluster states and manage the logic around processing committed cluster states. See #13303 for details. Closes #13062 , Closes #13303
This commit is contained in:
commit
15f4863ef6
|
@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
import org.elasticsearch.common.logging.support.LoggerMessageFormat;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
@ -595,7 +596,8 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
|
|||
ResourceNotFoundException.class,
|
||||
IndexNotFoundException.class,
|
||||
ShardNotFoundException.class,
|
||||
NotSerializableExceptionWrapper.class
|
||||
NotSerializableExceptionWrapper.class,
|
||||
Discovery.FailedToCommitClusterStateException.class
|
||||
};
|
||||
Map<String, Constructor<? extends ElasticsearchException>> mapping = new HashMap<>(exceptions.length);
|
||||
for (Class<? extends ElasticsearchException> e : exceptions) {
|
||||
|
|
|
@ -191,6 +191,7 @@ public class ClusterModule extends AbstractModule {
|
|||
registerClusterDynamicSetting(DestructiveOperations.REQUIRES_NAME, Validator.EMPTY);
|
||||
registerClusterDynamicSetting(DiscoverySettings.PUBLISH_TIMEOUT, Validator.TIME_NON_NEGATIVE);
|
||||
registerClusterDynamicSetting(DiscoverySettings.PUBLISH_DIFF_ENABLE, Validator.BOOLEAN);
|
||||
registerClusterDynamicSetting(DiscoverySettings.COMMIT_TIMEOUT, Validator.TIME_NON_NEGATIVE);
|
||||
registerClusterDynamicSetting(HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, Validator.MEMORY_SIZE);
|
||||
registerClusterDynamicSetting(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, Validator.MEMORY_SIZE);
|
||||
registerClusterDynamicSetting(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, Validator.NON_NEGATIVE_DOUBLE);
|
||||
|
|
|
@ -52,10 +52,7 @@ import org.elasticsearch.discovery.local.LocalDiscovery;
|
|||
import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.*;
|
||||
|
||||
/**
|
||||
* Represents the current state of the cluster.
|
||||
|
@ -256,7 +253,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
|
|||
}
|
||||
|
||||
// Used for testing and logging to determine how this cluster state was send over the wire
|
||||
boolean wasReadFromDiff() {
|
||||
public boolean wasReadFromDiff() {
|
||||
return wasReadFromDiff;
|
||||
}
|
||||
|
||||
|
@ -296,6 +293,16 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* a cluster state supersedes another state iff they are from the same master and the version this state is higher thant the other state.
|
||||
* <p/>
|
||||
* In essence that means that all the changes from the other cluster state are also reflected by the current one
|
||||
*/
|
||||
public boolean supersedes(ClusterState other) {
|
||||
return this.nodes().masterNodeId() != null && this.nodes().masterNodeId().equals(other.nodes().masterNodeId()) && this.version() > other.version();
|
||||
|
||||
}
|
||||
|
||||
public enum Metric {
|
||||
VERSION("version"),
|
||||
MASTER_NODE("master_node"),
|
||||
|
@ -814,6 +821,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
|
|||
builder.fromDiff(true);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.elasticsearch.common.logging.ESLogger;
|
|||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.text.StringText;
|
||||
import org.elasticsearch.common.transport.BoundTransportAddress;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.*;
|
||||
|
@ -482,8 +481,14 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
|||
// we publish here before we send a notification to all the listeners, since if it fails
|
||||
// we don't want to notify
|
||||
if (newClusterState.nodes().localNodeMaster()) {
|
||||
logger.debug("publishing cluster state version {}", newClusterState.version());
|
||||
discoveryService.publish(clusterChangedEvent, ackListener);
|
||||
logger.debug("publishing cluster state version [{}]", newClusterState.version());
|
||||
try {
|
||||
discoveryService.publish(clusterChangedEvent, ackListener);
|
||||
} catch (Discovery.FailedToCommitClusterStateException t) {
|
||||
logger.warn("failing [{}]: failed to commit cluster state version [{}]", t, source, newClusterState.version());
|
||||
updateTask.onFailure(source, t);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// update the current cluster state
|
||||
|
|
|
@ -19,15 +19,17 @@
|
|||
|
||||
package org.elasticsearch.discovery;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.RoutingService;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.component.LifecycleComponent;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A pluggable module allowing to implement discovery of other nodes, publishing of the cluster
|
||||
* state to all nodes, electing a master of the cluster that raises cluster state change
|
||||
|
@ -60,11 +62,29 @@ public interface Discovery extends LifecycleComponent<Discovery> {
|
|||
*
|
||||
* The {@link AckListener} allows to keep track of the ack received from nodes, and verify whether
|
||||
* they updated their own cluster state or not.
|
||||
*
|
||||
* The method is guaranteed to throw a {@link FailedToCommitClusterStateException} if the change is not committed and should be rejected.
|
||||
* Any other exception signals the something wrong happened but the change is committed.
|
||||
*/
|
||||
void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener);
|
||||
|
||||
public static interface AckListener {
|
||||
interface AckListener {
|
||||
void onNodeAck(DiscoveryNode node, @Nullable Throwable t);
|
||||
void onTimeout();
|
||||
}
|
||||
|
||||
class FailedToCommitClusterStateException extends ElasticsearchException {
|
||||
|
||||
public FailedToCommitClusterStateException(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
}
|
||||
|
||||
public FailedToCommitClusterStateException(String msg, Object... args) {
|
||||
super(msg, args);
|
||||
}
|
||||
|
||||
public FailedToCommitClusterStateException(String msg, Throwable cause, Object... args) {
|
||||
super(msg, cause, args);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,11 +35,22 @@ import java.util.EnumSet;
|
|||
*/
|
||||
public class DiscoverySettings extends AbstractComponent {
|
||||
|
||||
/**
|
||||
* sets the timeout for a complete publishing cycle, including both sending and committing. the master
|
||||
* will continute to process the next cluster state update after this time has elapsed
|
||||
**/
|
||||
public static final String PUBLISH_TIMEOUT = "discovery.zen.publish_timeout";
|
||||
|
||||
/**
|
||||
* sets the timeout for receiving enough acks for a specific cluster state and committing it. failing
|
||||
* to receive responses within this window will cause the cluster state change to be rejected.
|
||||
*/
|
||||
public static final String COMMIT_TIMEOUT = "discovery.zen.commit_timeout";
|
||||
public static final String NO_MASTER_BLOCK = "discovery.zen.no_master_block";
|
||||
public static final String PUBLISH_DIFF_ENABLE = "discovery.zen.publish_diff.enable";
|
||||
|
||||
public static final TimeValue DEFAULT_PUBLISH_TIMEOUT = TimeValue.timeValueSeconds(30);
|
||||
public static final TimeValue DEFAULT_COMMIT_TIMEOUT = TimeValue.timeValueSeconds(30);
|
||||
public static final String DEFAULT_NO_MASTER_BLOCK = "write";
|
||||
public final static int NO_MASTER_BLOCK_ID = 2;
|
||||
public final static boolean DEFAULT_PUBLISH_DIFF_ENABLE = true;
|
||||
|
@ -48,15 +59,17 @@ public class DiscoverySettings extends AbstractComponent {
|
|||
public final static ClusterBlock NO_MASTER_BLOCK_WRITES = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, false, RestStatus.SERVICE_UNAVAILABLE, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE));
|
||||
|
||||
private volatile ClusterBlock noMasterBlock;
|
||||
private volatile TimeValue publishTimeout = DEFAULT_PUBLISH_TIMEOUT;
|
||||
private volatile boolean publishDiff = DEFAULT_PUBLISH_DIFF_ENABLE;
|
||||
private volatile TimeValue publishTimeout;
|
||||
private volatile TimeValue commitTimeout;
|
||||
private volatile boolean publishDiff;
|
||||
|
||||
@Inject
|
||||
public DiscoverySettings(Settings settings, NodeSettingsService nodeSettingsService) {
|
||||
super(settings);
|
||||
nodeSettingsService.addListener(new ApplySettings());
|
||||
this.noMasterBlock = parseNoMasterBlock(settings.get(NO_MASTER_BLOCK, DEFAULT_NO_MASTER_BLOCK));
|
||||
this.publishTimeout = settings.getAsTime(PUBLISH_TIMEOUT, publishTimeout);
|
||||
this.publishTimeout = settings.getAsTime(PUBLISH_TIMEOUT, DEFAULT_PUBLISH_TIMEOUT);
|
||||
this.commitTimeout = settings.getAsTime(COMMIT_TIMEOUT, new TimeValue(Math.min(DEFAULT_COMMIT_TIMEOUT.millis(), publishTimeout.millis())));
|
||||
this.publishDiff = settings.getAsBoolean(PUBLISH_DIFF_ENABLE, DEFAULT_PUBLISH_DIFF_ENABLE);
|
||||
}
|
||||
|
||||
|
@ -67,6 +80,10 @@ public class DiscoverySettings extends AbstractComponent {
|
|||
return publishTimeout;
|
||||
}
|
||||
|
||||
public TimeValue getCommitTimeout() {
|
||||
return commitTimeout;
|
||||
}
|
||||
|
||||
public ClusterBlock getNoMasterBlock() {
|
||||
return noMasterBlock;
|
||||
}
|
||||
|
@ -81,6 +98,17 @@ public class DiscoverySettings extends AbstractComponent {
|
|||
if (newPublishTimeout.millis() != publishTimeout.millis()) {
|
||||
logger.info("updating [{}] from [{}] to [{}]", PUBLISH_TIMEOUT, publishTimeout, newPublishTimeout);
|
||||
publishTimeout = newPublishTimeout;
|
||||
if (settings.getAsTime(COMMIT_TIMEOUT, null) == null && commitTimeout.millis() > publishTimeout.millis()) {
|
||||
logger.info("reducing default [{}] to [{}] due to publish timeout change", COMMIT_TIMEOUT, publishTimeout);
|
||||
commitTimeout = publishTimeout;
|
||||
}
|
||||
}
|
||||
}
|
||||
TimeValue newCommitTimeout = settings.getAsTime(COMMIT_TIMEOUT, null);
|
||||
if (newCommitTimeout != null) {
|
||||
if (newCommitTimeout.millis() != commitTimeout.millis()) {
|
||||
logger.info("updating [{}] from [{}] to [{}]", COMMIT_TIMEOUT, commitTimeout, newCommitTimeout);
|
||||
commitTimeout = newCommitTimeout;
|
||||
}
|
||||
}
|
||||
String newNoMasterBlockValue = settings.get(NO_MASTER_BLOCK);
|
||||
|
|
|
@ -20,14 +20,7 @@
|
|||
package org.elasticsearch.discovery.local;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateNonMasterUpdateTask;
|
||||
import org.elasticsearch.cluster.Diff;
|
||||
import org.elasticsearch.cluster.IncompatibleClusterStateVersionException;
|
||||
import org.elasticsearch.cluster.ProcessedClusterStateNonMasterUpdateTask;
|
||||
import org.elasticsearch.cluster.*;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodeService;
|
||||
|
@ -42,17 +35,11 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.discovery.AckClusterStatePublishResponseHandler;
|
||||
import org.elasticsearch.discovery.BlockingClusterStatePublishResponseHandler;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoveryService;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.InitialStateDiscoveryListener;
|
||||
import org.elasticsearch.discovery.*;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Objects;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
@ -343,9 +330,9 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
|
|||
}
|
||||
try {
|
||||
newNodeSpecificClusterState = discovery.lastProcessedClusterState.readDiffFrom(StreamInput.wrap(clusterStateDiffBytes)).apply(discovery.lastProcessedClusterState);
|
||||
logger.debug("sending diff cluster state version with size {} to [{}]", clusterStateDiffBytes.length, discovery.localNode.getName());
|
||||
logger.trace("sending diff cluster state version [{}] with size {} to [{}]", clusterState.version(), clusterStateDiffBytes.length, discovery.localNode.getName());
|
||||
} catch (IncompatibleClusterStateVersionException ex) {
|
||||
logger.warn("incompatible cluster state version - resending complete cluster state", ex);
|
||||
logger.warn("incompatible cluster state version [{}] - resending complete cluster state", ex, clusterState.version());
|
||||
}
|
||||
}
|
||||
if (newNodeSpecificClusterState == null) {
|
||||
|
@ -367,7 +354,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
|
|||
discovery.clusterService.submitStateUpdateTask("local-disco-receive(from master)", new ProcessedClusterStateNonMasterUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
if (nodeSpecificClusterState.version() < currentState.version() && Objects.equals(nodeSpecificClusterState.nodes().masterNodeId(), currentState.nodes().masterNodeId())) {
|
||||
if (currentState.supersedes(nodeSpecificClusterState)) {
|
||||
return currentState;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,16 +19,10 @@
|
|||
|
||||
package org.elasticsearch.discovery.zen;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateNonMasterUpdateTask;
|
||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.ProcessedClusterStateNonMasterUpdateTask;
|
||||
import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.*;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
|
@ -47,7 +41,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.InitialStateDiscoveryListener;
|
||||
|
@ -62,22 +55,13 @@ import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
|
|||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.node.settings.NodeSettingsService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.EmptyTransportResponseHandler;
|
||||
import org.elasticsearch.transport.TransportChannel;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
import org.elasticsearch.transport.TransportRequestHandler;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.transport.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -199,7 +183,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
this.nodesFD = new NodesFaultDetection(settings, threadPool, transportService, clusterName);
|
||||
this.nodesFD.addListener(new NodeFaultDetectionListener());
|
||||
|
||||
this.publishClusterState = new PublishClusterStateAction(settings, transportService, this, new NewClusterStateListener(), discoverySettings);
|
||||
this.publishClusterState = new PublishClusterStateAction(settings, transportService, this, new NewPendingClusterStateListener(), discoverySettings, clusterName);
|
||||
this.pingService.setPingContextProvider(this);
|
||||
this.membership = new MembershipAction(settings, clusterService, transportService, this, new MembershipListener());
|
||||
|
||||
|
@ -329,7 +313,25 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
throw new IllegalStateException("Shouldn't publish state when not master");
|
||||
}
|
||||
nodesFD.updateNodesAndPing(clusterChangedEvent.state());
|
||||
publishClusterState.publish(clusterChangedEvent, ackListener);
|
||||
try {
|
||||
publishClusterState.publish(clusterChangedEvent, electMaster.minimumMasterNodes(), ackListener);
|
||||
} catch (FailedToCommitClusterStateException t) {
|
||||
// cluster service logs a WARN message
|
||||
logger.debug("failed to publish cluster state version [{}] (not enough nodes acknowledged, min master nodes [{}])", clusterChangedEvent.state().version(), electMaster.minimumMasterNodes());
|
||||
clusterService.submitStateUpdateTask("zen-disco-failed-to-publish", Priority.IMMEDIATE, new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
return rejoin(currentState, "failed to publish to min_master_nodes");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Throwable t) {
|
||||
logger.error("unexpected failure during [{}]", t, source);
|
||||
}
|
||||
|
||||
});
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -340,6 +342,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
return joinThreadControl.joinThreadActive();
|
||||
}
|
||||
|
||||
|
||||
// used for testing
|
||||
public ClusterState[] pendingClusterStates() {
|
||||
return publishClusterState.pendingStatesQueue().pendingClusterStates();
|
||||
}
|
||||
|
||||
/**
|
||||
* the main function of a join thread. This function is guaranteed to join the cluster
|
||||
* or spawn a new join thread upon failure to do so.
|
||||
|
@ -410,7 +418,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
return joinThreadControl.stopRunningThreadAndRejoin(currentState, "master_switched_while_finalizing_join");
|
||||
}
|
||||
|
||||
// Note: we do not have to start master fault detection here because it's set at {@link #handleNewClusterStateFromMaster }
|
||||
// Note: we do not have to start master fault detection here because it's set at {@link #processNextPendingClusterState }
|
||||
// when the first cluster state arrives.
|
||||
joinThreadControl.markThreadAsDone(currentThread);
|
||||
return currentState;
|
||||
|
@ -616,9 +624,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
.masterNodeId(null).build();
|
||||
|
||||
// flush any pending cluster states from old master, so it will not be set as master again
|
||||
ArrayList<ProcessClusterState> pendingNewClusterStates = new ArrayList<>();
|
||||
processNewClusterStates.drainTo(pendingNewClusterStates);
|
||||
logger.trace("removed [{}] pending cluster states", pendingNewClusterStates.size());
|
||||
publishClusterState.pendingStatesQueue().failAllStatesAndClear(new ElasticsearchException("master left [{}]", reason));
|
||||
|
||||
if (rejoinOnMasterGone) {
|
||||
return rejoin(ClusterState.builder(currentState).nodes(discoveryNodes).build(), "master left (reason = " + reason + ")");
|
||||
|
@ -664,181 +670,98 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
});
|
||||
}
|
||||
|
||||
static class ProcessClusterState {
|
||||
final ClusterState clusterState;
|
||||
volatile boolean processed;
|
||||
void processNextPendingClusterState(String reason) {
|
||||
clusterService.submitStateUpdateTask("zen-disco-receive(from master [" + reason + "])", Priority.URGENT, new ProcessedClusterStateNonMasterUpdateTask() {
|
||||
ClusterState newClusterState = null;
|
||||
|
||||
ProcessClusterState(ClusterState clusterState) {
|
||||
this.clusterState = clusterState;
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
newClusterState = publishClusterState.pendingStatesQueue().getNextClusterStateToProcess();
|
||||
|
||||
private final BlockingQueue<ProcessClusterState> processNewClusterStates = ConcurrentCollections.newBlockingQueue();
|
||||
|
||||
void handleNewClusterStateFromMaster(ClusterState newClusterState, final PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
final ClusterName incomingClusterName = newClusterState.getClusterName();
|
||||
/* The cluster name can still be null if the state comes from a node that is prev 1.1.1*/
|
||||
if (incomingClusterName != null && !incomingClusterName.equals(this.clusterName)) {
|
||||
logger.warn("received cluster state from [{}] which is also master but with a different cluster name [{}]", newClusterState.nodes().masterNode(), incomingClusterName);
|
||||
newStateProcessed.onNewClusterStateFailed(new IllegalStateException("received state from a node that is not part of the cluster"));
|
||||
return;
|
||||
}
|
||||
if (localNodeMaster()) {
|
||||
logger.debug("received cluster state from [{}] which is also master with cluster name [{}]", newClusterState.nodes().masterNode(), incomingClusterName);
|
||||
final ClusterState newState = newClusterState;
|
||||
clusterService.submitStateUpdateTask("zen-disco-master_receive_cluster_state_from_another_master [" + newState.nodes().masterNode() + "]", Priority.URGENT, new ProcessedClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
return handleAnotherMaster(currentState, newState.nodes().masterNode(), newState.version(), "via a new cluster state");
|
||||
// all pending states have been processed
|
||||
if (newClusterState == null) {
|
||||
return currentState;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
newStateProcessed.onNewClusterStateProcessed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Throwable t) {
|
||||
logger.error("unexpected failure during [{}]", t, source);
|
||||
newStateProcessed.onNewClusterStateFailed(t);
|
||||
}
|
||||
|
||||
});
|
||||
} else {
|
||||
if (newClusterState.nodes().localNode() == null) {
|
||||
logger.warn("received a cluster state from [{}] and not part of the cluster, should not happen", newClusterState.nodes().masterNode());
|
||||
newStateProcessed.onNewClusterStateFailed(new IllegalStateException("received state from a node that is not part of the cluster"));
|
||||
} else {
|
||||
|
||||
final ProcessClusterState processClusterState = new ProcessClusterState(newClusterState);
|
||||
processNewClusterStates.add(processClusterState);
|
||||
|
||||
assert newClusterState.nodes().masterNode() != null : "received a cluster state without a master";
|
||||
assert !newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block";
|
||||
|
||||
clusterService.submitStateUpdateTask("zen-disco-receive(from master [" + newClusterState.nodes().masterNode() + "])", Priority.URGENT, new ProcessedClusterStateNonMasterUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
// we already processed it in a previous event
|
||||
if (processClusterState.processed) {
|
||||
return currentState;
|
||||
}
|
||||
if (currentState.nodes().localNodeMaster()) {
|
||||
return handleAnotherMaster(currentState, newClusterState.nodes().masterNode(), newClusterState.version(), "via a new cluster state");
|
||||
}
|
||||
|
||||
// TODO: once improvement that we can do is change the message structure to include version and masterNodeId
|
||||
// at the start, this will allow us to keep the "compressed bytes" around, and only parse the first page
|
||||
// to figure out if we need to use it or not, and only once we picked the latest one, parse the whole state
|
||||
if (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) {
|
||||
return currentState;
|
||||
}
|
||||
|
||||
// check to see that we monitor the correct master of the cluster
|
||||
if (masterFD.masterNode() == null || !masterFD.masterNode().equals(newClusterState.nodes().masterNode())) {
|
||||
masterFD.restart(newClusterState.nodes().masterNode(), "new cluster state received and we are monitoring the wrong master [" + masterFD.masterNode() + "]");
|
||||
}
|
||||
|
||||
if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) {
|
||||
// its a fresh update from the master as we transition from a start of not having a master to having one
|
||||
logger.debug("got first state from fresh master [{}]", newClusterState.nodes().masterNodeId());
|
||||
long count = clusterJoinsCounter.incrementAndGet();
|
||||
logger.trace("updated cluster join cluster to [{}]", count);
|
||||
|
||||
return newClusterState;
|
||||
}
|
||||
|
||||
|
||||
ClusterState updatedState = selectNextStateToProcess(processNewClusterStates);
|
||||
if (updatedState == null) {
|
||||
updatedState = currentState;
|
||||
}
|
||||
if (shouldIgnoreOrRejectNewClusterState(logger, currentState, updatedState)) {
|
||||
return currentState;
|
||||
}
|
||||
// some optimizations to make sure we keep old objects where possible
|
||||
ClusterState.Builder builder = ClusterState.builder(newClusterState);
|
||||
|
||||
// we don't need to do this, since we ping the master, and get notified when it has moved from being a master
|
||||
// because it doesn't have enough master nodes...
|
||||
//if (!electMaster.hasEnoughMasterNodes(newState.nodes())) {
|
||||
// return disconnectFromCluster(newState, "not enough master nodes on new cluster state wreceived from [" + newState.nodes().masterNode() + "]");
|
||||
//}
|
||||
|
||||
// check to see that we monitor the correct master of the cluster
|
||||
if (masterFD.masterNode() == null || !masterFD.masterNode().equals(updatedState.nodes().masterNode())) {
|
||||
masterFD.restart(updatedState.nodes().masterNode(), "new cluster state received and we are monitoring the wrong master [" + masterFD.masterNode() + "]");
|
||||
}
|
||||
|
||||
if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) {
|
||||
// its a fresh update from the master as we transition from a start of not having a master to having one
|
||||
logger.debug("got first state from fresh master [{}]", updatedState.nodes().masterNodeId());
|
||||
long count = clusterJoinsCounter.incrementAndGet();
|
||||
logger.trace("updated cluster join cluster to [{}]", count);
|
||||
|
||||
return updatedState;
|
||||
}
|
||||
|
||||
|
||||
// some optimizations to make sure we keep old objects where possible
|
||||
ClusterState.Builder builder = ClusterState.builder(updatedState);
|
||||
|
||||
// if the routing table did not change, use the original one
|
||||
if (updatedState.routingTable().version() == currentState.routingTable().version()) {
|
||||
builder.routingTable(currentState.routingTable());
|
||||
}
|
||||
// same for metadata
|
||||
if (updatedState.metaData().version() == currentState.metaData().version()) {
|
||||
builder.metaData(currentState.metaData());
|
||||
// if the routing table did not change, use the original one
|
||||
if (newClusterState.routingTable().version() == currentState.routingTable().version()) {
|
||||
builder.routingTable(currentState.routingTable());
|
||||
}
|
||||
// same for metadata
|
||||
if (newClusterState.metaData().version() == currentState.metaData().version()) {
|
||||
builder.metaData(currentState.metaData());
|
||||
} else {
|
||||
// if its not the same version, only copy over new indices or ones that changed the version
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder(newClusterState.metaData()).removeAllIndices();
|
||||
for (IndexMetaData indexMetaData : newClusterState.metaData()) {
|
||||
IndexMetaData currentIndexMetaData = currentState.metaData().index(indexMetaData.index());
|
||||
if (currentIndexMetaData != null && currentIndexMetaData.isSameUUID(indexMetaData.indexUUID()) &&
|
||||
currentIndexMetaData.version() == indexMetaData.version()) {
|
||||
// safe to reuse
|
||||
metaDataBuilder.put(currentIndexMetaData, false);
|
||||
} else {
|
||||
// if its not the same version, only copy over new indices or ones that changed the version
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder(updatedState.metaData()).removeAllIndices();
|
||||
for (IndexMetaData indexMetaData : updatedState.metaData()) {
|
||||
IndexMetaData currentIndexMetaData = currentState.metaData().index(indexMetaData.index());
|
||||
if (currentIndexMetaData != null && currentIndexMetaData.isSameUUID(indexMetaData.indexUUID()) &&
|
||||
currentIndexMetaData.version() == indexMetaData.version()) {
|
||||
// safe to reuse
|
||||
metaDataBuilder.put(currentIndexMetaData, false);
|
||||
} else {
|
||||
metaDataBuilder.put(indexMetaData, false);
|
||||
}
|
||||
}
|
||||
builder.metaData(metaDataBuilder);
|
||||
metaDataBuilder.put(indexMetaData, false);
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
builder.metaData(metaDataBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Throwable t) {
|
||||
logger.error("unexpected failure during [{}]", t, source);
|
||||
newStateProcessed.onNewClusterStateFailed(t);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Throwable t) {
|
||||
logger.error("unexpected failure during [{}]", t, source);
|
||||
if (newClusterState != null) {
|
||||
try {
|
||||
publishClusterState.pendingStatesQueue().markAsFailed(newClusterState, t);
|
||||
} catch (Throwable unexpected) {
|
||||
logger.error("unexpected exception while failing [{}]", unexpected, source);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
sendInitialStateEventIfNeeded();
|
||||
newStateProcessed.onNewClusterStateProcessed();
|
||||
@Override
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
try {
|
||||
sendInitialStateEventIfNeeded();
|
||||
if (newClusterState != null) {
|
||||
publishClusterState.pendingStatesQueue().markAsProcessed(newClusterState);
|
||||
}
|
||||
});
|
||||
} catch (Throwable t) {
|
||||
onFailure(source, t);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Picks the cluster state with highest version with the same master from the queue. All cluster states with
|
||||
* lower versions are ignored. If a cluster state with a different master is seen the processing logic stops and the
|
||||
* last processed state is returned.
|
||||
*/
|
||||
static ClusterState selectNextStateToProcess(Queue<ProcessClusterState> processNewClusterStates) {
|
||||
// try and get the state with the highest version out of all the ones with the same master node id
|
||||
ProcessClusterState stateToProcess = processNewClusterStates.poll();
|
||||
if (stateToProcess == null) {
|
||||
return null;
|
||||
}
|
||||
stateToProcess.processed = true;
|
||||
while (true) {
|
||||
ProcessClusterState potentialState = processNewClusterStates.peek();
|
||||
// nothing else in the queue, bail
|
||||
if (potentialState == null) {
|
||||
break;
|
||||
}
|
||||
// if its not from the same master, then bail
|
||||
if (!Objects.equals(stateToProcess.clusterState.nodes().masterNodeId(), potentialState.clusterState.nodes().masterNodeId())) {
|
||||
break;
|
||||
}
|
||||
// we are going to use it for sure, poll (remove) it
|
||||
potentialState = processNewClusterStates.poll();
|
||||
if (potentialState == null) {
|
||||
// might happen if the queue is drained
|
||||
break;
|
||||
}
|
||||
potentialState.processed = true;
|
||||
|
||||
if (potentialState.clusterState.version() > stateToProcess.clusterState.version()) {
|
||||
// we found a new one
|
||||
stateToProcess = potentialState;
|
||||
}
|
||||
}
|
||||
return stateToProcess.clusterState;
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -848,13 +771,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
* If the second condition fails we ignore the cluster state.
|
||||
*/
|
||||
static boolean shouldIgnoreOrRejectNewClusterState(ESLogger logger, ClusterState currentState, ClusterState newClusterState) {
|
||||
if (currentState.nodes().masterNodeId() == null) {
|
||||
return false;
|
||||
}
|
||||
if (!currentState.nodes().masterNodeId().equals(newClusterState.nodes().masterNodeId())) {
|
||||
logger.warn("received a cluster state from a different master then the current one, rejecting (received {}, current {})", newClusterState.nodes().masterNode(), currentState.nodes().masterNode());
|
||||
throw new IllegalStateException("cluster state from a different master than the current one, rejecting (received " + newClusterState.nodes().masterNode() + ", current " + currentState.nodes().masterNode() + ")");
|
||||
} else if (newClusterState.version() < currentState.version()) {
|
||||
validateStateIsFromCurrentMaster(logger, currentState.nodes(), newClusterState);
|
||||
if (currentState.supersedes(newClusterState)) {
|
||||
// if the new state has a smaller version, and it has the same master node, then no need to process it
|
||||
logger.debug("received a cluster state that has a lower version than the current one, ignoring (received {}, current {})", newClusterState.version(), currentState.version());
|
||||
return true;
|
||||
|
@ -863,6 +781,21 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* In the case we follow an elected master the new cluster state needs to have the same elected master
|
||||
* This method checks for this and throws an exception if needed
|
||||
*/
|
||||
|
||||
public static void validateStateIsFromCurrentMaster(ESLogger logger, DiscoveryNodes currentNodes, ClusterState newClusterState) {
|
||||
if (currentNodes.masterNodeId() == null) {
|
||||
return;
|
||||
}
|
||||
if (!currentNodes.masterNodeId().equals(newClusterState.nodes().masterNodeId())) {
|
||||
logger.warn("received a cluster state from a different master than the current one, rejecting (received {}, current {})", newClusterState.nodes().masterNode(), currentNodes.masterNode());
|
||||
throw new IllegalStateException("cluster state from a different master than the current one, rejecting (received " + newClusterState.nodes().masterNode() + ", current " + currentNodes.masterNode() + ")");
|
||||
}
|
||||
}
|
||||
|
||||
void handleJoinRequest(final DiscoveryNode node, final MembershipAction.JoinCallback callback) {
|
||||
|
||||
if (!transportService.addressSupported(node.address().getClass())) {
|
||||
|
@ -1055,11 +988,11 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
}
|
||||
}
|
||||
|
||||
private class NewClusterStateListener implements PublishClusterStateAction.NewClusterStateListener {
|
||||
private class NewPendingClusterStateListener implements PublishClusterStateAction.NewPendingClusterStateListener {
|
||||
|
||||
@Override
|
||||
public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) {
|
||||
handleNewClusterStateFromMaster(clusterState, newStateProcessed);
|
||||
public void onNewClusterState(String reason) {
|
||||
processNextPendingClusterState(reason);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1093,11 +1026,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
return;
|
||||
}
|
||||
|
||||
// nodes pre 1.4.0 do not send this information
|
||||
if (pingRequest.masterNode() == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (pingsWhileMaster.incrementAndGet() < maxPingsFromAnotherMaster) {
|
||||
logger.trace("got a ping from another master {}. current ping count: [{}]", pingRequest.masterNode(), pingsWhileMaster.get());
|
||||
return;
|
||||
|
@ -1300,4 +1228,4 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,286 @@
|
|||
/*
|
||||
* 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.discovery.zen.publish;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Locale;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A queue that holds all "in-flight" incoming cluster states from the master. Once a master commits a cluster
|
||||
* state, it is made available via {@link #getNextClusterStateToProcess()}. The class also takes care of batching
|
||||
* cluster states for processing and failures.
|
||||
* <p/>
|
||||
* The queue is bound by {@link #maxQueueSize}. When the queue is at capacity and a new cluster state is inserted
|
||||
* the oldest cluster state will be dropped. This is safe because:
|
||||
* 1) Under normal operations, master will publish & commit a cluster state before processing another change (i.e., the queue length is 1)
|
||||
* 2) If the master fails to commit a change, it will step down, causing a master election, which will flush the queue.
|
||||
* 3) In general it's safe to process the incoming cluster state as a replacement to the cluster state that's dropped.
|
||||
* a) If the dropped cluster is from the same master as the incoming one is, it is likely to be superseded by the incoming state (or another state in the queue).
|
||||
* This is only not true in very extreme cases of out of order delivery.
|
||||
* b) If the dropping cluster state is not from the same master, it means that:
|
||||
* i) we are no longer following the master of the dropped cluster state but follow the incoming one
|
||||
* ii) we are no longer following any master, in which case it doesn't matter which cluster state will be processed first.
|
||||
* <p/>
|
||||
* The class is fully thread safe and can be used concurrently.
|
||||
*/
|
||||
public class PendingClusterStatesQueue {
|
||||
|
||||
interface StateProcessedListener {
|
||||
|
||||
void onNewClusterStateProcessed();
|
||||
|
||||
void onNewClusterStateFailed(Throwable t);
|
||||
}
|
||||
|
||||
final ArrayList<ClusterStateContext> pendingStates = new ArrayList<>();
|
||||
final ESLogger logger;
|
||||
final int maxQueueSize;
|
||||
|
||||
public PendingClusterStatesQueue(ESLogger logger, int maxQueueSize) {
|
||||
this.logger = logger;
|
||||
this.maxQueueSize = maxQueueSize;
|
||||
}
|
||||
|
||||
/** Add an incoming, not yet committed cluster state */
|
||||
public synchronized void addPending(ClusterState state) {
|
||||
pendingStates.add(new ClusterStateContext(state));
|
||||
if (pendingStates.size() > maxQueueSize) {
|
||||
ClusterStateContext context = pendingStates.remove(0);
|
||||
logger.warn("dropping pending state [{}]. more than [{}] pending states.", context, maxQueueSize);
|
||||
if (context.committed()) {
|
||||
context.listener.onNewClusterStateFailed(new ElasticsearchException("too many pending states ([{}] pending)", maxQueueSize));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark a previously added cluster state as committed. This will make it available via {@link #getNextClusterStateToProcess()}
|
||||
* When the cluster state is processed (or failed), the supplied listener will be called
|
||||
**/
|
||||
public synchronized ClusterState markAsCommitted(String stateUUID, StateProcessedListener listener) {
|
||||
final ClusterStateContext context = findState(stateUUID);
|
||||
if (context == null) {
|
||||
listener.onNewClusterStateFailed(new IllegalStateException("can't resolve cluster state with uuid [" + stateUUID + "] to commit"));
|
||||
return null;
|
||||
}
|
||||
if (context.committed()) {
|
||||
listener.onNewClusterStateFailed(new IllegalStateException("cluster state with uuid [" + stateUUID + "] is already committed"));
|
||||
return null;
|
||||
}
|
||||
context.markAsCommitted(listener);
|
||||
return context.state;
|
||||
}
|
||||
|
||||
/**
|
||||
* mark that the processing of the given state has failed. All committed states that are {@link ClusterState#supersedes(ClusterState)}-ed
|
||||
* by this failed state, will be failed as well
|
||||
*/
|
||||
public synchronized void markAsFailed(ClusterState state, Throwable reason) {
|
||||
final ClusterStateContext failedContext = findState(state.stateUUID());
|
||||
if (failedContext == null) {
|
||||
throw new IllegalArgumentException("can't resolve failed cluster state with uuid [" + state.stateUUID() + "], version [" + state.version() + "]");
|
||||
}
|
||||
if (failedContext.committed() == false) {
|
||||
throw new IllegalArgumentException("failed cluster state is not committed " + state);
|
||||
}
|
||||
|
||||
// fail all committed states which are batch together with the failed state
|
||||
ArrayList<ClusterStateContext> statesToRemove = new ArrayList<>();
|
||||
for (int index = 0; index < pendingStates.size(); index++) {
|
||||
final ClusterStateContext pendingContext = pendingStates.get(index);
|
||||
if (pendingContext.committed() == false) {
|
||||
continue;
|
||||
}
|
||||
final ClusterState pendingState = pendingContext.state;
|
||||
if (pendingContext.equals(failedContext)) {
|
||||
statesToRemove.add(pendingContext);
|
||||
pendingContext.listener.onNewClusterStateFailed(reason);
|
||||
} else if (state.supersedes(pendingState)) {
|
||||
statesToRemove.add(pendingContext);
|
||||
logger.debug("failing committed state {} together with state {}", pendingContext, failedContext);
|
||||
pendingContext.listener.onNewClusterStateFailed(reason);
|
||||
}
|
||||
}
|
||||
pendingStates.removeAll(statesToRemove);
|
||||
assert findState(state.stateUUID()) == null : "state was marked as processed but can still be found in pending list " + state;
|
||||
}
|
||||
|
||||
/**
|
||||
* indicates that a cluster state was successfully processed. Any committed state that is {@link ClusterState#supersedes(ClusterState)}-ed
|
||||
* by the processed state will be marked as processed as well.
|
||||
* <p/>
|
||||
* NOTE: successfully processing a state indicates we are following the master it came from. Any committed state from another master will
|
||||
* be failed by this method
|
||||
*/
|
||||
public synchronized void markAsProcessed(ClusterState state) {
|
||||
if (findState(state.stateUUID()) == null) {
|
||||
throw new IllegalStateException("can't resolve processed cluster state with uuid [" + state.stateUUID() + "], version [" + state.version() + "]");
|
||||
}
|
||||
final DiscoveryNode currentMaster = state.nodes().masterNode();
|
||||
assert currentMaster != null : "processed cluster state mast have a master. " + state;
|
||||
|
||||
// fail or remove any incoming state from a different master
|
||||
// respond to any committed state from the same master with same or lower version (we processed a higher version)
|
||||
ArrayList<ClusterStateContext> contextsToRemove = new ArrayList<>();
|
||||
for (int index = 0; index < pendingStates.size(); index++) {
|
||||
final ClusterStateContext pendingContext = pendingStates.get(index);
|
||||
final ClusterState pendingState = pendingContext.state;
|
||||
final DiscoveryNode pendingMasterNode = pendingState.nodes().masterNode();
|
||||
if (Objects.equals(currentMaster, pendingMasterNode) == false) {
|
||||
contextsToRemove.add(pendingContext);
|
||||
if (pendingContext.committed()) {
|
||||
// this is a committed state , warn
|
||||
logger.warn("received a cluster state (uuid[{}]/v[{}]) from a different master than the current one, rejecting (received {}, current {})",
|
||||
pendingState.stateUUID(), pendingState.version(),
|
||||
pendingMasterNode, currentMaster);
|
||||
pendingContext.listener.onNewClusterStateFailed(
|
||||
new IllegalStateException("cluster state from a different master than the current one, rejecting (received " + pendingMasterNode + ", current " + currentMaster + ")")
|
||||
);
|
||||
} else {
|
||||
logger.trace("removing non-committed state with uuid[{}]/v[{}] from [{}] - a state from [{}] was successfully processed",
|
||||
pendingState.stateUUID(), pendingState.version(), pendingMasterNode,
|
||||
currentMaster
|
||||
);
|
||||
}
|
||||
} else if (state.supersedes(pendingState) && pendingContext.committed()) {
|
||||
logger.trace("processing pending state uuid[{}]/v[{}] together with state uuid[{}]/v[{}]",
|
||||
pendingState.stateUUID(), pendingState.version(), state.stateUUID(), state.version()
|
||||
);
|
||||
contextsToRemove.add(pendingContext);
|
||||
pendingContext.listener.onNewClusterStateProcessed();
|
||||
} else if (pendingState.stateUUID().equals(state.stateUUID())) {
|
||||
assert pendingContext.committed() : "processed cluster state is not committed " + state;
|
||||
contextsToRemove.add(pendingContext);
|
||||
pendingContext.listener.onNewClusterStateProcessed();
|
||||
}
|
||||
}
|
||||
// now ack the processed state
|
||||
pendingStates.removeAll(contextsToRemove);
|
||||
assert findState(state.stateUUID()) == null : "state was marked as processed but can still be found in pending list " + state;
|
||||
|
||||
}
|
||||
|
||||
ClusterStateContext findState(String stateUUID) {
|
||||
for (int i = 0; i < pendingStates.size(); i++) {
|
||||
final ClusterStateContext context = pendingStates.get(i);
|
||||
if (context.stateUUID().equals(stateUUID)) {
|
||||
return context;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/** clear the incoming queue. any committed state will be failed */
|
||||
public synchronized void failAllStatesAndClear(Throwable reason) {
|
||||
for (ClusterStateContext pendingState : pendingStates) {
|
||||
if (pendingState.committed()) {
|
||||
pendingState.listener.onNewClusterStateFailed(reason);
|
||||
}
|
||||
}
|
||||
pendingStates.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the next committed state to process.
|
||||
* <p/>
|
||||
* The method tries to batch operation by getting the cluster state the highest possible committed states
|
||||
* which succeeds the first committed state in queue (i.e., it comes from the same master).
|
||||
*/
|
||||
public synchronized ClusterState getNextClusterStateToProcess() {
|
||||
if (pendingStates.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
ClusterStateContext stateToProcess = null;
|
||||
int index = 0;
|
||||
for (; index < pendingStates.size(); index++) {
|
||||
ClusterStateContext potentialState = pendingStates.get(index);
|
||||
if (potentialState.committed()) {
|
||||
stateToProcess = potentialState;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (stateToProcess == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// now try to find the highest committed state from the same master
|
||||
for (; index < pendingStates.size(); index++) {
|
||||
ClusterStateContext potentialState = pendingStates.get(index);
|
||||
|
||||
if (potentialState.state.supersedes(stateToProcess.state) && potentialState.committed()) {
|
||||
// we found a new one
|
||||
stateToProcess = potentialState;
|
||||
}
|
||||
}
|
||||
assert stateToProcess.committed() : "should only return committed cluster state. found " + stateToProcess.state;
|
||||
return stateToProcess.state;
|
||||
}
|
||||
|
||||
/** returns all pending states, committed or not */
|
||||
public synchronized ClusterState[] pendingClusterStates() {
|
||||
ArrayList<ClusterState> states = new ArrayList<>();
|
||||
for (ClusterStateContext context : pendingStates) {
|
||||
states.add(context.state);
|
||||
}
|
||||
return states.toArray(new ClusterState[states.size()]);
|
||||
}
|
||||
|
||||
static class ClusterStateContext {
|
||||
final ClusterState state;
|
||||
StateProcessedListener listener;
|
||||
|
||||
ClusterStateContext(ClusterState clusterState) {
|
||||
this.state = clusterState;
|
||||
}
|
||||
|
||||
void markAsCommitted(StateProcessedListener listener) {
|
||||
if (this.listener != null) {
|
||||
throw new IllegalStateException(toString() + "is already committed");
|
||||
}
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
boolean committed() {
|
||||
return listener != null;
|
||||
}
|
||||
|
||||
public String stateUUID() {
|
||||
return state.stateUUID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
Locale.ROOT,
|
||||
"[uuid[%s], v[%d], m[%s]]",
|
||||
stateUUID(),
|
||||
state.version(),
|
||||
state.nodes().masterNodeId()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -19,13 +19,11 @@
|
|||
|
||||
package org.elasticsearch.discovery.zen.publish;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.Diff;
|
||||
import org.elasticsearch.cluster.IncompatibleClusterStateVersionException;
|
||||
import org.elasticsearch.cluster.*;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.compress.Compressor;
|
||||
|
@ -40,21 +38,14 @@ import org.elasticsearch.discovery.BlockingClusterStatePublishResponseHandler;
|
|||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.zen.DiscoveryNodesProvider;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.BytesTransportRequest;
|
||||
import org.elasticsearch.transport.EmptyTransportResponseHandler;
|
||||
import org.elasticsearch.transport.TransportChannel;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportRequestHandler;
|
||||
import org.elasticsearch.transport.TransportRequestOptions;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.transport.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
|
@ -62,176 +53,264 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
*/
|
||||
public class PublishClusterStateAction extends AbstractComponent {
|
||||
|
||||
public static final String ACTION_NAME = "internal:discovery/zen/publish";
|
||||
public static final String SEND_ACTION_NAME = "internal:discovery/zen/publish/send";
|
||||
public static final String COMMIT_ACTION_NAME = "internal:discovery/zen/publish/commit";
|
||||
|
||||
public interface NewClusterStateListener {
|
||||
public static final String SETTINGS_MAX_PENDING_CLUSTER_STATES = "discovery.zen.publish.max_pending_cluster_states";
|
||||
|
||||
interface NewStateProcessed {
|
||||
public interface NewPendingClusterStateListener {
|
||||
|
||||
void onNewClusterStateProcessed();
|
||||
|
||||
void onNewClusterStateFailed(Throwable t);
|
||||
}
|
||||
|
||||
void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed);
|
||||
/** a new cluster state has been committed and is ready to process via {@link #pendingStatesQueue()} */
|
||||
void onNewClusterState(String reason);
|
||||
}
|
||||
|
||||
private final TransportService transportService;
|
||||
private final DiscoveryNodesProvider nodesProvider;
|
||||
private final NewClusterStateListener listener;
|
||||
private final NewPendingClusterStateListener newPendingClusterStatelistener;
|
||||
private final DiscoverySettings discoverySettings;
|
||||
private final ClusterName clusterName;
|
||||
private final PendingClusterStatesQueue pendingStatesQueue;
|
||||
|
||||
public PublishClusterStateAction(Settings settings, TransportService transportService, DiscoveryNodesProvider nodesProvider,
|
||||
NewClusterStateListener listener, DiscoverySettings discoverySettings) {
|
||||
NewPendingClusterStateListener listener, DiscoverySettings discoverySettings, ClusterName clusterName) {
|
||||
super(settings);
|
||||
this.transportService = transportService;
|
||||
this.nodesProvider = nodesProvider;
|
||||
this.listener = listener;
|
||||
this.newPendingClusterStatelistener = listener;
|
||||
this.discoverySettings = discoverySettings;
|
||||
transportService.registerRequestHandler(ACTION_NAME, BytesTransportRequest::new, ThreadPool.Names.SAME, new PublishClusterStateRequestHandler());
|
||||
this.clusterName = clusterName;
|
||||
this.pendingStatesQueue = new PendingClusterStatesQueue(logger, settings.getAsInt(SETTINGS_MAX_PENDING_CLUSTER_STATES, 25));
|
||||
transportService.registerRequestHandler(SEND_ACTION_NAME, BytesTransportRequest::new, ThreadPool.Names.SAME, new SendClusterStateRequestHandler());
|
||||
transportService.registerRequestHandler(COMMIT_ACTION_NAME, CommitClusterStateRequest::new, ThreadPool.Names.SAME, new CommitClusterStateRequestHandler());
|
||||
}
|
||||
|
||||
public void close() {
|
||||
transportService.removeHandler(ACTION_NAME);
|
||||
transportService.removeHandler(SEND_ACTION_NAME);
|
||||
transportService.removeHandler(COMMIT_ACTION_NAME);
|
||||
}
|
||||
|
||||
public void publish(ClusterChangedEvent clusterChangedEvent, final Discovery.AckListener ackListener) {
|
||||
Set<DiscoveryNode> nodesToPublishTo = new HashSet<>(clusterChangedEvent.state().nodes().size());
|
||||
DiscoveryNode localNode = nodesProvider.nodes().localNode();
|
||||
for (final DiscoveryNode node : clusterChangedEvent.state().nodes()) {
|
||||
if (node.equals(localNode)) {
|
||||
continue;
|
||||
public PendingClusterStatesQueue pendingStatesQueue() {
|
||||
return pendingStatesQueue;
|
||||
}
|
||||
|
||||
/**
|
||||
* publishes a cluster change event to other nodes. if at least minMasterNodes acknowledge the change it is committed and will
|
||||
* be processed by the master and the other nodes.
|
||||
* <p/>
|
||||
* The method is guaranteed to throw a {@link Discovery.FailedToCommitClusterStateException} if the change is not committed and should be rejected.
|
||||
* Any other exception signals the something wrong happened but the change is committed.
|
||||
*/
|
||||
public void publish(final ClusterChangedEvent clusterChangedEvent, final int minMasterNodes, final Discovery.AckListener ackListener) throws Discovery.FailedToCommitClusterStateException {
|
||||
final DiscoveryNodes nodes;
|
||||
final SendingController sendingController;
|
||||
final Set<DiscoveryNode> nodesToPublishTo;
|
||||
final Map<Version, BytesReference> serializedStates;
|
||||
final Map<Version, BytesReference> serializedDiffs;
|
||||
final boolean sendFullVersion;
|
||||
try {
|
||||
nodes = clusterChangedEvent.state().nodes();
|
||||
nodesToPublishTo = new HashSet<>(nodes.size());
|
||||
DiscoveryNode localNode = nodes.localNode();
|
||||
final int totalMasterNodes = nodes.masterNodes().size();
|
||||
for (final DiscoveryNode node : nodes) {
|
||||
if (node.equals(localNode) == false) {
|
||||
nodesToPublishTo.add(node);
|
||||
}
|
||||
}
|
||||
sendFullVersion = !discoverySettings.getPublishDiff() || clusterChangedEvent.previousState() == null;
|
||||
serializedStates = new HashMap<>();
|
||||
serializedDiffs = new HashMap<>();
|
||||
|
||||
// we build these early as a best effort not to commit in the case of error.
|
||||
// sadly this is not water tight as it may that a failed diff based publishing to a node
|
||||
// will cause a full serialization based on an older version, which may fail after the
|
||||
// change has been committed.
|
||||
buildDiffAndSerializeStates(clusterChangedEvent.state(), clusterChangedEvent.previousState(),
|
||||
nodesToPublishTo, sendFullVersion, serializedStates, serializedDiffs);
|
||||
|
||||
final BlockingClusterStatePublishResponseHandler publishResponseHandler = new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener);
|
||||
sendingController = new SendingController(clusterChangedEvent.state(), minMasterNodes, totalMasterNodes, publishResponseHandler);
|
||||
} catch (Throwable t) {
|
||||
throw new Discovery.FailedToCommitClusterStateException("unexpected error while preparing to publish", t);
|
||||
}
|
||||
|
||||
try {
|
||||
innerPublish(clusterChangedEvent, nodesToPublishTo, sendingController, sendFullVersion, serializedStates, serializedDiffs);
|
||||
} catch (Discovery.FailedToCommitClusterStateException t) {
|
||||
throw t;
|
||||
} catch (Throwable t) {
|
||||
// try to fail committing, in cause it's still on going
|
||||
if (sendingController.markAsFailed("unexpected error [" + t.getMessage() + "]")) {
|
||||
// signal the change should be rejected
|
||||
throw new Discovery.FailedToCommitClusterStateException("unexpected error [{}]", t, t.getMessage());
|
||||
} else {
|
||||
throw t;
|
||||
}
|
||||
nodesToPublishTo.add(node);
|
||||
}
|
||||
publish(clusterChangedEvent, nodesToPublishTo, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener));
|
||||
}
|
||||
|
||||
private void publish(final ClusterChangedEvent clusterChangedEvent, final Set<DiscoveryNode> nodesToPublishTo,
|
||||
final BlockingClusterStatePublishResponseHandler publishResponseHandler) {
|
||||
|
||||
Map<Version, BytesReference> serializedStates = new HashMap<>();
|
||||
Map<Version, BytesReference> serializedDiffs = new HashMap<>();
|
||||
private void innerPublish(final ClusterChangedEvent clusterChangedEvent, final Set<DiscoveryNode> nodesToPublishTo,
|
||||
final SendingController sendingController, final boolean sendFullVersion,
|
||||
final Map<Version, BytesReference> serializedStates, final Map<Version, BytesReference> serializedDiffs) {
|
||||
|
||||
final ClusterState clusterState = clusterChangedEvent.state();
|
||||
final ClusterState previousState = clusterChangedEvent.previousState();
|
||||
final AtomicBoolean timedOutWaitingForNodes = new AtomicBoolean(false);
|
||||
final TimeValue publishTimeout = discoverySettings.getPublishTimeout();
|
||||
final boolean sendFullVersion = !discoverySettings.getPublishDiff() || previousState == null;
|
||||
Diff<ClusterState> diff = null;
|
||||
|
||||
final long publishingStartInNanos = System.nanoTime();
|
||||
|
||||
for (final DiscoveryNode node : nodesToPublishTo) {
|
||||
|
||||
// try and serialize the cluster state once (or per version), so we don't serialize it
|
||||
// per node when we send it over the wire, compress it while we are at it...
|
||||
// we don't send full version if node didn't exist in the previous version of cluster state
|
||||
if (sendFullVersion || !previousState.nodes().nodeExists(node.id())) {
|
||||
sendFullClusterState(clusterState, serializedStates, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler);
|
||||
sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController);
|
||||
} else {
|
||||
if (diff == null) {
|
||||
diff = clusterState.diff(previousState);
|
||||
}
|
||||
sendClusterStateDiff(clusterState, diff, serializedDiffs, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler);
|
||||
sendClusterStateDiff(clusterState, serializedDiffs, serializedStates, node, publishTimeout, sendingController);
|
||||
}
|
||||
}
|
||||
|
||||
if (publishTimeout.millis() > 0) {
|
||||
// only wait if the publish timeout is configured...
|
||||
sendingController.waitForCommit(discoverySettings.getCommitTimeout());
|
||||
|
||||
try {
|
||||
long timeLeftInNanos = Math.max(0, publishTimeout.nanos() - (System.nanoTime() - publishingStartInNanos));
|
||||
final BlockingClusterStatePublishResponseHandler publishResponseHandler = sendingController.getPublishResponseHandler();
|
||||
sendingController.setPublishingTimedOut(!publishResponseHandler.awaitAllNodes(TimeValue.timeValueNanos(timeLeftInNanos)));
|
||||
if (sendingController.getPublishingTimedOut()) {
|
||||
DiscoveryNode[] pendingNodes = publishResponseHandler.pendingNodes();
|
||||
// everyone may have just responded
|
||||
if (pendingNodes.length > 0) {
|
||||
logger.warn("timed out waiting for all nodes to process published state [{}] (timeout [{}], pending nodes: {})", clusterState.version(), publishTimeout, pendingNodes);
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
// ignore & restore interrupt
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
private void buildDiffAndSerializeStates(ClusterState clusterState, ClusterState previousState, Set<DiscoveryNode> nodesToPublishTo,
|
||||
boolean sendFullVersion, Map<Version, BytesReference> serializedStates, Map<Version, BytesReference> serializedDiffs) {
|
||||
Diff<ClusterState> diff = null;
|
||||
for (final DiscoveryNode node : nodesToPublishTo) {
|
||||
try {
|
||||
timedOutWaitingForNodes.set(!publishResponseHandler.awaitAllNodes(publishTimeout));
|
||||
if (timedOutWaitingForNodes.get()) {
|
||||
DiscoveryNode[] pendingNodes = publishResponseHandler.pendingNodes();
|
||||
// everyone may have just responded
|
||||
if (pendingNodes.length > 0) {
|
||||
logger.warn("timed out waiting for all nodes to process published state [{}] (timeout [{}], pending nodes: {})", clusterState.version(), publishTimeout, pendingNodes);
|
||||
if (sendFullVersion || !previousState.nodes().nodeExists(node.id())) {
|
||||
// will send a full reference
|
||||
if (serializedStates.containsKey(node.version()) == false) {
|
||||
serializedStates.put(node.version(), serializeFullClusterState(clusterState, node.version()));
|
||||
}
|
||||
} else {
|
||||
// will send a diff
|
||||
if (diff == null) {
|
||||
diff = clusterState.diff(previousState);
|
||||
}
|
||||
if (serializedDiffs.containsKey(node.version()) == false) {
|
||||
serializedDiffs.put(node.version(), serializeDiffClusterState(diff, node.version()));
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
// ignore & restore interrupt
|
||||
Thread.currentThread().interrupt();
|
||||
} catch (IOException e) {
|
||||
throw new ElasticsearchException("failed to serialize cluster_state for publishing to node {}", e, node);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void sendFullClusterState(ClusterState clusterState, @Nullable Map<Version, BytesReference> serializedStates,
|
||||
DiscoveryNode node, AtomicBoolean timedOutWaitingForNodes, TimeValue publishTimeout,
|
||||
BlockingClusterStatePublishResponseHandler publishResponseHandler) {
|
||||
BytesReference bytes = null;
|
||||
if (serializedStates != null) {
|
||||
bytes = serializedStates.get(node.version());
|
||||
}
|
||||
private void sendFullClusterState(ClusterState clusterState, Map<Version, BytesReference> serializedStates,
|
||||
DiscoveryNode node, TimeValue publishTimeout, SendingController sendingController) {
|
||||
BytesReference bytes = serializedStates.get(node.version());
|
||||
if (bytes == null) {
|
||||
try {
|
||||
bytes = serializeFullClusterState(clusterState, node.version());
|
||||
if (serializedStates != null) {
|
||||
serializedStates.put(node.version(), bytes);
|
||||
}
|
||||
serializedStates.put(node.version(), bytes);
|
||||
} catch (Throwable e) {
|
||||
logger.warn("failed to serialize cluster_state before publishing it to node {}", e, node);
|
||||
publishResponseHandler.onFailure(node, e);
|
||||
sendingController.onNodeSendFailed(node, e);
|
||||
return;
|
||||
}
|
||||
}
|
||||
publishClusterStateToNode(clusterState, bytes, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler, false);
|
||||
sendClusterStateToNode(clusterState, bytes, node, publishTimeout, sendingController, false, serializedStates);
|
||||
}
|
||||
|
||||
private void sendClusterStateDiff(ClusterState clusterState, Diff diff, Map<Version, BytesReference> serializedDiffs, DiscoveryNode node,
|
||||
AtomicBoolean timedOutWaitingForNodes, TimeValue publishTimeout,
|
||||
BlockingClusterStatePublishResponseHandler publishResponseHandler) {
|
||||
private void sendClusterStateDiff(ClusterState clusterState,
|
||||
Map<Version, BytesReference> serializedDiffs, Map<Version, BytesReference> serializedStates,
|
||||
DiscoveryNode node, TimeValue publishTimeout, SendingController sendingController) {
|
||||
BytesReference bytes = serializedDiffs.get(node.version());
|
||||
if (bytes == null) {
|
||||
try {
|
||||
bytes = serializeDiffClusterState(diff, node.version());
|
||||
serializedDiffs.put(node.version(), bytes);
|
||||
} catch (Throwable e) {
|
||||
logger.warn("failed to serialize diff of cluster_state before publishing it to node {}", e, node);
|
||||
publishResponseHandler.onFailure(node, e);
|
||||
return;
|
||||
}
|
||||
}
|
||||
publishClusterStateToNode(clusterState, bytes, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler, true);
|
||||
assert bytes != null : "failed to find serialized diff for node " + node + " of version [" + node.version() + "]";
|
||||
sendClusterStateToNode(clusterState, bytes, node, publishTimeout, sendingController, true, serializedStates);
|
||||
}
|
||||
|
||||
private void publishClusterStateToNode(final ClusterState clusterState, BytesReference bytes,
|
||||
final DiscoveryNode node, final AtomicBoolean timedOutWaitingForNodes,
|
||||
final TimeValue publishTimeout,
|
||||
final BlockingClusterStatePublishResponseHandler publishResponseHandler,
|
||||
final boolean sendDiffs) {
|
||||
private void sendClusterStateToNode(final ClusterState clusterState, BytesReference bytes,
|
||||
final DiscoveryNode node,
|
||||
final TimeValue publishTimeout,
|
||||
final SendingController sendingController,
|
||||
final boolean sendDiffs, final Map<Version, BytesReference> serializedStates) {
|
||||
try {
|
||||
TransportRequestOptions options = TransportRequestOptions.options().withType(TransportRequestOptions.Type.STATE).withCompress(false);
|
||||
// no need to put a timeout on the options here, because we want the response to eventually be received
|
||||
// and not log an error if it arrives after the timeout
|
||||
transportService.sendRequest(node, ACTION_NAME,
|
||||
new BytesTransportRequest(bytes, node.version()),
|
||||
options, // no need to compress, we already compressed the bytes
|
||||
|
||||
// -> no need to put a timeout on the options here, because we want the response to eventually be received
|
||||
// and not log an error if it arrives after the timeout
|
||||
// -> no need to compress, we already compressed the bytes
|
||||
TransportRequestOptions options = TransportRequestOptions.options().withType(TransportRequestOptions.Type.STATE).withCompress(false);
|
||||
transportService.sendRequest(node, SEND_ACTION_NAME,
|
||||
new BytesTransportRequest(bytes, node.version()),
|
||||
options,
|
||||
new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
|
||||
|
||||
@Override
|
||||
public void handleResponse(TransportResponse.Empty response) {
|
||||
if (timedOutWaitingForNodes.get()) {
|
||||
if (sendingController.getPublishingTimedOut()) {
|
||||
logger.debug("node {} responded for cluster state [{}] (took longer than [{}])", node, clusterState.version(), publishTimeout);
|
||||
}
|
||||
publishResponseHandler.onResponse(node);
|
||||
sendingController.onNodeSendAck(node);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleException(TransportException exp) {
|
||||
if (sendDiffs && exp.unwrapCause() instanceof IncompatibleClusterStateVersionException) {
|
||||
logger.debug("resending full cluster state to node {} reason {}", node, exp.getDetailedMessage());
|
||||
sendFullClusterState(clusterState, null, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler);
|
||||
sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController);
|
||||
} else {
|
||||
logger.debug("failed to send cluster state to {}", exp, node);
|
||||
publishResponseHandler.onFailure(node, exp);
|
||||
sendingController.onNodeSendFailed(node, exp);
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (Throwable t) {
|
||||
logger.warn("error sending cluster state to {}", t, node);
|
||||
publishResponseHandler.onFailure(node, t);
|
||||
sendingController.onNodeSendFailed(node, t);
|
||||
}
|
||||
}
|
||||
|
||||
private void sendCommitToNode(final DiscoveryNode node, final ClusterState clusterState, final SendingController sendingController) {
|
||||
try {
|
||||
logger.trace("sending commit for cluster state (uuid: [{}], version [{}]) to [{}]", clusterState.stateUUID(), clusterState.version(), node);
|
||||
TransportRequestOptions options = TransportRequestOptions.options().withType(TransportRequestOptions.Type.STATE);
|
||||
// no need to put a timeout on the options here, because we want the response to eventually be received
|
||||
// and not log an error if it arrives after the timeout
|
||||
transportService.sendRequest(node, COMMIT_ACTION_NAME,
|
||||
new CommitClusterStateRequest(clusterState.stateUUID()),
|
||||
options,
|
||||
new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
|
||||
|
||||
@Override
|
||||
public void handleResponse(TransportResponse.Empty response) {
|
||||
if (sendingController.getPublishingTimedOut()) {
|
||||
logger.debug("node {} responded to cluster state commit [{}]", node, clusterState.version());
|
||||
}
|
||||
sendingController.getPublishResponseHandler().onResponse(node);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleException(TransportException exp) {
|
||||
logger.debug("failed to commit cluster state (uuid [{}], version [{}]) to {}", exp, clusterState.stateUUID(), clusterState.version(), node);
|
||||
sendingController.getPublishResponseHandler().onFailure(node, exp);
|
||||
}
|
||||
});
|
||||
} catch (Throwable t) {
|
||||
logger.warn("error sending cluster state commit (uuid [{}], version [{}]) to {}", t, clusterState.stateUUID(), clusterState.version(), node);
|
||||
sendingController.getPublishResponseHandler().onFailure(node, t);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static BytesReference serializeFullClusterState(ClusterState clusterState, Version nodeVersion) throws IOException {
|
||||
BytesStreamOutput bStream = new BytesStreamOutput();
|
||||
try (StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream)) {
|
||||
|
@ -252,63 +331,279 @@ public class PublishClusterStateAction extends AbstractComponent {
|
|||
return bStream.bytes();
|
||||
}
|
||||
|
||||
private class PublishClusterStateRequestHandler implements TransportRequestHandler<BytesTransportRequest> {
|
||||
private ClusterState lastSeenClusterState;
|
||||
private Object lastSeenClusterStateMutex = new Object();
|
||||
private ClusterState lastSeenClusterState;
|
||||
|
||||
protected void handleIncomingClusterStateRequest(BytesTransportRequest request, TransportChannel channel) throws IOException {
|
||||
Compressor compressor = CompressorFactory.compressor(request.bytes());
|
||||
StreamInput in;
|
||||
if (compressor != null) {
|
||||
in = compressor.streamInput(request.bytes().streamInput());
|
||||
} else {
|
||||
in = request.bytes().streamInput();
|
||||
}
|
||||
in.setVersion(request.version());
|
||||
synchronized (lastSeenClusterStateMutex) {
|
||||
final ClusterState incomingState;
|
||||
// If true we received full cluster state - otherwise diffs
|
||||
if (in.readBoolean()) {
|
||||
incomingState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode());
|
||||
logger.debug("received full cluster state version [{}] with size [{}]", incomingState.version(), request.bytes().length());
|
||||
} else if (lastSeenClusterState != null) {
|
||||
Diff<ClusterState> diff = lastSeenClusterState.readDiffFrom(in);
|
||||
incomingState = diff.apply(lastSeenClusterState);
|
||||
logger.debug("received diff cluster state version [{}] with uuid [{}], diff size [{}]", incomingState.version(), incomingState.stateUUID(), request.bytes().length());
|
||||
} else {
|
||||
logger.debug("received diff for but don't have any local cluster state - requesting full state");
|
||||
throw new IncompatibleClusterStateVersionException("have no local cluster state");
|
||||
}
|
||||
// sanity check incoming state
|
||||
validateIncomingState(incomingState, lastSeenClusterState);
|
||||
|
||||
pendingStatesQueue.addPending(incomingState);
|
||||
lastSeenClusterState = incomingState;
|
||||
lastSeenClusterState.status(ClusterState.ClusterStateStatus.RECEIVED);
|
||||
}
|
||||
channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
||||
}
|
||||
|
||||
// package private for testing
|
||||
|
||||
/**
|
||||
* does simple sanity check of the incoming cluster state. Throws an exception on rejections.
|
||||
*/
|
||||
void validateIncomingState(ClusterState incomingState, ClusterState lastSeenClusterState) {
|
||||
final ClusterName incomingClusterName = incomingState.getClusterName();
|
||||
if (!incomingClusterName.equals(this.clusterName)) {
|
||||
logger.warn("received cluster state from [{}] which is also master but with a different cluster name [{}]", incomingState.nodes().masterNode(), incomingClusterName);
|
||||
throw new IllegalStateException("received state from a node that is not part of the cluster");
|
||||
}
|
||||
final DiscoveryNodes currentNodes = nodesProvider.nodes();
|
||||
|
||||
if (currentNodes.localNode().equals(incomingState.nodes().localNode()) == false) {
|
||||
logger.warn("received a cluster state from [{}] and not part of the cluster, should not happen", incomingState.nodes().masterNode());
|
||||
throw new IllegalStateException("received state from a node that is not part of the cluster");
|
||||
}
|
||||
|
||||
ZenDiscovery.validateStateIsFromCurrentMaster(logger, currentNodes, incomingState);
|
||||
}
|
||||
|
||||
protected void handleCommitRequest(CommitClusterStateRequest request, final TransportChannel channel) {
|
||||
final ClusterState state = pendingStatesQueue.markAsCommitted(request.stateUUID, new PendingClusterStatesQueue.StateProcessedListener() {
|
||||
@Override
|
||||
public void onNewClusterStateProcessed() {
|
||||
try {
|
||||
// send a response to the master to indicate that this cluster state has been processed post committing it.
|
||||
channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
||||
} catch (Throwable e) {
|
||||
logger.debug("failed to send response on cluster state processed", e);
|
||||
onNewClusterStateFailed(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNewClusterStateFailed(Throwable t) {
|
||||
try {
|
||||
channel.sendResponse(t);
|
||||
} catch (Throwable e) {
|
||||
logger.debug("failed to send response on cluster state processed", e);
|
||||
}
|
||||
}
|
||||
});
|
||||
if (state != null) {
|
||||
newPendingClusterStatelistener.onNewClusterState("master " + state.nodes().masterNode() + " committed version [" + state.version() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
private class SendClusterStateRequestHandler implements TransportRequestHandler<BytesTransportRequest> {
|
||||
|
||||
@Override
|
||||
public void messageReceived(BytesTransportRequest request, final TransportChannel channel) throws Exception {
|
||||
Compressor compressor = CompressorFactory.compressor(request.bytes());
|
||||
StreamInput in;
|
||||
if (compressor != null) {
|
||||
in = compressor.streamInput(request.bytes().streamInput());
|
||||
} else {
|
||||
in = request.bytes().streamInput();
|
||||
}
|
||||
in.setVersion(request.version());
|
||||
synchronized (this) {
|
||||
// If true we received full cluster state - otherwise diffs
|
||||
if (in.readBoolean()) {
|
||||
lastSeenClusterState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode());
|
||||
logger.debug("received full cluster state version {} with size {}", lastSeenClusterState.version(), request.bytes().length());
|
||||
} else if (lastSeenClusterState != null) {
|
||||
Diff<ClusterState> diff = lastSeenClusterState.readDiffFrom(in);
|
||||
lastSeenClusterState = diff.apply(lastSeenClusterState);
|
||||
logger.debug("received diff cluster state version {} with uuid {}, diff size {}", lastSeenClusterState.version(), lastSeenClusterState.stateUUID(), request.bytes().length());
|
||||
} else {
|
||||
logger.debug("received diff for but don't have any local cluster state - requesting full state");
|
||||
throw new IncompatibleClusterStateVersionException("have no local cluster state");
|
||||
}
|
||||
lastSeenClusterState.status(ClusterState.ClusterStateStatus.RECEIVED);
|
||||
}
|
||||
handleIncomingClusterStateRequest(request, channel);
|
||||
}
|
||||
}
|
||||
|
||||
private class CommitClusterStateRequestHandler implements TransportRequestHandler<CommitClusterStateRequest> {
|
||||
@Override
|
||||
public void messageReceived(CommitClusterStateRequest request, final TransportChannel channel) throws Exception {
|
||||
handleCommitRequest(request, channel);
|
||||
}
|
||||
}
|
||||
|
||||
protected static class CommitClusterStateRequest extends TransportRequest {
|
||||
|
||||
String stateUUID;
|
||||
|
||||
public CommitClusterStateRequest() {
|
||||
}
|
||||
|
||||
public CommitClusterStateRequest(String stateUUID) {
|
||||
this.stateUUID = stateUUID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
stateUUID = in.readString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(stateUUID);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Coordinates acknowledgments of the sent cluster state from the different nodes. Commits the change
|
||||
* after `minimum_master_nodes` have successfully responded or fails the entire change. After committing
|
||||
* the cluster state, will trigger a commit message to all nodes that responded previously and responds immediately
|
||||
* to all future acknowledgments.
|
||||
*/
|
||||
class SendingController {
|
||||
|
||||
private final ClusterState clusterState;
|
||||
|
||||
public BlockingClusterStatePublishResponseHandler getPublishResponseHandler() {
|
||||
return publishResponseHandler;
|
||||
}
|
||||
|
||||
private final BlockingClusterStatePublishResponseHandler publishResponseHandler;
|
||||
final ArrayList<DiscoveryNode> sendAckedBeforeCommit = new ArrayList<>();
|
||||
|
||||
// writes and reads of these are protected under synchronization
|
||||
final CountDownLatch committedOrFailedLatch; // 0 count indicates that a decision was made w.r.t committing or failing
|
||||
boolean committed; // true if cluster state was committed
|
||||
int neededMastersToCommit; // number of master nodes acks still needed before committing
|
||||
int pendingMasterNodes; // how many master node still need to respond
|
||||
|
||||
// an external marker to note that the publishing process is timed out. This is useful for proper logging.
|
||||
final AtomicBoolean publishingTimedOut = new AtomicBoolean();
|
||||
|
||||
private SendingController(ClusterState clusterState, int minMasterNodes, int totalMasterNodes, BlockingClusterStatePublishResponseHandler publishResponseHandler) {
|
||||
this.clusterState = clusterState;
|
||||
this.publishResponseHandler = publishResponseHandler;
|
||||
this.neededMastersToCommit = Math.max(0, minMasterNodes - 1); // we are one of the master nodes
|
||||
this.pendingMasterNodes = totalMasterNodes - 1;
|
||||
if (this.neededMastersToCommit > this.pendingMasterNodes) {
|
||||
throw new Discovery.FailedToCommitClusterStateException("not enough masters to ack sent cluster state. [{}] needed , have [{}]", neededMastersToCommit, pendingMasterNodes);
|
||||
}
|
||||
this.committed = neededMastersToCommit == 0;
|
||||
this.committedOrFailedLatch = new CountDownLatch(committed ? 0 : 1);
|
||||
}
|
||||
|
||||
public void waitForCommit(TimeValue commitTimeout) {
|
||||
boolean timedout = false;
|
||||
try {
|
||||
listener.onNewClusterState(lastSeenClusterState, new NewClusterStateListener.NewStateProcessed() {
|
||||
@Override
|
||||
public void onNewClusterStateProcessed() {
|
||||
try {
|
||||
channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
||||
} catch (Throwable e) {
|
||||
logger.debug("failed to send response on cluster state processed", e);
|
||||
}
|
||||
}
|
||||
timedout = committedOrFailedLatch.await(commitTimeout.millis(), TimeUnit.MILLISECONDS) == false;
|
||||
} catch (InterruptedException e) {
|
||||
// the commit check bellow will either translate to an exception or we are committed and we can safely continue
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNewClusterStateFailed(Throwable t) {
|
||||
try {
|
||||
channel.sendResponse(t);
|
||||
} catch (Throwable e) {
|
||||
logger.debug("failed to send response on cluster state processed", e);
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
logger.warn("unexpected error while processing cluster state version [{}]", e, lastSeenClusterState.version());
|
||||
try {
|
||||
channel.sendResponse(e);
|
||||
} catch (Throwable e1) {
|
||||
logger.debug("failed to send response on cluster state processed", e1);
|
||||
if (timedout) {
|
||||
markAsFailed("timed out waiting for commit (commit timeout [" + commitTimeout + "]");
|
||||
}
|
||||
if (isCommitted() == false) {
|
||||
throw new Discovery.FailedToCommitClusterStateException("{} enough masters to ack sent cluster state. [{}] left",
|
||||
timedout ? "timed out while waiting for" : "failed to get", neededMastersToCommit);
|
||||
}
|
||||
}
|
||||
|
||||
synchronized public boolean isCommitted() {
|
||||
return committed;
|
||||
}
|
||||
|
||||
synchronized public void onNodeSendAck(DiscoveryNode node) {
|
||||
if (committed) {
|
||||
assert sendAckedBeforeCommit.isEmpty();
|
||||
sendCommitToNode(node, clusterState, this);
|
||||
} else if (committedOrFailed()) {
|
||||
logger.trace("ignoring ack from [{}] for cluster state version [{}]. already failed", node, clusterState.version());
|
||||
} else {
|
||||
// we're still waiting
|
||||
sendAckedBeforeCommit.add(node);
|
||||
if (node.isMasterNode()) {
|
||||
checkForCommitOrFailIfNoPending(node);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized boolean committedOrFailed() {
|
||||
return committedOrFailedLatch.getCount() == 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* check if enough master node responded to commit the change. fails the commit
|
||||
* if there are no more pending master nodes but not enough acks to commit.
|
||||
*/
|
||||
synchronized private void checkForCommitOrFailIfNoPending(DiscoveryNode masterNode) {
|
||||
logger.trace("master node {} acked cluster state version [{}]. processing ... (current pending [{}], needed [{}])",
|
||||
masterNode, clusterState.version(), pendingMasterNodes, neededMastersToCommit);
|
||||
neededMastersToCommit--;
|
||||
if (neededMastersToCommit == 0) {
|
||||
if (markAsCommitted()) {
|
||||
for (DiscoveryNode nodeToCommit : sendAckedBeforeCommit) {
|
||||
sendCommitToNode(nodeToCommit, clusterState, this);
|
||||
}
|
||||
sendAckedBeforeCommit.clear();
|
||||
}
|
||||
}
|
||||
decrementPendingMasterAcksAndChangeForFailure();
|
||||
}
|
||||
|
||||
synchronized private void decrementPendingMasterAcksAndChangeForFailure() {
|
||||
pendingMasterNodes--;
|
||||
if (pendingMasterNodes == 0 && neededMastersToCommit > 0) {
|
||||
markAsFailed("no more pending master nodes, but failed to reach needed acks ([" + neededMastersToCommit + "] left)");
|
||||
}
|
||||
}
|
||||
|
||||
synchronized public void onNodeSendFailed(DiscoveryNode node, Throwable t) {
|
||||
if (node.isMasterNode()) {
|
||||
logger.trace("master node {} failed to ack cluster state version [{}]. processing ... (current pending [{}], needed [{}])",
|
||||
node, clusterState.version(), pendingMasterNodes, neededMastersToCommit);
|
||||
decrementPendingMasterAcksAndChangeForFailure();
|
||||
}
|
||||
publishResponseHandler.onFailure(node, t);
|
||||
}
|
||||
|
||||
/**
|
||||
* tries and commit the current state, if a decision wasn't made yet
|
||||
*
|
||||
* @return true if successful
|
||||
*/
|
||||
synchronized private boolean markAsCommitted() {
|
||||
if (committedOrFailed()) {
|
||||
return committed;
|
||||
}
|
||||
logger.trace("committing version [{}]", clusterState.version());
|
||||
committed = true;
|
||||
committedOrFailedLatch.countDown();
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* tries marking the publishing as failed, if a decision wasn't made yet
|
||||
*
|
||||
* @return true if the publishing was failed and the cluster state is *not* committed
|
||||
**/
|
||||
synchronized private boolean markAsFailed(String reason) {
|
||||
if (committedOrFailed()) {
|
||||
return committed == false;
|
||||
}
|
||||
logger.trace("failed to commit version [{}]. {}", clusterState.version(), reason);
|
||||
committed = false;
|
||||
committedOrFailedLatch.countDown();
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean getPublishingTimedOut() {
|
||||
return publishingTimedOut.get();
|
||||
}
|
||||
|
||||
public void setPublishingTimedOut(boolean isTimedOut) {
|
||||
publishingTimedOut.set(isTimedOut);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,629 +0,0 @@
|
|||
/*
|
||||
* 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.cluster;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.zen.DiscoveryNodesProvider;
|
||||
import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.node.settings.NodeSettingsService;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportConnectionListener;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.transport.local.LocalTransport;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.emptyIterable;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class ClusterStateDiffPublishingTests extends ESTestCase {
|
||||
|
||||
protected ThreadPool threadPool;
|
||||
protected Map<String, MockNode> nodes = new HashMap<>();
|
||||
|
||||
public static class MockNode {
|
||||
public final DiscoveryNode discoveryNode;
|
||||
public final MockTransportService service;
|
||||
public final PublishClusterStateAction action;
|
||||
public final MockDiscoveryNodesProvider nodesProvider;
|
||||
|
||||
public MockNode(DiscoveryNode discoveryNode, MockTransportService service, PublishClusterStateAction action, MockDiscoveryNodesProvider nodesProvider) {
|
||||
this.discoveryNode = discoveryNode;
|
||||
this.service = service;
|
||||
this.action = action;
|
||||
this.nodesProvider = nodesProvider;
|
||||
}
|
||||
|
||||
public void connectTo(DiscoveryNode node) {
|
||||
service.connectToNode(node);
|
||||
nodesProvider.addNode(node);
|
||||
}
|
||||
}
|
||||
|
||||
public MockNode createMockNode(final String name, Settings settings, Version version) throws Exception {
|
||||
return createMockNode(name, settings, version, new PublishClusterStateAction.NewClusterStateListener() {
|
||||
@Override
|
||||
public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) {
|
||||
logger.debug("Node [{}] onNewClusterState version [{}], uuid [{}]", name, clusterState.version(), clusterState.stateUUID());
|
||||
newStateProcessed.onNewClusterStateProcessed();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public MockNode createMockNode(String name, Settings settings, Version version, PublishClusterStateAction.NewClusterStateListener listener) throws Exception {
|
||||
MockTransportService service = buildTransportService(
|
||||
Settings.builder().put(settings).put("name", name, TransportService.SETTING_TRACE_LOG_INCLUDE, "", TransportService.SETTING_TRACE_LOG_EXCLUDE, "NOTHING").build(),
|
||||
version
|
||||
);
|
||||
DiscoveryNode discoveryNode = new DiscoveryNode(name, name, service.boundAddress().publishAddress(), ImmutableMap.<String, String>of(), version);
|
||||
MockDiscoveryNodesProvider nodesProvider = new MockDiscoveryNodesProvider(discoveryNode);
|
||||
PublishClusterStateAction action = buildPublishClusterStateAction(settings, service, nodesProvider, listener);
|
||||
MockNode node = new MockNode(discoveryNode, service, action, nodesProvider);
|
||||
nodesProvider.addNode(discoveryNode);
|
||||
final CountDownLatch latch = new CountDownLatch(nodes.size() * 2 + 1);
|
||||
TransportConnectionListener waitForConnection = new TransportConnectionListener() {
|
||||
@Override
|
||||
public void onNodeConnected(DiscoveryNode node) {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node) {
|
||||
fail("disconnect should not be called " + node);
|
||||
}
|
||||
};
|
||||
node.service.addConnectionListener(waitForConnection);
|
||||
for (MockNode curNode : nodes.values()) {
|
||||
curNode.service.addConnectionListener(waitForConnection);
|
||||
curNode.connectTo(node.discoveryNode);
|
||||
node.connectTo(curNode.discoveryNode);
|
||||
}
|
||||
node.connectTo(node.discoveryNode);
|
||||
assertThat("failed to wait for all nodes to connect", latch.await(5, TimeUnit.SECONDS), equalTo(true));
|
||||
for (MockNode curNode : nodes.values()) {
|
||||
curNode.service.removeConnectionListener(waitForConnection);
|
||||
}
|
||||
node.service.removeConnectionListener(waitForConnection);
|
||||
if (nodes.put(name, node) != null) {
|
||||
fail("Node with the name " + name + " already exist");
|
||||
}
|
||||
return node;
|
||||
}
|
||||
|
||||
public MockTransportService service(String name) {
|
||||
MockNode node = nodes.get(name);
|
||||
if (node != null) {
|
||||
return node.service;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public PublishClusterStateAction action(String name) {
|
||||
MockNode node = nodes.get(name);
|
||||
if (node != null) {
|
||||
return node.action;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
threadPool = new ThreadPool(getClass().getName());
|
||||
}
|
||||
|
||||
@Override
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
super.tearDown();
|
||||
for (MockNode curNode : nodes.values()) {
|
||||
curNode.action.close();
|
||||
curNode.service.close();
|
||||
}
|
||||
terminate(threadPool);
|
||||
}
|
||||
|
||||
protected MockTransportService buildTransportService(Settings settings, Version version) {
|
||||
MockTransportService transportService = new MockTransportService(settings, new LocalTransport(settings, threadPool, version, new NamedWriteableRegistry()), threadPool);
|
||||
transportService.start();
|
||||
return transportService;
|
||||
}
|
||||
|
||||
protected PublishClusterStateAction buildPublishClusterStateAction(Settings settings, MockTransportService transportService, MockDiscoveryNodesProvider nodesProvider,
|
||||
PublishClusterStateAction.NewClusterStateListener listener) {
|
||||
DiscoverySettings discoverySettings = new DiscoverySettings(settings, new NodeSettingsService(settings));
|
||||
return new PublishClusterStateAction(settings, transportService, nodesProvider, listener, discoverySettings);
|
||||
}
|
||||
|
||||
|
||||
static class MockDiscoveryNodesProvider implements DiscoveryNodesProvider {
|
||||
|
||||
private DiscoveryNodes discoveryNodes = DiscoveryNodes.EMPTY_NODES;
|
||||
|
||||
public MockDiscoveryNodesProvider(DiscoveryNode localNode) {
|
||||
discoveryNodes = DiscoveryNodes.builder().put(localNode).localNodeId(localNode.id()).build();
|
||||
}
|
||||
|
||||
public void addNode(DiscoveryNode node) {
|
||||
discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(node).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DiscoveryNodes nodes() {
|
||||
return discoveryNodes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NodeService nodeService() {
|
||||
assert false;
|
||||
throw new UnsupportedOperationException("Shouldn't be here");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
@TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG")
|
||||
public void testSimpleClusterStatePublishing() throws Exception {
|
||||
MockNewClusterStateListener mockListenerA = new MockNewClusterStateListener();
|
||||
MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, Version.CURRENT, mockListenerA);
|
||||
|
||||
MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener();
|
||||
MockNode nodeB = createMockNode("nodeB", Settings.EMPTY, Version.CURRENT, mockListenerB);
|
||||
|
||||
// Initial cluster state
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build();
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build();
|
||||
|
||||
// cluster state update - add nodeB
|
||||
discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeB.discoveryNode).build();
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build();
|
||||
mockListenerB.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertFalse(clusterState.wasReadFromDiff());
|
||||
}
|
||||
});
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update - add block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build();
|
||||
mockListenerB.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertTrue(clusterState.wasReadFromDiff());
|
||||
assertThat(clusterState.blocks().global().size(), equalTo(1));
|
||||
}
|
||||
});
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update - remove block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).incrementVersion().build();
|
||||
mockListenerB.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertTrue(clusterState.wasReadFromDiff());
|
||||
assertThat(clusterState.blocks().global().size(), equalTo(0));
|
||||
}
|
||||
});
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// Adding new node - this node should get full cluster state while nodeB should still be getting diffs
|
||||
|
||||
MockNewClusterStateListener mockListenerC = new MockNewClusterStateListener();
|
||||
MockNode nodeC = createMockNode("nodeC", Settings.EMPTY, Version.CURRENT, mockListenerC);
|
||||
|
||||
// cluster state update 3 - register node C
|
||||
previousClusterState = clusterState;
|
||||
discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeC.discoveryNode).build();
|
||||
clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build();
|
||||
mockListenerB.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertTrue(clusterState.wasReadFromDiff());
|
||||
assertThat(clusterState.blocks().global().size(), equalTo(0));
|
||||
}
|
||||
});
|
||||
mockListenerC.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
// First state
|
||||
assertFalse(clusterState.wasReadFromDiff());
|
||||
}
|
||||
});
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update 4 - update settings
|
||||
previousClusterState = clusterState;
|
||||
MetaData metaData = MetaData.builder(clusterState.metaData()).transientSettings(Settings.settingsBuilder().put("foo", "bar").build()).build();
|
||||
clusterState = ClusterState.builder(clusterState).metaData(metaData).incrementVersion().build();
|
||||
NewClusterStateExpectation expectation = new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertTrue(clusterState.wasReadFromDiff());
|
||||
assertThat(clusterState.blocks().global().size(), equalTo(0));
|
||||
}
|
||||
};
|
||||
mockListenerB.add(expectation);
|
||||
mockListenerC.add(expectation);
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update - skipping one version change - should request full cluster state
|
||||
previousClusterState = ClusterState.builder(clusterState).incrementVersion().build();
|
||||
clusterState = ClusterState.builder(clusterState).incrementVersion().build();
|
||||
expectation = new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertFalse(clusterState.wasReadFromDiff());
|
||||
}
|
||||
};
|
||||
mockListenerB.add(expectation);
|
||||
mockListenerC.add(expectation);
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update - skipping one version change - should request full cluster state
|
||||
previousClusterState = ClusterState.builder(clusterState).incrementVersion().build();
|
||||
clusterState = ClusterState.builder(clusterState).incrementVersion().build();
|
||||
expectation = new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertFalse(clusterState.wasReadFromDiff());
|
||||
}
|
||||
};
|
||||
mockListenerB.add(expectation);
|
||||
mockListenerC.add(expectation);
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// node B becomes the master and sends a version of the cluster state that goes back
|
||||
discoveryNodes = DiscoveryNodes.builder(discoveryNodes)
|
||||
.put(nodeA.discoveryNode)
|
||||
.put(nodeB.discoveryNode)
|
||||
.put(nodeC.discoveryNode)
|
||||
.build();
|
||||
previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build();
|
||||
clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build();
|
||||
expectation = new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertFalse(clusterState.wasReadFromDiff());
|
||||
}
|
||||
};
|
||||
mockListenerA.add(expectation);
|
||||
mockListenerC.add(expectation);
|
||||
publishStateDiffAndWait(nodeB.action, clusterState, previousClusterState);
|
||||
}
|
||||
|
||||
@Test
|
||||
@TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG")
|
||||
public void testUnexpectedDiffPublishing() throws Exception {
|
||||
|
||||
MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() {
|
||||
@Override
|
||||
public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) {
|
||||
fail("Shouldn't send cluster state to myself");
|
||||
}
|
||||
});
|
||||
|
||||
MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener();
|
||||
MockNode nodeB = createMockNode("nodeB", Settings.EMPTY, Version.CURRENT, mockListenerB);
|
||||
|
||||
// Initial cluster state with both states - the second node still shouldn't get diff even though it's present in the previous cluster state
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).put(nodeB.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build();
|
||||
ClusterState previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build();
|
||||
ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build();
|
||||
mockListenerB.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertFalse(clusterState.wasReadFromDiff());
|
||||
}
|
||||
});
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update - add block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build();
|
||||
mockListenerB.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertTrue(clusterState.wasReadFromDiff());
|
||||
}
|
||||
});
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
}
|
||||
|
||||
@Test
|
||||
@TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG")
|
||||
public void testDisablingDiffPublishing() throws Exception {
|
||||
Settings noDiffPublishingSettings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE, false).build();
|
||||
|
||||
MockNode nodeA = createMockNode("nodeA", noDiffPublishingSettings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() {
|
||||
@Override
|
||||
public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) {
|
||||
fail("Shouldn't send cluster state to myself");
|
||||
}
|
||||
});
|
||||
|
||||
MockNode nodeB = createMockNode("nodeB", noDiffPublishingSettings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() {
|
||||
@Override
|
||||
public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) {
|
||||
logger.debug("Got cluster state update, version [{}], guid [{}], from diff [{}]", clusterState.version(), clusterState.stateUUID(), clusterState.wasReadFromDiff());
|
||||
assertFalse(clusterState.wasReadFromDiff());
|
||||
newStateProcessed.onNewClusterStateProcessed();
|
||||
}
|
||||
});
|
||||
|
||||
// Initial cluster state
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build();
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build();
|
||||
|
||||
// cluster state update - add nodeB
|
||||
discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeB.discoveryNode).build();
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build();
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update - add block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build();
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
@TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG")
|
||||
public void testSimultaneousClusterStatePublishing() throws Exception {
|
||||
int numberOfNodes = randomIntBetween(2, 10);
|
||||
int numberOfIterations = randomIntBetween(50, 200);
|
||||
Settings settings = Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT, "100ms").put(DiscoverySettings.PUBLISH_DIFF_ENABLE, true).build();
|
||||
MockNode[] nodes = new MockNode[numberOfNodes];
|
||||
DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder();
|
||||
for (int i = 0; i < nodes.length; i++) {
|
||||
final String name = "node" + i;
|
||||
nodes[i] = createMockNode(name, settings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() {
|
||||
@Override
|
||||
public synchronized void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) {
|
||||
assertProperMetaDataForVersion(clusterState.metaData(), clusterState.version());
|
||||
if (randomInt(10) < 2) {
|
||||
// Cause timeouts from time to time
|
||||
try {
|
||||
Thread.sleep(randomInt(110));
|
||||
} catch (InterruptedException ex) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
newStateProcessed.onNewClusterStateProcessed();
|
||||
}
|
||||
});
|
||||
discoveryNodesBuilder.put(nodes[i].discoveryNode);
|
||||
}
|
||||
|
||||
AssertingAckListener[] listeners = new AssertingAckListener[numberOfIterations];
|
||||
DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build();
|
||||
MetaData metaData = MetaData.EMPTY_META_DATA;
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("test")).metaData(metaData).build();
|
||||
ClusterState previousState;
|
||||
for (int i = 0; i < numberOfIterations; i++) {
|
||||
previousState = clusterState;
|
||||
metaData = buildMetaDataForVersion(metaData, i + 1);
|
||||
clusterState = ClusterState.builder(clusterState).incrementVersion().metaData(metaData).nodes(discoveryNodes).build();
|
||||
listeners[i] = publishStateDiff(nodes[0].action, clusterState, previousState);
|
||||
}
|
||||
|
||||
for (int i = 0; i < numberOfIterations; i++) {
|
||||
listeners[i].await(1, TimeUnit.SECONDS);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG")
|
||||
public void testSerializationFailureDuringDiffPublishing() throws Exception {
|
||||
|
||||
MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() {
|
||||
@Override
|
||||
public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) {
|
||||
fail("Shouldn't send cluster state to myself");
|
||||
}
|
||||
});
|
||||
|
||||
MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener();
|
||||
MockNode nodeB = createMockNode("nodeB", Settings.EMPTY, Version.CURRENT, mockListenerB);
|
||||
|
||||
// Initial cluster state with both states - the second node still shouldn't get diff even though it's present in the previous cluster state
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).put(nodeB.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build();
|
||||
ClusterState previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build();
|
||||
ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build();
|
||||
mockListenerB.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertFalse(clusterState.wasReadFromDiff());
|
||||
}
|
||||
});
|
||||
publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update - add block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build();
|
||||
mockListenerB.add(new NewClusterStateExpectation() {
|
||||
@Override
|
||||
public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) {
|
||||
assertTrue(clusterState.wasReadFromDiff());
|
||||
}
|
||||
});
|
||||
|
||||
ClusterState unserializableClusterState = new ClusterState(clusterState.version(), clusterState.stateUUID(), clusterState) {
|
||||
@Override
|
||||
public Diff<ClusterState> diff(ClusterState previousState) {
|
||||
return new Diff<ClusterState>() {
|
||||
@Override
|
||||
public ClusterState apply(ClusterState part) {
|
||||
fail("this diff shouldn't be applied");
|
||||
return part;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
throw new IOException("Simulated failure of diff serialization");
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
List<Tuple<DiscoveryNode, Throwable>> errors = publishStateDiff(nodeA.action, unserializableClusterState, previousClusterState).awaitErrors(1, TimeUnit.SECONDS);
|
||||
assertThat(errors.size(), equalTo(1));
|
||||
assertThat(errors.get(0).v2().getMessage(), containsString("Simulated failure of diff serialization"));
|
||||
}
|
||||
|
||||
private MetaData buildMetaDataForVersion(MetaData metaData, long version) {
|
||||
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.builder(metaData.indices());
|
||||
indices.put("test" + version, IndexMetaData.builder("test" + version).settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
|
||||
.numberOfShards((int) version).numberOfReplicas(0).build());
|
||||
return MetaData.builder(metaData)
|
||||
.transientSettings(Settings.builder().put("test", version).build())
|
||||
.indices(indices.build())
|
||||
.build();
|
||||
}
|
||||
|
||||
private void assertProperMetaDataForVersion(MetaData metaData, long version) {
|
||||
for (long i = 1; i <= version; i++) {
|
||||
assertThat(metaData.index("test" + i), notNullValue());
|
||||
assertThat(metaData.index("test" + i).numberOfShards(), equalTo((int) i));
|
||||
}
|
||||
assertThat(metaData.index("test" + (version + 1)), nullValue());
|
||||
assertThat(metaData.transientSettings().get("test"), equalTo(Long.toString(version)));
|
||||
}
|
||||
|
||||
public void publishStateDiffAndWait(PublishClusterStateAction action, ClusterState state, ClusterState previousState) throws InterruptedException {
|
||||
publishStateDiff(action, state, previousState).await(1, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
public AssertingAckListener publishStateDiff(PublishClusterStateAction action, ClusterState state, ClusterState previousState) throws InterruptedException {
|
||||
AssertingAckListener assertingAckListener = new AssertingAckListener(state.nodes().getSize() - 1);
|
||||
ClusterChangedEvent changedEvent = new ClusterChangedEvent("test update", state, previousState);
|
||||
action.publish(changedEvent, assertingAckListener);
|
||||
return assertingAckListener;
|
||||
}
|
||||
|
||||
public static class AssertingAckListener implements Discovery.AckListener {
|
||||
private final List<Tuple<DiscoveryNode, Throwable>> errors = new CopyOnWriteArrayList<>();
|
||||
private final AtomicBoolean timeoutOccured = new AtomicBoolean();
|
||||
private final CountDownLatch countDown;
|
||||
|
||||
public AssertingAckListener(int nodeCount) {
|
||||
countDown = new CountDownLatch(nodeCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) {
|
||||
if (t != null) {
|
||||
errors.add(new Tuple<>(node, t));
|
||||
}
|
||||
countDown.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTimeout() {
|
||||
timeoutOccured.set(true);
|
||||
// Fast forward the counter - no reason to wait here
|
||||
long currentCount = countDown.getCount();
|
||||
for (long i = 0; i < currentCount; i++) {
|
||||
countDown.countDown();
|
||||
}
|
||||
}
|
||||
|
||||
public void await(long timeout, TimeUnit unit) throws InterruptedException {
|
||||
assertThat(awaitErrors(timeout, unit), emptyIterable());
|
||||
}
|
||||
|
||||
public List<Tuple<DiscoveryNode, Throwable>> awaitErrors(long timeout, TimeUnit unit) throws InterruptedException {
|
||||
countDown.await(timeout, unit);
|
||||
assertFalse(timeoutOccured.get());
|
||||
return errors;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public interface NewClusterStateExpectation {
|
||||
void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed);
|
||||
}
|
||||
|
||||
public static class MockNewClusterStateListener implements PublishClusterStateAction.NewClusterStateListener {
|
||||
CopyOnWriteArrayList<NewClusterStateExpectation> expectations = new CopyOnWriteArrayList();
|
||||
|
||||
@Override
|
||||
public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) {
|
||||
final NewClusterStateExpectation expectation;
|
||||
try {
|
||||
expectation = expectations.remove(0);
|
||||
} catch (ArrayIndexOutOfBoundsException ex) {
|
||||
fail("Unexpected cluster state update " + clusterState.prettyPrint());
|
||||
return;
|
||||
}
|
||||
expectation.check(clusterState, newStateProcessed);
|
||||
newStateProcessed.onNewClusterStateProcessed();
|
||||
}
|
||||
|
||||
public void add(NewClusterStateExpectation expectation) {
|
||||
expectations.add(expectation);
|
||||
}
|
||||
}
|
||||
|
||||
public static class DelegatingClusterState extends ClusterState {
|
||||
|
||||
public DelegatingClusterState(ClusterState clusterState) {
|
||||
super(clusterState.version(), clusterState.stateUUID(), clusterState);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* 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.cluster;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.Repeat;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.transport.DummyTransportAddress;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class ClusterStateTests extends ESTestCase {
|
||||
|
||||
public void testSupersedes() {
|
||||
final DiscoveryNode node1 = new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, Version.CURRENT);
|
||||
final DiscoveryNode node2 = new DiscoveryNode("node2", DummyTransportAddress.INSTANCE, Version.CURRENT);
|
||||
final DiscoveryNodes nodes = DiscoveryNodes.builder().put(node1).put(node2).build();
|
||||
ClusterState noMaster1 = ClusterState.builder(ClusterName.DEFAULT).version(randomInt(5)).nodes(nodes).build();
|
||||
ClusterState noMaster2 = ClusterState.builder(ClusterName.DEFAULT).version(randomInt(5)).nodes(nodes).build();
|
||||
ClusterState withMaster1a = ClusterState.builder(ClusterName.DEFAULT).version(randomInt(5)).nodes(DiscoveryNodes.builder(nodes).masterNodeId(node1.id())).build();
|
||||
ClusterState withMaster1b = ClusterState.builder(ClusterName.DEFAULT).version(randomInt(5)).nodes(DiscoveryNodes.builder(nodes).masterNodeId(node1.id())).build();
|
||||
ClusterState withMaster2 = ClusterState.builder(ClusterName.DEFAULT).version(randomInt(5)).nodes(DiscoveryNodes.builder(nodes).masterNodeId(node2.id())).build();
|
||||
|
||||
// states with no master should never supersede anything
|
||||
assertFalse(noMaster1.supersedes(noMaster2));
|
||||
assertFalse(noMaster1.supersedes(withMaster1a));
|
||||
|
||||
// states should never supersede states from another master
|
||||
assertFalse(withMaster1a.supersedes(withMaster2));
|
||||
assertFalse(withMaster1a.supersedes(noMaster1));
|
||||
|
||||
// state from the same master compare by version
|
||||
assertThat(withMaster1a.supersedes(withMaster1b), equalTo(withMaster1a.version() > withMaster1b.version()));
|
||||
|
||||
}
|
||||
}
|
|
@ -21,34 +21,47 @@ package org.elasticsearch.cluster;
|
|||
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
|
||||
import org.elasticsearch.discovery.zen.fd.FaultDetection;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||
import org.elasticsearch.test.disruption.NetworkDelaysPartition;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
|
||||
import static org.elasticsearch.test.ESIntegTestCase.Scope;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.hamcrest.Matchers.empty;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.isOneOf;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
@ClusterScope(scope = Scope.TEST, numDataNodes = 0)
|
||||
@ESIntegTestCase.SuppressLocalMode
|
||||
public class MinimumMasterNodesIT extends ESIntegTestCase {
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
final HashSet<Class<? extends Plugin>> classes = new HashSet<>(super.nodePlugins());
|
||||
classes.add(MockTransportService.TestPlugin.class);
|
||||
return classes;
|
||||
}
|
||||
|
||||
@Test
|
||||
@TestLogging("cluster.service:TRACE,discovery.zen:TRACE,gateway:TRACE,transport.tracer:TRACE")
|
||||
public void simpleMinimumMasterNodes() throws Exception {
|
||||
|
@ -337,4 +350,69 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
|
|||
logger.info("--> verifying no node left and master is up");
|
||||
assertFalse(client().admin().cluster().prepareHealth().setWaitForNodes(Integer.toString(nodeCount)).get().isTimedOut());
|
||||
}
|
||||
|
||||
public void testCanNotPublishWithoutMinMastNodes() throws Exception {
|
||||
Settings settings = settingsBuilder()
|
||||
.put("discovery.type", "zen")
|
||||
.put(FaultDetection.SETTING_PING_TIMEOUT, "1h") // disable it
|
||||
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
|
||||
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, 2)
|
||||
.put(DiscoverySettings.COMMIT_TIMEOUT, "100ms") // speed things up
|
||||
.build();
|
||||
internalCluster().startNodesAsync(3, settings).get();
|
||||
|
||||
final String master = internalCluster().getMasterName();
|
||||
Set<String> otherNodes = new HashSet<>(Arrays.asList(internalCluster().getNodeNames()));
|
||||
otherNodes.remove(master);
|
||||
NetworkDelaysPartition partition = new NetworkDelaysPartition(Collections.singleton(master), otherNodes, 60000, random());
|
||||
internalCluster().setDisruptionScheme(partition);
|
||||
partition.startDisrupting();
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
final AtomicReference<Throwable> failure = new AtomicReference<>();
|
||||
logger.debug("--> submitting for cluster state to be rejected");
|
||||
final ClusterService masterClusterService = internalCluster().clusterService(master);
|
||||
masterClusterService.submitStateUpdateTask("test", new ProcessedClusterStateUpdateTask() {
|
||||
@Override
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
MetaData.Builder metaData = MetaData.builder(currentState.metaData()).persistentSettings(
|
||||
Settings.builder().put(currentState.metaData().persistentSettings()).put("_SHOULD_NOT_BE_THERE_", true).build()
|
||||
);
|
||||
return ClusterState.builder(currentState).metaData(metaData).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Throwable t) {
|
||||
failure.set(t);
|
||||
latch.countDown();
|
||||
}
|
||||
});
|
||||
|
||||
logger.debug("--> waiting for cluster state to be processed/rejected");
|
||||
latch.await();
|
||||
|
||||
assertThat(failure.get(), instanceOf(Discovery.FailedToCommitClusterStateException.class));
|
||||
assertBusy(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
assertThat(masterClusterService.state().nodes().masterNode(), nullValue());
|
||||
}
|
||||
});
|
||||
|
||||
partition.stopDisrupting();
|
||||
|
||||
logger.debug("--> waiting for cluster to heal");
|
||||
assertNoTimeout(client().admin().cluster().prepareHealth().setWaitForNodes("3").setWaitForEvents(Priority.LANGUID));
|
||||
|
||||
for (String node : internalCluster().getNodeNames()) {
|
||||
Settings nodeSetting = internalCluster().clusterService(node).state().metaData().settings();
|
||||
assertThat(node + " processed the cluster state despite of a min master node violation", nodeSetting.get("_SHOULD_NOT_BE_THERE_"), nullValue());
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,15 +49,11 @@ import org.junit.Test;
|
|||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.State;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.*;
|
||||
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
|
||||
import static org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.hamcrest.Matchers.anyOf;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
@ClusterScope(minNumDataNodes = 2)
|
||||
public class AckIT extends ESIntegTestCase {
|
||||
|
@ -75,7 +71,7 @@ public class AckIT extends ESIntegTestCase {
|
|||
createIndex("test");
|
||||
|
||||
assertAcked(client().admin().indices().prepareUpdateSettings("test")
|
||||
.setSettings(Settings.builder().put("refresh_interval", 9999, TimeUnit.MILLISECONDS)));
|
||||
.setSettings(Settings.builder().put("refresh_interval", 9999, TimeUnit.MILLISECONDS)));
|
||||
|
||||
for (Client client : clients()) {
|
||||
String refreshInterval = getLocalClusterState(client).metaData().index("test").settings().get("index.refresh_interval");
|
||||
|
|
|
@ -30,11 +30,7 @@ import org.elasticsearch.action.get.GetResponse;
|
|||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateListener;
|
||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.*;
|
||||
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
||||
import org.elasticsearch.cluster.block.ClusterBlock;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
|
@ -67,16 +63,7 @@ import org.elasticsearch.plugins.Plugin;
|
|||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.InternalTestCluster;
|
||||
import org.elasticsearch.test.discovery.ClusterDiscoveryConfiguration;
|
||||
import org.elasticsearch.test.disruption.BlockClusterStateProcessing;
|
||||
import org.elasticsearch.test.disruption.IntermittentLongGCDisruption;
|
||||
import org.elasticsearch.test.disruption.LongGCDisruption;
|
||||
import org.elasticsearch.test.disruption.NetworkDelaysPartition;
|
||||
import org.elasticsearch.test.disruption.NetworkDisconnectPartition;
|
||||
import org.elasticsearch.test.disruption.NetworkPartition;
|
||||
import org.elasticsearch.test.disruption.NetworkUnresponsivePartition;
|
||||
import org.elasticsearch.test.disruption.ServiceDisruptionScheme;
|
||||
import org.elasticsearch.test.disruption.SingleNodeDisruption;
|
||||
import org.elasticsearch.test.disruption.SlowClusterStateProcessing;
|
||||
import org.elasticsearch.test.disruption.*;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
|
@ -87,22 +74,8 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.*;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
@ -111,10 +84,7 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
|||
import static org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||
import static org.elasticsearch.test.ESIntegTestCase.Scope;
|
||||
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.not;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
@ClusterScope(scope = Scope.TEST, numDataNodes = 0, transportClientRatio = 0)
|
||||
@ESIntegTestCase.SuppressLocalMode
|
||||
|
@ -785,7 +755,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
|
|||
*/
|
||||
@Test
|
||||
public void unicastSinglePingResponseContainsMaster() throws Exception {
|
||||
List<String> nodes = startCluster(4, -1, new int[] {0});
|
||||
List<String> nodes = startCluster(4, -1, new int[]{0});
|
||||
// Figure out what is the elected master node
|
||||
final String masterNode = internalCluster().getMasterName();
|
||||
logger.info("---> legit elected master node=" + masterNode);
|
||||
|
@ -881,7 +851,11 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
|
|||
|
||||
logger.info("blocking cluster state publishing from master [{}] to non master [{}]", masterNode, nonMasterNode);
|
||||
MockTransportService masterTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, masterNode);
|
||||
masterTransportService.addFailToSendNoConnectRule(discoveryNodes.localNode(), PublishClusterStateAction.ACTION_NAME);
|
||||
if (randomBoolean()) {
|
||||
masterTransportService.addFailToSendNoConnectRule(discoveryNodes.localNode(), PublishClusterStateAction.SEND_ACTION_NAME);
|
||||
} else {
|
||||
masterTransportService.addFailToSendNoConnectRule(discoveryNodes.localNode(), PublishClusterStateAction.COMMIT_ACTION_NAME);
|
||||
}
|
||||
|
||||
logger.info("allowing requests from non master [{}] to master [{}], waiting for two join request", nonMasterNode, masterNode);
|
||||
final CountDownLatch countDownLatch = new CountDownLatch(2);
|
||||
|
@ -902,6 +876,10 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
|
|||
nonMasterTransportService.clearRule(discoveryNodes.masterNode());
|
||||
|
||||
ensureStableCluster(2);
|
||||
|
||||
// shutting down the nodes, to avoid the leakage check tripping
|
||||
// on the states associated with the commit requests we may have dropped
|
||||
internalCluster().stopRandomNonMasterNode();
|
||||
}
|
||||
|
||||
|
||||
|
@ -1084,7 +1062,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
|
|||
final Client node1Client = internalCluster().client(node_1);
|
||||
final Client node4Client = internalCluster().client(node_4);
|
||||
logger.info("--> index doc");
|
||||
logLocalClusterStates(node1Client, node2Client, node3Client, node4Client);
|
||||
logLocalClusterStates(node1Client, node2Client, node3Client, node4Client);
|
||||
assertTrue(node3Client.prepareIndex("test", "doc").setSource("{\"text\":\"a\"}").get().isCreated());
|
||||
//sometimes refresh and sometimes flush
|
||||
int refreshOrFlushType = randomIntBetween(1, 2);
|
||||
|
|
|
@ -209,7 +209,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
|
|||
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
final AtomicReference<Exception> reference = new AtomicReference<>();
|
||||
internalCluster().getInstance(TransportService.class, noneMasterNode).sendRequest(node, PublishClusterStateAction.ACTION_NAME, new BytesTransportRequest(bytes, Version.CURRENT), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
|
||||
internalCluster().getInstance(TransportService.class, noneMasterNode).sendRequest(node, PublishClusterStateAction.SEND_ACTION_NAME, new BytesTransportRequest(bytes, Version.CURRENT), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
|
||||
|
||||
@Override
|
||||
public void handleResponse(TransportResponse.Empty response) {
|
||||
|
|
|
@ -19,19 +19,16 @@
|
|||
|
||||
package org.elasticsearch.discovery.zen;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.transport.DummyTransportAddress;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedList;
|
||||
import java.util.Queue;
|
||||
|
||||
import static org.elasticsearch.discovery.zen.ZenDiscovery.ProcessClusterState;
|
||||
import static org.elasticsearch.discovery.zen.ZenDiscovery.shouldIgnoreOrRejectNewClusterState;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
import static org.hamcrest.core.IsNull.nullValue;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -41,9 +38,9 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
|
|||
ClusterName clusterName = new ClusterName("abc");
|
||||
|
||||
DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder();
|
||||
currentNodes.masterNodeId("a");
|
||||
currentNodes.masterNodeId("a").put(new DiscoveryNode("a", DummyTransportAddress.INSTANCE, Version.CURRENT));
|
||||
DiscoveryNodes.Builder newNodes = DiscoveryNodes.builder();
|
||||
newNodes.masterNodeId("a");
|
||||
newNodes.masterNodeId("a").put(new DiscoveryNode("a", DummyTransportAddress.INSTANCE, Version.CURRENT));
|
||||
|
||||
ClusterState.Builder currentState = ClusterState.builder(clusterName);
|
||||
currentState.nodes(currentNodes);
|
||||
|
@ -61,7 +58,8 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
|
|||
assertFalse("should not ignore, because new state's version is higher to current state's version", shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build()));
|
||||
|
||||
currentNodes = DiscoveryNodes.builder();
|
||||
currentNodes.masterNodeId("b");
|
||||
currentNodes.masterNodeId("b").put(new DiscoveryNode("b", DummyTransportAddress.INSTANCE, Version.CURRENT));
|
||||
;
|
||||
// version isn't taken into account, so randomize it to ensure this.
|
||||
if (randomBoolean()) {
|
||||
currentState.version(2);
|
||||
|
@ -91,53 +89,4 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
|
|||
}
|
||||
assertFalse("should not ignore, because current state doesn't have a master", shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build()));
|
||||
}
|
||||
|
||||
public void testSelectNextStateToProcess_empty() {
|
||||
Queue<ProcessClusterState> queue = new LinkedList<>();
|
||||
assertThat(ZenDiscovery.selectNextStateToProcess(queue), nullValue());
|
||||
}
|
||||
|
||||
public void testSelectNextStateToProcess() {
|
||||
ClusterName clusterName = new ClusterName("abc");
|
||||
DiscoveryNodes nodes = DiscoveryNodes.builder().masterNodeId("a").build();
|
||||
|
||||
int numUpdates = scaledRandomIntBetween(50, 100);
|
||||
LinkedList<ProcessClusterState> queue = new LinkedList<>();
|
||||
for (int i = 0; i < numUpdates; i++) {
|
||||
queue.add(new ProcessClusterState(ClusterState.builder(clusterName).version(i).nodes(nodes).build()));
|
||||
}
|
||||
ProcessClusterState mostRecent = queue.get(numUpdates - 1);
|
||||
Collections.shuffle(queue, getRandom());
|
||||
|
||||
assertThat(ZenDiscovery.selectNextStateToProcess(queue), sameInstance(mostRecent.clusterState));
|
||||
assertThat(mostRecent.processed, is(true));
|
||||
assertThat(queue.size(), equalTo(0));
|
||||
}
|
||||
|
||||
public void testSelectNextStateToProcess_differentMasters() {
|
||||
ClusterName clusterName = new ClusterName("abc");
|
||||
DiscoveryNodes nodes1 = DiscoveryNodes.builder().masterNodeId("a").build();
|
||||
DiscoveryNodes nodes2 = DiscoveryNodes.builder().masterNodeId("b").build();
|
||||
|
||||
LinkedList<ProcessClusterState> queue = new LinkedList<>();
|
||||
ProcessClusterState thirdMostRecent = new ProcessClusterState(ClusterState.builder(clusterName).version(1).nodes(nodes1).build());
|
||||
queue.offer(thirdMostRecent);
|
||||
ProcessClusterState secondMostRecent = new ProcessClusterState(ClusterState.builder(clusterName).version(2).nodes(nodes1).build());
|
||||
queue.offer(secondMostRecent);
|
||||
ProcessClusterState mostRecent = new ProcessClusterState(ClusterState.builder(clusterName).version(3).nodes(nodes1).build());
|
||||
queue.offer(mostRecent);
|
||||
Collections.shuffle(queue, getRandom());
|
||||
queue.offer(new ProcessClusterState(ClusterState.builder(clusterName).version(4).nodes(nodes2).build()));
|
||||
queue.offer(new ProcessClusterState(ClusterState.builder(clusterName).version(5).nodes(nodes1).build()));
|
||||
|
||||
|
||||
assertThat(ZenDiscovery.selectNextStateToProcess(queue), sameInstance(mostRecent.clusterState));
|
||||
assertThat(thirdMostRecent.processed, is(true));
|
||||
assertThat(secondMostRecent.processed, is(true));
|
||||
assertThat(mostRecent.processed, is(true));
|
||||
assertThat(queue.size(), equalTo(2));
|
||||
assertThat(queue.get(0).processed, is(false));
|
||||
assertThat(queue.get(1).processed, is(false));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,224 @@
|
|||
/*
|
||||
* 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.discovery.zen.publish;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.transport.DummyTransportAddress;
|
||||
import org.elasticsearch.discovery.zen.publish.PendingClusterStatesQueue.ClusterStateContext;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
public class PendingClusterStatesQueueTests extends ESTestCase {
|
||||
|
||||
public void testSelectNextStateToProcess_empty() {
|
||||
PendingClusterStatesQueue queue = new PendingClusterStatesQueue(logger, randomIntBetween(1, 200));
|
||||
assertThat(queue.getNextClusterStateToProcess(), nullValue());
|
||||
}
|
||||
|
||||
public void testDroppingStatesAtCapacity() {
|
||||
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 300), "master1", "master2", "master3", "master4");
|
||||
Collections.shuffle(states, random());
|
||||
// insert half of the states
|
||||
final int numberOfStateToDrop = states.size() / 2;
|
||||
List<ClusterState> stateToDrop = states.subList(0, numberOfStateToDrop);
|
||||
final int queueSize = states.size() - numberOfStateToDrop;
|
||||
PendingClusterStatesQueue queue = createQueueWithStates(stateToDrop, queueSize);
|
||||
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
|
||||
for (ClusterState state : states.subList(numberOfStateToDrop, states.size())) {
|
||||
queue.addPending(state);
|
||||
}
|
||||
|
||||
assertThat(queue.pendingClusterStates().length, equalTo(queueSize));
|
||||
// check all committed states got a failure due to the drop
|
||||
for (ClusterStateContext context : committedContexts) {
|
||||
assertThat(((MockListener) context.listener).failure, notNullValue());
|
||||
}
|
||||
|
||||
// all states that should have dropped are indeed dropped.
|
||||
for (ClusterState state : stateToDrop) {
|
||||
assertThat(queue.findState(state.stateUUID()), nullValue());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testSimpleQueueSameMaster() {
|
||||
final int numUpdates = scaledRandomIntBetween(50, 100);
|
||||
List<ClusterState> states = randomStates(numUpdates, "master");
|
||||
Collections.shuffle(states, random());
|
||||
PendingClusterStatesQueue queue;
|
||||
queue = createQueueWithStates(states);
|
||||
|
||||
// no state is committed yet
|
||||
assertThat(queue.getNextClusterStateToProcess(), nullValue());
|
||||
|
||||
ClusterState highestCommitted = null;
|
||||
for (ClusterStateContext context : randomCommitStates(queue)) {
|
||||
if (highestCommitted == null || context.state.supersedes(highestCommitted)) {
|
||||
highestCommitted = context.state;
|
||||
}
|
||||
}
|
||||
|
||||
assertThat(queue.getNextClusterStateToProcess(), sameInstance(highestCommitted));
|
||||
|
||||
queue.markAsProcessed(highestCommitted);
|
||||
|
||||
// now there is nothing more to process
|
||||
assertThat(queue.getNextClusterStateToProcess(), nullValue());
|
||||
}
|
||||
|
||||
public void testProcessedStateCleansStatesFromOtherMasters() {
|
||||
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 300), "master1", "master2", "master3", "master4");
|
||||
PendingClusterStatesQueue queue = createQueueWithStates(states);
|
||||
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
|
||||
ClusterState randomCommitted = randomFrom(committedContexts).state;
|
||||
queue.markAsProcessed(randomCommitted);
|
||||
final String processedMaster = randomCommitted.nodes().masterNodeId();
|
||||
|
||||
// now check that queue doesn't contain anything pending from another master
|
||||
for (ClusterStateContext context : queue.pendingStates) {
|
||||
final String pendingMaster = context.state.nodes().masterNodeId();
|
||||
assertThat("found a cluster state from [" + pendingMaster
|
||||
+ "], after a state from [" + processedMaster + "] was proccessed",
|
||||
pendingMaster, equalTo(processedMaster));
|
||||
}
|
||||
// and check all committed contexts from another master were failed
|
||||
for (ClusterStateContext context : committedContexts) {
|
||||
if (context.state.nodes().masterNodeId().equals(processedMaster) == false) {
|
||||
assertThat(((MockListener) context.listener).failure, notNullValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testFailedStateCleansSupersededStatesOnly() {
|
||||
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4");
|
||||
PendingClusterStatesQueue queue = createQueueWithStates(states);
|
||||
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
|
||||
ClusterState toFail = randomFrom(committedContexts).state;
|
||||
queue.markAsFailed(toFail, new ElasticsearchException("boo!"));
|
||||
final Map<String, ClusterStateContext> committedContextsById = new HashMap<>();
|
||||
for (ClusterStateContext context : committedContexts) {
|
||||
committedContextsById.put(context.stateUUID(), context);
|
||||
}
|
||||
|
||||
// now check that queue doesn't contain superseded states
|
||||
for (ClusterStateContext context : queue.pendingStates) {
|
||||
if (context.committed()) {
|
||||
assertFalse("found a committed cluster state, which is superseded by a failed state.\nFound:" + context.state + "\nfailed:" + toFail,
|
||||
toFail.supersedes(context.state));
|
||||
}
|
||||
}
|
||||
// check no state has been erroneously removed
|
||||
for (ClusterState state : states) {
|
||||
ClusterStateContext pendingContext = queue.findState(state.stateUUID());
|
||||
if (pendingContext != null) {
|
||||
continue;
|
||||
}
|
||||
if (state.equals(toFail)) {
|
||||
continue;
|
||||
}
|
||||
assertThat("non-committed states should never be removed", committedContextsById, hasKey(state.stateUUID()));
|
||||
final ClusterStateContext context = committedContextsById.get(state.stateUUID());
|
||||
assertThat("removed state is not superseded by failed state. \nRemoved state:" + context + "\nfailed: " + toFail,
|
||||
toFail.supersedes(context.state), equalTo(true));
|
||||
assertThat("removed state was failed with wrong exception", ((MockListener) context.listener).failure, notNullValue());
|
||||
assertThat("removed state was failed with wrong exception", ((MockListener) context.listener).failure.getMessage(), containsString("boo"));
|
||||
}
|
||||
}
|
||||
|
||||
public void testFailAllAndClear() {
|
||||
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4");
|
||||
PendingClusterStatesQueue queue = createQueueWithStates(states);
|
||||
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
|
||||
queue.failAllStatesAndClear(new ElasticsearchException("boo!"));
|
||||
assertThat(queue.pendingStates, empty());
|
||||
assertThat(queue.getNextClusterStateToProcess(), nullValue());
|
||||
for (ClusterStateContext context : committedContexts) {
|
||||
assertThat("state was failed with wrong exception", ((MockListener) context.listener).failure, notNullValue());
|
||||
assertThat("state was failed with wrong exception", ((MockListener) context.listener).failure.getMessage(), containsString("boo"));
|
||||
}
|
||||
}
|
||||
|
||||
protected List<ClusterStateContext> randomCommitStates(PendingClusterStatesQueue queue) {
|
||||
List<ClusterStateContext> committedContexts = new ArrayList<>();
|
||||
for (int iter = randomInt(queue.pendingStates.size() - 1); iter >= 0; iter--) {
|
||||
ClusterState state = queue.markAsCommitted(randomFrom(queue.pendingStates).stateUUID(), new MockListener());
|
||||
if (state != null) {
|
||||
// null cluster state means we committed twice
|
||||
committedContexts.add(queue.findState(state.stateUUID()));
|
||||
}
|
||||
}
|
||||
return committedContexts;
|
||||
}
|
||||
|
||||
PendingClusterStatesQueue createQueueWithStates(List<ClusterState> states) {
|
||||
return createQueueWithStates(states, states.size() * 2); // we don't care about limits (there are dedicated tests for that)
|
||||
}
|
||||
|
||||
PendingClusterStatesQueue createQueueWithStates(List<ClusterState> states, int maxQueueSize) {
|
||||
PendingClusterStatesQueue queue;
|
||||
queue = new PendingClusterStatesQueue(logger, maxQueueSize);
|
||||
for (ClusterState state : states) {
|
||||
queue.addPending(state);
|
||||
}
|
||||
return queue;
|
||||
}
|
||||
|
||||
List<ClusterState> randomStates(int count, String... masters) {
|
||||
ArrayList<ClusterState> states = new ArrayList<>(count);
|
||||
ClusterState[] lastClusterStatePerMaster = new ClusterState[masters.length];
|
||||
for (; count > 0; count--) {
|
||||
int masterIndex = randomInt(masters.length - 1);
|
||||
ClusterState state = lastClusterStatePerMaster[masterIndex];
|
||||
if (state == null) {
|
||||
state = ClusterState.builder(ClusterName.DEFAULT).nodes(DiscoveryNodes.builder()
|
||||
.put(new DiscoveryNode(masters[masterIndex], DummyTransportAddress.INSTANCE, Version.CURRENT)).masterNodeId(masters[masterIndex]).build()
|
||||
).build();
|
||||
} else {
|
||||
state = ClusterState.builder(state).incrementVersion().build();
|
||||
}
|
||||
states.add(state);
|
||||
lastClusterStatePerMaster[masterIndex] = state;
|
||||
}
|
||||
return states;
|
||||
}
|
||||
|
||||
static class MockListener implements PendingClusterStatesQueue.StateProcessedListener {
|
||||
volatile boolean processed;
|
||||
volatile Throwable failure;
|
||||
|
||||
@Override
|
||||
public void onNewClusterStateProcessed() {
|
||||
processed = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNewClusterStateFailed(Throwable t) {
|
||||
failure = t;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,879 @@
|
|||
/*
|
||||
* 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.discovery.zen.publish;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.*;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.zen.DiscoveryNodesProvider;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
import org.elasticsearch.node.settings.NodeSettingsService;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.*;
|
||||
import org.elasticsearch.transport.local.LocalTransport;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
@TestLogging("discovery.zen.publish:TRACE")
|
||||
public class PublishClusterStateActionTests extends ESTestCase {
|
||||
|
||||
protected ThreadPool threadPool;
|
||||
protected Map<String, MockNode> nodes = new HashMap<>();
|
||||
|
||||
public static class MockNode implements PublishClusterStateAction.NewPendingClusterStateListener, DiscoveryNodesProvider {
|
||||
public final DiscoveryNode discoveryNode;
|
||||
public final MockTransportService service;
|
||||
public MockPublishAction action;
|
||||
public final ClusterStateListener listener;
|
||||
|
||||
public volatile ClusterState clusterState;
|
||||
|
||||
private final ESLogger logger;
|
||||
|
||||
public MockNode(DiscoveryNode discoveryNode, MockTransportService service, @Nullable ClusterStateListener listener, ESLogger logger) {
|
||||
this.discoveryNode = discoveryNode;
|
||||
this.service = service;
|
||||
this.listener = listener;
|
||||
this.logger = logger;
|
||||
this.clusterState = ClusterState.builder(ClusterName.DEFAULT).nodes(DiscoveryNodes.builder().put(discoveryNode).localNodeId(discoveryNode.id()).build()).build();
|
||||
}
|
||||
|
||||
public MockNode setAsMaster() {
|
||||
this.clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).masterNodeId(discoveryNode.id())).build();
|
||||
return this;
|
||||
}
|
||||
|
||||
public MockNode resetMasterId() {
|
||||
this.clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).masterNodeId(null)).build();
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
public void connectTo(DiscoveryNode node) {
|
||||
service.connectToNode(node);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNewClusterState(String reason) {
|
||||
ClusterState newClusterState = action.pendingStatesQueue().getNextClusterStateToProcess();
|
||||
logger.debug("[{}] received version [{}], uuid [{}]", discoveryNode.name(), newClusterState.version(), newClusterState.stateUUID());
|
||||
if (listener != null) {
|
||||
ClusterChangedEvent event = new ClusterChangedEvent("", newClusterState, clusterState);
|
||||
listener.clusterChanged(event);
|
||||
}
|
||||
if (clusterState.nodes().masterNode() == null || newClusterState.supersedes(clusterState)) {
|
||||
clusterState = newClusterState;
|
||||
}
|
||||
action.pendingStatesQueue().markAsProcessed(newClusterState);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DiscoveryNodes nodes() {
|
||||
return clusterState.nodes();
|
||||
}
|
||||
|
||||
@Override
|
||||
public NodeService nodeService() {
|
||||
assert false;
|
||||
throw new UnsupportedOperationException("Shouldn't be here");
|
||||
}
|
||||
}
|
||||
|
||||
public MockNode createMockNode(final String name) throws Exception {
|
||||
return createMockNode(name, Settings.EMPTY, Version.CURRENT);
|
||||
}
|
||||
|
||||
public MockNode createMockNode(String name, Settings settings) throws Exception {
|
||||
return createMockNode(name, settings, Version.CURRENT);
|
||||
}
|
||||
|
||||
public MockNode createMockNode(final String name, Settings settings, Version version) throws Exception {
|
||||
return createMockNode(name, settings, version, null);
|
||||
}
|
||||
|
||||
public MockNode createMockNode(String name, Settings settings, Version version, @Nullable ClusterStateListener listener) throws Exception {
|
||||
settings = Settings.builder()
|
||||
.put("name", name)
|
||||
.put(TransportService.SETTING_TRACE_LOG_INCLUDE, "", TransportService.SETTING_TRACE_LOG_EXCLUDE, "NOTHING")
|
||||
.put(settings)
|
||||
.build();
|
||||
|
||||
MockTransportService service = buildTransportService(settings, version);
|
||||
DiscoveryNode discoveryNode = new DiscoveryNode(name, name, service.boundAddress().publishAddress(),
|
||||
settings.getByPrefix("node.").getAsMap(), version);
|
||||
MockNode node = new MockNode(discoveryNode, service, listener, logger);
|
||||
node.action = buildPublishClusterStateAction(settings, service, node, node);
|
||||
final CountDownLatch latch = new CountDownLatch(nodes.size() * 2 + 1);
|
||||
TransportConnectionListener waitForConnection = new TransportConnectionListener() {
|
||||
@Override
|
||||
public void onNodeConnected(DiscoveryNode node) {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node) {
|
||||
fail("disconnect should not be called " + node);
|
||||
}
|
||||
};
|
||||
node.service.addConnectionListener(waitForConnection);
|
||||
for (MockNode curNode : nodes.values()) {
|
||||
curNode.service.addConnectionListener(waitForConnection);
|
||||
curNode.connectTo(node.discoveryNode);
|
||||
node.connectTo(curNode.discoveryNode);
|
||||
}
|
||||
node.connectTo(node.discoveryNode);
|
||||
assertThat("failed to wait for all nodes to connect", latch.await(5, TimeUnit.SECONDS), equalTo(true));
|
||||
for (MockNode curNode : nodes.values()) {
|
||||
curNode.service.removeConnectionListener(waitForConnection);
|
||||
}
|
||||
node.service.removeConnectionListener(waitForConnection);
|
||||
if (nodes.put(name, node) != null) {
|
||||
fail("Node with the name " + name + " already exist");
|
||||
}
|
||||
return node;
|
||||
}
|
||||
|
||||
public MockTransportService service(String name) {
|
||||
MockNode node = nodes.get(name);
|
||||
if (node != null) {
|
||||
return node.service;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public PublishClusterStateAction action(String name) {
|
||||
MockNode node = nodes.get(name);
|
||||
if (node != null) {
|
||||
return node.action;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
threadPool = new ThreadPool(getClass().getName());
|
||||
}
|
||||
|
||||
@Override
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
super.tearDown();
|
||||
for (MockNode curNode : nodes.values()) {
|
||||
curNode.action.close();
|
||||
curNode.service.close();
|
||||
}
|
||||
terminate(threadPool);
|
||||
}
|
||||
|
||||
protected MockTransportService buildTransportService(Settings settings, Version version) {
|
||||
MockTransportService transportService = new MockTransportService(settings, new LocalTransport(settings, threadPool, version, new NamedWriteableRegistry()), threadPool);
|
||||
transportService.start();
|
||||
return transportService;
|
||||
}
|
||||
|
||||
protected MockPublishAction buildPublishClusterStateAction(Settings settings, MockTransportService transportService, DiscoveryNodesProvider nodesProvider,
|
||||
PublishClusterStateAction.NewPendingClusterStateListener listener) {
|
||||
DiscoverySettings discoverySettings = new DiscoverySettings(settings, new NodeSettingsService(settings));
|
||||
return new MockPublishAction(settings, transportService, nodesProvider, listener, discoverySettings, ClusterName.DEFAULT);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleClusterStatePublishing() throws Exception {
|
||||
MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, Version.CURRENT).setAsMaster();
|
||||
MockNode nodeB = createMockNode("nodeB", Settings.EMPTY, Version.CURRENT);
|
||||
|
||||
// Initial cluster state
|
||||
ClusterState clusterState = nodeA.clusterState;
|
||||
|
||||
// cluster state update - add nodeB
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(clusterState.nodes()).put(nodeB.discoveryNode).build();
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromFull(nodeB.clusterState, clusterState);
|
||||
|
||||
// cluster state update - add block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromDiff(nodeB.clusterState, clusterState);
|
||||
assertThat(nodeB.clusterState.blocks().global().size(), equalTo(1));
|
||||
|
||||
// cluster state update - remove block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromDiff(nodeB.clusterState, clusterState);
|
||||
assertTrue(nodeB.clusterState.wasReadFromDiff());
|
||||
|
||||
// Adding new node - this node should get full cluster state while nodeB should still be getting diffs
|
||||
|
||||
MockNode nodeC = createMockNode("nodeC", Settings.EMPTY, Version.CURRENT);
|
||||
|
||||
// cluster state update 3 - register node C
|
||||
previousClusterState = clusterState;
|
||||
discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeC.discoveryNode).build();
|
||||
clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromDiff(nodeB.clusterState, clusterState);
|
||||
// First state
|
||||
assertSameStateFromFull(nodeC.clusterState, clusterState);
|
||||
|
||||
// cluster state update 4 - update settings
|
||||
previousClusterState = clusterState;
|
||||
MetaData metaData = MetaData.builder(clusterState.metaData()).transientSettings(Settings.settingsBuilder().put("foo", "bar").build()).build();
|
||||
clusterState = ClusterState.builder(clusterState).metaData(metaData).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromDiff(nodeB.clusterState, clusterState);
|
||||
assertThat(nodeB.clusterState.blocks().global().size(), equalTo(0));
|
||||
assertSameStateFromDiff(nodeC.clusterState, clusterState);
|
||||
assertThat(nodeC.clusterState.blocks().global().size(), equalTo(0));
|
||||
|
||||
// cluster state update - skipping one version change - should request full cluster state
|
||||
previousClusterState = ClusterState.builder(clusterState).incrementVersion().build();
|
||||
clusterState = ClusterState.builder(clusterState).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromFull(nodeB.clusterState, clusterState);
|
||||
assertSameStateFromFull(nodeC.clusterState, clusterState);
|
||||
assertFalse(nodeC.clusterState.wasReadFromDiff());
|
||||
|
||||
// node A steps down from being master
|
||||
nodeA.resetMasterId();
|
||||
nodeB.resetMasterId();
|
||||
nodeC.resetMasterId();
|
||||
|
||||
// node B becomes the master and sends a version of the cluster state that goes back
|
||||
discoveryNodes = DiscoveryNodes.builder(discoveryNodes)
|
||||
.put(nodeA.discoveryNode)
|
||||
.put(nodeB.discoveryNode)
|
||||
.put(nodeC.discoveryNode)
|
||||
.masterNodeId(nodeB.discoveryNode.id())
|
||||
.localNodeId(nodeB.discoveryNode.id())
|
||||
.build();
|
||||
previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build();
|
||||
clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build();
|
||||
publishStateAndWait(nodeB.action, clusterState, previousClusterState);
|
||||
assertSameStateFromFull(nodeA.clusterState, clusterState);
|
||||
assertSameStateFromFull(nodeC.clusterState, clusterState);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnexpectedDiffPublishing() throws Exception {
|
||||
|
||||
MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, Version.CURRENT, new ClusterStateListener() {
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
fail("Shouldn't send cluster state to myself");
|
||||
}
|
||||
}).setAsMaster();
|
||||
|
||||
MockNode nodeB = createMockNode("nodeB", Settings.EMPTY, Version.CURRENT);
|
||||
|
||||
// Initial cluster state with both states - the second node still shouldn't get diff even though it's present in the previous cluster state
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(nodeA.nodes()).put(nodeB.discoveryNode).build();
|
||||
ClusterState previousClusterState = ClusterState.builder(ClusterName.DEFAULT).nodes(discoveryNodes).build();
|
||||
ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromFull(nodeB.clusterState, clusterState);
|
||||
|
||||
// cluster state update - add block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromDiff(nodeB.clusterState, clusterState);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDisablingDiffPublishing() throws Exception {
|
||||
Settings noDiffPublishingSettings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE, false).build();
|
||||
|
||||
MockNode nodeA = createMockNode("nodeA", noDiffPublishingSettings, Version.CURRENT, new ClusterStateListener() {
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
fail("Shouldn't send cluster state to myself");
|
||||
}
|
||||
});
|
||||
|
||||
MockNode nodeB = createMockNode("nodeB", noDiffPublishingSettings, Version.CURRENT, new ClusterStateListener() {
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
assertFalse(event.state().wasReadFromDiff());
|
||||
}
|
||||
});
|
||||
|
||||
// Initial cluster state
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).localNodeId(nodeA.discoveryNode.id()).masterNodeId(nodeA.discoveryNode.id()).build();
|
||||
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).nodes(discoveryNodes).build();
|
||||
|
||||
// cluster state update - add nodeB
|
||||
discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeB.discoveryNode).build();
|
||||
ClusterState previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
|
||||
// cluster state update - add block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Test not waiting on publishing works correctly (i.e., publishing times out)
|
||||
*/
|
||||
@Test
|
||||
public void testSimultaneousClusterStatePublishing() throws Exception {
|
||||
int numberOfNodes = randomIntBetween(2, 10);
|
||||
int numberOfIterations = scaledRandomIntBetween(5, 50);
|
||||
Settings settings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE, randomBoolean()).build();
|
||||
MockNode master = createMockNode("node0", settings, Version.CURRENT, new ClusterStateListener() {
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
assertProperMetaDataForVersion(event.state().metaData(), event.state().version());
|
||||
}
|
||||
}).setAsMaster();
|
||||
DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder(master.nodes());
|
||||
for (int i = 1; i < numberOfNodes; i++) {
|
||||
final String name = "node" + i;
|
||||
final MockNode node = createMockNode(name, settings, Version.CURRENT, new ClusterStateListener() {
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
assertProperMetaDataForVersion(event.state().metaData(), event.state().version());
|
||||
}
|
||||
});
|
||||
discoveryNodesBuilder.put(node.discoveryNode);
|
||||
}
|
||||
|
||||
AssertingAckListener[] listeners = new AssertingAckListener[numberOfIterations];
|
||||
DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build();
|
||||
MetaData metaData = MetaData.EMPTY_META_DATA;
|
||||
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).metaData(metaData).build();
|
||||
ClusterState previousState;
|
||||
for (int i = 0; i < numberOfIterations; i++) {
|
||||
previousState = clusterState;
|
||||
metaData = buildMetaDataForVersion(metaData, i + 1);
|
||||
clusterState = ClusterState.builder(clusterState).incrementVersion().metaData(metaData).nodes(discoveryNodes).build();
|
||||
listeners[i] = publishState(master.action, clusterState, previousState);
|
||||
}
|
||||
|
||||
for (int i = 0; i < numberOfIterations; i++) {
|
||||
listeners[i].await(1, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
// set the master cs
|
||||
master.clusterState = clusterState;
|
||||
|
||||
for (MockNode node : nodes.values()) {
|
||||
assertSameState(node.clusterState, clusterState);
|
||||
assertThat(node.clusterState.nodes().localNode(), equalTo(node.discoveryNode));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerializationFailureDuringDiffPublishing() throws Exception {
|
||||
|
||||
MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, Version.CURRENT, new ClusterStateListener() {
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
fail("Shouldn't send cluster state to myself");
|
||||
}
|
||||
}).setAsMaster();
|
||||
|
||||
MockNode nodeB = createMockNode("nodeB", Settings.EMPTY, Version.CURRENT);
|
||||
|
||||
// Initial cluster state with both states - the second node still shouldn't get diff even though it's present in the previous cluster state
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(nodeA.nodes()).put(nodeB.discoveryNode).build();
|
||||
ClusterState previousClusterState = ClusterState.builder(ClusterName.DEFAULT).nodes(discoveryNodes).build();
|
||||
ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build();
|
||||
publishStateAndWait(nodeA.action, clusterState, previousClusterState);
|
||||
assertSameStateFromFull(nodeB.clusterState, clusterState);
|
||||
|
||||
// cluster state update - add block
|
||||
previousClusterState = clusterState;
|
||||
clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build();
|
||||
|
||||
ClusterState unserializableClusterState = new ClusterState(clusterState.version(), clusterState.stateUUID(), clusterState) {
|
||||
@Override
|
||||
public Diff<ClusterState> diff(ClusterState previousState) {
|
||||
return new Diff<ClusterState>() {
|
||||
@Override
|
||||
public ClusterState apply(ClusterState part) {
|
||||
fail("this diff shouldn't be applied");
|
||||
return part;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
throw new IOException("Simulated failure of diff serialization");
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
try {
|
||||
publishStateAndWait(nodeA.action, unserializableClusterState, previousClusterState);
|
||||
fail("cluster state published despite of diff errors");
|
||||
} catch (Discovery.FailedToCommitClusterStateException e) {
|
||||
assertThat(e.getCause(), notNullValue());
|
||||
assertThat(e.getCause().getMessage(), containsString("failed to serialize"));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void testFailToPublishWithLessThanMinMasterNodes() throws Exception {
|
||||
final int masterNodes = randomIntBetween(1, 10);
|
||||
|
||||
MockNode master = createMockNode("master");
|
||||
DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder().put(master.discoveryNode);
|
||||
for (int i = 1; i < masterNodes; i++) {
|
||||
discoveryNodesBuilder.put(createMockNode("node" + i).discoveryNode);
|
||||
}
|
||||
final int dataNodes = randomIntBetween(0, 5);
|
||||
final Settings dataSettings = Settings.builder().put("node.master", false).build();
|
||||
for (int i = 0; i < dataNodes; i++) {
|
||||
discoveryNodesBuilder.put(createMockNode("data_" + i, dataSettings).discoveryNode);
|
||||
}
|
||||
discoveryNodesBuilder.localNodeId(master.discoveryNode.id()).masterNodeId(master.discoveryNode.id());
|
||||
DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build();
|
||||
MetaData metaData = MetaData.EMPTY_META_DATA;
|
||||
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).metaData(metaData).nodes(discoveryNodes).build();
|
||||
ClusterState previousState = master.clusterState;
|
||||
try {
|
||||
publishState(master.action, clusterState, previousState, masterNodes + randomIntBetween(1, 5));
|
||||
fail("cluster state publishing didn't fail despite of not having enough nodes");
|
||||
} catch (Discovery.FailedToCommitClusterStateException expected) {
|
||||
logger.debug("failed to publish as expected", expected);
|
||||
}
|
||||
}
|
||||
|
||||
public void testPublishingWithSendingErrors() throws Exception {
|
||||
int goodNodes = randomIntBetween(2, 5);
|
||||
int errorNodes = randomIntBetween(1, 5);
|
||||
int timeOutNodes = randomBoolean() ? 0 : randomIntBetween(1, 5); // adding timeout nodes will force timeout errors
|
||||
final int numberOfMasterNodes = goodNodes + errorNodes + timeOutNodes + 1; // master
|
||||
final boolean expectingToCommit = randomBoolean();
|
||||
Settings.Builder settings = Settings.builder();
|
||||
// make sure we have a reasonable timeout if we expect to timeout, o.w. one that will make the test "hang"
|
||||
settings.put(DiscoverySettings.COMMIT_TIMEOUT, expectingToCommit == false && timeOutNodes > 0 ? "100ms" : "1h")
|
||||
.put(DiscoverySettings.PUBLISH_TIMEOUT, "5ms"); // test is about committing
|
||||
|
||||
MockNode master = createMockNode("master", settings.build());
|
||||
|
||||
// randomize things a bit
|
||||
int[] nodeTypes = new int[goodNodes + errorNodes + timeOutNodes];
|
||||
for (int i = 0; i < goodNodes; i++) {
|
||||
nodeTypes[i] = 0;
|
||||
}
|
||||
for (int i = goodNodes; i < goodNodes + errorNodes; i++) {
|
||||
nodeTypes[i] = 1;
|
||||
}
|
||||
for (int i = goodNodes + errorNodes; i < nodeTypes.length; i++) {
|
||||
nodeTypes[i] = 2;
|
||||
}
|
||||
Collections.shuffle(Arrays.asList(nodeTypes), random());
|
||||
|
||||
DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder().put(master.discoveryNode);
|
||||
for (int i = 0; i < nodeTypes.length; i++) {
|
||||
final MockNode mockNode = createMockNode("node" + i);
|
||||
discoveryNodesBuilder.put(mockNode.discoveryNode);
|
||||
switch (nodeTypes[i]) {
|
||||
case 1:
|
||||
mockNode.action.errorOnSend.set(true);
|
||||
break;
|
||||
case 2:
|
||||
mockNode.action.timeoutOnSend.set(true);
|
||||
break;
|
||||
}
|
||||
}
|
||||
final int dataNodes = randomIntBetween(0, 3); // data nodes don't matter
|
||||
for (int i = 0; i < dataNodes; i++) {
|
||||
final MockNode mockNode = createMockNode("data_" + i, Settings.builder().put("node.master", false).build());
|
||||
discoveryNodesBuilder.put(mockNode.discoveryNode);
|
||||
if (randomBoolean()) {
|
||||
// we really don't care - just chaos monkey
|
||||
mockNode.action.errorOnCommit.set(randomBoolean());
|
||||
mockNode.action.errorOnSend.set(randomBoolean());
|
||||
mockNode.action.timeoutOnCommit.set(randomBoolean());
|
||||
mockNode.action.timeoutOnSend.set(randomBoolean());
|
||||
}
|
||||
}
|
||||
|
||||
final int minMasterNodes;
|
||||
final String expectedBehavior;
|
||||
if (expectingToCommit) {
|
||||
minMasterNodes = randomIntBetween(0, goodNodes + 1); // count master
|
||||
expectedBehavior = "succeed";
|
||||
} else {
|
||||
minMasterNodes = randomIntBetween(goodNodes + 2, numberOfMasterNodes); // +2 because of master
|
||||
expectedBehavior = timeOutNodes > 0 ? "timeout" : "fail";
|
||||
}
|
||||
logger.info("--> expecting commit to {}. good nodes [{}], errors [{}], timeouts [{}]. min_master_nodes [{}]",
|
||||
expectedBehavior, goodNodes + 1, errorNodes, timeOutNodes, minMasterNodes);
|
||||
|
||||
discoveryNodesBuilder.localNodeId(master.discoveryNode.id()).masterNodeId(master.discoveryNode.id());
|
||||
DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build();
|
||||
MetaData metaData = MetaData.EMPTY_META_DATA;
|
||||
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).metaData(metaData).nodes(discoveryNodes).build();
|
||||
ClusterState previousState = master.clusterState;
|
||||
try {
|
||||
publishState(master.action, clusterState, previousState, minMasterNodes);
|
||||
if (expectingToCommit == false) {
|
||||
fail("cluster state publishing didn't fail despite of not have enough nodes");
|
||||
}
|
||||
} catch (Discovery.FailedToCommitClusterStateException exception) {
|
||||
logger.debug("failed to publish as expected", exception);
|
||||
if (expectingToCommit) {
|
||||
throw exception;
|
||||
}
|
||||
assertThat(exception.getMessage(), containsString(timeOutNodes > 0 ? "timed out" : "failed"));
|
||||
}
|
||||
}
|
||||
|
||||
public void testIncomingClusterStateValidation() throws Exception {
|
||||
MockNode node = createMockNode("node");
|
||||
|
||||
logger.info("--> testing acceptances of any master when having no master");
|
||||
ClusterState state = ClusterState.builder(node.clusterState)
|
||||
.nodes(DiscoveryNodes.builder(node.nodes()).masterNodeId(randomAsciiOfLength(10))).incrementVersion().build();
|
||||
node.action.validateIncomingState(state, null);
|
||||
|
||||
// now set a master node
|
||||
node.clusterState = ClusterState.builder(node.clusterState).nodes(DiscoveryNodes.builder(node.nodes()).masterNodeId("master")).build();
|
||||
logger.info("--> testing rejection of another master");
|
||||
try {
|
||||
node.action.validateIncomingState(state, node.clusterState);
|
||||
fail("node accepted state from another master");
|
||||
} catch (IllegalStateException OK) {
|
||||
}
|
||||
|
||||
logger.info("--> test state from the current master is accepted");
|
||||
node.action.validateIncomingState(ClusterState.builder(node.clusterState)
|
||||
.nodes(DiscoveryNodes.builder(node.nodes()).masterNodeId("master")).build(), node.clusterState);
|
||||
|
||||
|
||||
logger.info("--> testing rejection of another cluster name");
|
||||
try {
|
||||
node.action.validateIncomingState(ClusterState.builder(new ClusterName(randomAsciiOfLength(10))).nodes(node.nodes()).build(), node.clusterState);
|
||||
fail("node accepted state with another cluster name");
|
||||
} catch (IllegalStateException OK) {
|
||||
}
|
||||
|
||||
logger.info("--> testing rejection of a cluster state with wrong local node");
|
||||
try {
|
||||
state = ClusterState.builder(node.clusterState)
|
||||
.nodes(DiscoveryNodes.builder(node.nodes()).localNodeId("_non_existing_").build())
|
||||
.incrementVersion().build();
|
||||
node.action.validateIncomingState(state, node.clusterState);
|
||||
fail("node accepted state with non-existence local node");
|
||||
} catch (IllegalStateException OK) {
|
||||
}
|
||||
|
||||
try {
|
||||
MockNode otherNode = createMockNode("otherNode");
|
||||
state = ClusterState.builder(node.clusterState).nodes(
|
||||
DiscoveryNodes.builder(node.nodes()).put(otherNode.discoveryNode).localNodeId(otherNode.discoveryNode.id()).build()
|
||||
).incrementVersion().build();
|
||||
node.action.validateIncomingState(state, node.clusterState);
|
||||
fail("node accepted state with existent but wrong local node");
|
||||
} catch (IllegalStateException OK) {
|
||||
}
|
||||
|
||||
logger.info("--> testing acceptance of an old cluster state");
|
||||
state = node.clusterState;
|
||||
node.clusterState = ClusterState.builder(node.clusterState).incrementVersion().build();
|
||||
node.action.validateIncomingState(state, node.clusterState);
|
||||
|
||||
// an older version from a *new* master is also OK!
|
||||
ClusterState previousState = ClusterState.builder(node.clusterState).incrementVersion().build();
|
||||
state = ClusterState.builder(node.clusterState)
|
||||
.nodes(DiscoveryNodes.builder(node.clusterState.nodes()).masterNodeId("_new_master_").build())
|
||||
.build();
|
||||
// remove the master of the node (but still have a previous cluster state with it)!
|
||||
node.resetMasterId();
|
||||
|
||||
node.action.validateIncomingState(state, previousState);
|
||||
}
|
||||
|
||||
public void testInterleavedPublishCommit() throws Throwable {
|
||||
MockNode node = createMockNode("node").setAsMaster();
|
||||
final CapturingTransportChannel channel = new CapturingTransportChannel();
|
||||
|
||||
List<ClusterState> states = new ArrayList<>();
|
||||
final int numOfStates = scaledRandomIntBetween(3, 10);
|
||||
for (int i = 1; i <= numOfStates; i++) {
|
||||
states.add(ClusterState.builder(node.clusterState).version(i).stateUUID(ClusterState.UNKNOWN_UUID).build());
|
||||
}
|
||||
|
||||
final ClusterState finalState = states.get(numOfStates - 1);
|
||||
Collections.shuffle(states, random());
|
||||
|
||||
logger.info("--> publishing states");
|
||||
for (ClusterState state : states) {
|
||||
node.action.handleIncomingClusterStateRequest(
|
||||
new BytesTransportRequest(PublishClusterStateAction.serializeFullClusterState(state, Version.CURRENT), Version.CURRENT),
|
||||
channel);
|
||||
assertThat(channel.response.get(), equalTo((TransportResponse) TransportResponse.Empty.INSTANCE));
|
||||
assertThat(channel.error.get(), nullValue());
|
||||
channel.clear();
|
||||
}
|
||||
|
||||
logger.info("--> committing states");
|
||||
|
||||
Collections.shuffle(states, random());
|
||||
for (ClusterState state : states) {
|
||||
node.action.handleCommitRequest(new PublishClusterStateAction.CommitClusterStateRequest(state.stateUUID()), channel);
|
||||
assertThat(channel.response.get(), equalTo((TransportResponse) TransportResponse.Empty.INSTANCE));
|
||||
if (channel.error.get() != null) {
|
||||
throw channel.error.get();
|
||||
}
|
||||
}
|
||||
channel.clear();
|
||||
|
||||
//now check the last state held
|
||||
assertSameState(node.clusterState, finalState);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that cluster is committed or times out. It should never be the case that we fail
|
||||
* an update due to a commit timeout, but it ends up being committed anyway
|
||||
*/
|
||||
public void testTimeoutOrCommit() throws Exception {
|
||||
Settings settings = Settings.builder()
|
||||
.put(DiscoverySettings.COMMIT_TIMEOUT, "1ms").build(); // short but so we will sometime commit sometime timeout
|
||||
|
||||
MockNode master = createMockNode("master", settings);
|
||||
MockNode node = createMockNode("node", settings);
|
||||
ClusterState state = ClusterState.builder(master.clusterState)
|
||||
.nodes(DiscoveryNodes.builder(master.clusterState.nodes()).put(node.discoveryNode).masterNodeId(master.discoveryNode.id())).build();
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
state = ClusterState.builder(state).incrementVersion().build();
|
||||
logger.debug("--> publishing version [{}], UUID [{}]", state.version(), state.stateUUID());
|
||||
boolean success;
|
||||
try {
|
||||
publishState(master.action, state, master.clusterState, 2).await(1, TimeUnit.HOURS);
|
||||
success = true;
|
||||
} catch (Discovery.FailedToCommitClusterStateException OK) {
|
||||
success = false;
|
||||
}
|
||||
logger.debug("--> publishing [{}], verifying...", success ? "succeeded" : "failed");
|
||||
|
||||
if (success) {
|
||||
assertSameState(node.clusterState, state);
|
||||
} else {
|
||||
assertThat(node.clusterState.stateUUID(), not(equalTo(state.stateUUID())));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private MetaData buildMetaDataForVersion(MetaData metaData, long version) {
|
||||
ImmutableOpenMap.Builder<String, IndexMetaData> indices = ImmutableOpenMap.builder(metaData.indices());
|
||||
indices.put("test" + version, IndexMetaData.builder("test" + version).settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
|
||||
.numberOfShards((int) version).numberOfReplicas(0).build());
|
||||
return MetaData.builder(metaData)
|
||||
.transientSettings(Settings.builder().put("test", version).build())
|
||||
.indices(indices.build())
|
||||
.build();
|
||||
}
|
||||
|
||||
private void assertProperMetaDataForVersion(MetaData metaData, long version) {
|
||||
for (long i = 1; i <= version; i++) {
|
||||
assertThat(metaData.index("test" + i), notNullValue());
|
||||
assertThat(metaData.index("test" + i).numberOfShards(), equalTo((int) i));
|
||||
}
|
||||
assertThat(metaData.index("test" + (version + 1)), nullValue());
|
||||
assertThat(metaData.transientSettings().get("test"), equalTo(Long.toString(version)));
|
||||
}
|
||||
|
||||
public void publishStateAndWait(PublishClusterStateAction action, ClusterState state, ClusterState previousState) throws InterruptedException {
|
||||
publishState(action, state, previousState).await(1, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
public AssertingAckListener publishState(PublishClusterStateAction action, ClusterState state, ClusterState previousState) throws InterruptedException {
|
||||
final int minimumMasterNodes = randomIntBetween(-1, state.nodes().getMasterNodes().size());
|
||||
return publishState(action, state, previousState, minimumMasterNodes);
|
||||
}
|
||||
|
||||
public AssertingAckListener publishState(PublishClusterStateAction action, ClusterState state, ClusterState previousState, int minMasterNodes) throws InterruptedException {
|
||||
AssertingAckListener assertingAckListener = new AssertingAckListener(state.nodes().getSize() - 1);
|
||||
ClusterChangedEvent changedEvent = new ClusterChangedEvent("test update", state, previousState);
|
||||
action.publish(changedEvent, minMasterNodes, assertingAckListener);
|
||||
return assertingAckListener;
|
||||
}
|
||||
|
||||
public static class AssertingAckListener implements Discovery.AckListener {
|
||||
private final List<Tuple<DiscoveryNode, Throwable>> errors = new CopyOnWriteArrayList<>();
|
||||
private final AtomicBoolean timeoutOccurred = new AtomicBoolean();
|
||||
private final CountDownLatch countDown;
|
||||
|
||||
public AssertingAckListener(int nodeCount) {
|
||||
countDown = new CountDownLatch(nodeCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) {
|
||||
if (t != null) {
|
||||
errors.add(new Tuple<>(node, t));
|
||||
}
|
||||
countDown.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTimeout() {
|
||||
timeoutOccurred.set(true);
|
||||
// Fast forward the counter - no reason to wait here
|
||||
long currentCount = countDown.getCount();
|
||||
for (long i = 0; i < currentCount; i++) {
|
||||
countDown.countDown();
|
||||
}
|
||||
}
|
||||
|
||||
public void await(long timeout, TimeUnit unit) throws InterruptedException {
|
||||
assertThat(awaitErrors(timeout, unit), emptyIterable());
|
||||
}
|
||||
|
||||
public List<Tuple<DiscoveryNode, Throwable>> awaitErrors(long timeout, TimeUnit unit) throws InterruptedException {
|
||||
countDown.await(timeout, unit);
|
||||
assertFalse(timeoutOccurred.get());
|
||||
return errors;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void assertSameState(ClusterState actual, ClusterState expected) {
|
||||
assertThat(actual, notNullValue());
|
||||
final String reason = "\n--> actual ClusterState: " + actual.prettyPrint() + "\n--> expected ClusterState:" + expected.prettyPrint();
|
||||
assertThat("unequal UUIDs" + reason, actual.stateUUID(), equalTo(expected.stateUUID()));
|
||||
assertThat("unequal versions" + reason, actual.version(), equalTo(expected.version()));
|
||||
}
|
||||
|
||||
void assertSameStateFromDiff(ClusterState actual, ClusterState expected) {
|
||||
assertSameState(actual, expected);
|
||||
assertTrue(actual.wasReadFromDiff());
|
||||
}
|
||||
|
||||
void assertSameStateFromFull(ClusterState actual, ClusterState expected) {
|
||||
assertSameState(actual, expected);
|
||||
assertFalse(actual.wasReadFromDiff());
|
||||
}
|
||||
|
||||
static class MockPublishAction extends PublishClusterStateAction {
|
||||
|
||||
AtomicBoolean timeoutOnSend = new AtomicBoolean();
|
||||
AtomicBoolean errorOnSend = new AtomicBoolean();
|
||||
AtomicBoolean timeoutOnCommit = new AtomicBoolean();
|
||||
AtomicBoolean errorOnCommit = new AtomicBoolean();
|
||||
|
||||
public MockPublishAction(Settings settings, TransportService transportService, DiscoveryNodesProvider nodesProvider, NewPendingClusterStateListener listener, DiscoverySettings discoverySettings, ClusterName clusterName) {
|
||||
super(settings, transportService, nodesProvider, listener, discoverySettings, clusterName);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void handleIncomingClusterStateRequest(BytesTransportRequest request, TransportChannel channel) throws IOException {
|
||||
if (errorOnSend.get()) {
|
||||
throw new ElasticsearchException("forced error on incoming cluster state");
|
||||
}
|
||||
if (timeoutOnSend.get()) {
|
||||
return;
|
||||
}
|
||||
super.handleIncomingClusterStateRequest(request, channel);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void handleCommitRequest(PublishClusterStateAction.CommitClusterStateRequest request, TransportChannel channel) {
|
||||
if (errorOnCommit.get()) {
|
||||
throw new ElasticsearchException("forced error on incoming commit");
|
||||
}
|
||||
if (timeoutOnCommit.get()) {
|
||||
return;
|
||||
}
|
||||
super.handleCommitRequest(request, channel);
|
||||
}
|
||||
}
|
||||
|
||||
static class CapturingTransportChannel implements TransportChannel {
|
||||
|
||||
AtomicReference<TransportResponse> response = new AtomicReference<>();
|
||||
AtomicReference<Throwable> error = new AtomicReference<>();
|
||||
|
||||
public void clear() {
|
||||
response.set(null);
|
||||
error.set(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String action() {
|
||||
return "_noop_";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getProfileName() {
|
||||
return "_noop_";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendResponse(TransportResponse response) throws IOException {
|
||||
this.response.set(response);
|
||||
assertThat(error.get(), nullValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendResponse(TransportResponse response, TransportResponseOptions options) throws IOException {
|
||||
this.response.set(response);
|
||||
assertThat(error.get(), nullValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendResponse(Throwable error) throws IOException {
|
||||
this.error.set(error);
|
||||
assertThat(response.get(), nullValue());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -91,6 +91,8 @@ import org.elasticsearch.common.xcontent.XContentFactory;
|
|||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
|
@ -128,33 +130,15 @@ import org.junit.BeforeClass;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.lang.annotation.Annotation;
|
||||
import java.lang.annotation.ElementType;
|
||||
import java.lang.annotation.Inherited;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
import java.lang.annotation.*;
|
||||
import java.net.InetAddress;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.UnknownHostException;
|
||||
import java.nio.file.DirectoryStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.*;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.BooleanSupplier;
|
||||
|
@ -166,14 +150,8 @@ import static org.elasticsearch.common.util.CollectionUtils.eagerPartition;
|
|||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
import static org.elasticsearch.test.XContentTestUtils.convertToMap;
|
||||
import static org.elasticsearch.test.XContentTestUtils.differenceBetweenMapsIgnoringArrayOrder;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout;
|
||||
import static org.hamcrest.Matchers.emptyIterable;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.startsWith;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
/**
|
||||
* {@link ESIntegTestCase} is an abstract base class to run integration
|
||||
|
@ -604,6 +582,21 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
}
|
||||
ensureClusterSizeConsistency();
|
||||
ensureClusterStateConsistency();
|
||||
if (isInternalCluster()) {
|
||||
// check no pending cluster states are leaked
|
||||
for (Discovery discovery : internalCluster().getInstances(Discovery.class)) {
|
||||
if (discovery instanceof ZenDiscovery) {
|
||||
final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery;
|
||||
assertBusy(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
assertThat("still having pending states: " + Strings.arrayToDelimitedString(zenDiscovery.pendingClusterStates(), "\n"),
|
||||
zenDiscovery.pendingClusterStates(), emptyArray());
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
beforeIndexDeletion();
|
||||
cluster().wipe(); // wipe after to make sure we fail in the test that didn't ack the delete
|
||||
if (afterClass || currentClusterScope == Scope.TEST) {
|
||||
|
@ -878,7 +871,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
String failMsg = sb.toString();
|
||||
for (SearchHit hit : searchResponse.getHits().getHits()) {
|
||||
sb.append("\n-> _index: [").append(hit.getIndex()).append("] type [").append(hit.getType())
|
||||
.append("] id [").append(hit.id()).append("]");
|
||||
.append("] id [").append(hit.id()).append("]");
|
||||
}
|
||||
logger.warn(sb.toString());
|
||||
fail(failMsg);
|
||||
|
@ -1642,7 +1635,6 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
|
||||
|
||||
private Scope getCurrentClusterScope() {
|
||||
return getCurrentClusterScope(this.getClass());
|
||||
}
|
||||
|
@ -1777,14 +1769,17 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
return Settings.builder().put(Node.HTTP_ENABLED, false).
|
||||
put(ESIntegTestCase.this.nodeSettings(nodeOrdinal)).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
return ESIntegTestCase.this.nodePlugins();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Settings transportClientSettings() {
|
||||
return ESIntegTestCase.this.transportClientSettings();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Class<? extends Plugin>> transportClientPlugins() {
|
||||
return ESIntegTestCase.this.transportClientPlugins();
|
||||
|
@ -1805,7 +1800,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
String nodeMode = InternalTestCluster.configuredNodeMode();
|
||||
if (noLocal != null && noNetwork != null) {
|
||||
throw new IllegalStateException("Can't suppress both network and local mode");
|
||||
} else if (noLocal != null){
|
||||
} else if (noLocal != null) {
|
||||
nodeMode = "network";
|
||||
} else if (noNetwork != null) {
|
||||
nodeMode = "local";
|
||||
|
@ -2110,13 +2105,15 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
*/
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@Inherited
|
||||
public @interface SuppressLocalMode {}
|
||||
public @interface SuppressLocalMode {
|
||||
}
|
||||
|
||||
/**
|
||||
* If used the test will never run in network mode
|
||||
*/
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@Inherited
|
||||
public @interface SuppressNetworkMode {}
|
||||
public @interface SuppressNetworkMode {
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -60,6 +60,10 @@ public class NetworkDelaysPartition extends NetworkPartition {
|
|||
this(nodesSideOne, nodesSideTwo, DEFAULT_DELAY_MIN, DEFAULT_DELAY_MAX, random);
|
||||
}
|
||||
|
||||
public NetworkDelaysPartition(Set<String> nodesSideOne, Set<String> nodesSideTwo, long delay, Random random) {
|
||||
this(nodesSideOne, nodesSideTwo, delay, delay, random);
|
||||
}
|
||||
|
||||
public NetworkDelaysPartition(Set<String> nodesSideOne, Set<String> nodesSideTwo, long delayMin, long delayMax, Random random) {
|
||||
super(nodesSideOne, nodesSideTwo, random);
|
||||
this.delayMin = delayMin;
|
||||
|
@ -69,7 +73,7 @@ public class NetworkDelaysPartition extends NetworkPartition {
|
|||
|
||||
@Override
|
||||
public synchronized void startDisrupting() {
|
||||
duration = new TimeValue(delayMin + random.nextInt((int) (delayMax - delayMin)));
|
||||
duration = new TimeValue(delayMin == delayMax ? delayMin : delayMin + random.nextInt((int) (delayMax - delayMin)));
|
||||
super.startDisrupting();
|
||||
}
|
||||
|
||||
|
|
|
@ -108,12 +108,18 @@ considered failed. Defaults to `3`.
|
|||
The master node is the only node in a cluster that can make changes to the
|
||||
cluster state. The master node processes one cluster state update at a time,
|
||||
applies the required changes and publishes the updated cluster state to all
|
||||
the other nodes in the cluster. Each node receives the publish message,
|
||||
updates its own cluster state and replies to the master node, which waits for
|
||||
all nodes to respond, up to a timeout, before going ahead processing the next
|
||||
updates in the queue. The `discovery.zen.publish_timeout` is set by default
|
||||
to 30 seconds and can be changed dynamically through the
|
||||
<<cluster-update-settings,cluster update settings api>>
|
||||
the other nodes in the cluster. Each node receives the publish message, acknowledges
|
||||
it, but does *not* yet apply it. If the master does not receive acknowledgement from
|
||||
at least `discovery.zen.minimum_master_nodes` nodes within a certain time (controlled by
|
||||
the `discovery.zen.commit_timeout` setting and defaults to 30 seconds) the cluster state
|
||||
change is rejected.
|
||||
|
||||
Once enough nodes have responded, the cluster state is committed and a message will
|
||||
be sent to all the nodes. The nodes then proceed to apply the new cluster state to their
|
||||
internal state. The master node waits for all nodes to respond, up to a timeout, before
|
||||
going ahead processing the next updates in the queue. The `discovery.zen.publish_timeout` is
|
||||
set by default to 30 seconds and is measured from the moment the publishing started. Both
|
||||
timeout settings can be changed dynamically through the <<cluster-update-settings,cluster update settings api>>
|
||||
|
||||
[float]
|
||||
[[no-master-block]]
|
||||
|
|
|
@ -55,6 +55,21 @@ If you encounter an issue, https://github.com/elasticsearch/elasticsearch/issues
|
|||
|
||||
We are committed to tracking down and fixing all the issues that are posted.
|
||||
|
||||
[float]
|
||||
=== Use two phase commit for Cluster State publishing (STATUS: ONGOING)
|
||||
|
||||
A master node in Elasticsearch continuously https://www.elastic.co/guide/en/elasticsearch/reference/current/modules-discovery-zen.html#fault-detection[monitors the cluster nodes]
|
||||
and removes any node from the cluster that doesn't respond to its pings in a timely
|
||||
fashion. If the master is left with fewer nodes than the `discovery.zen.minimum_master_nodes`
|
||||
settings, it will step down and a new master election will start.
|
||||
|
||||
When a network partition causes a master node to lose many followers, there is a short window
|
||||
in time until the node loss is detected and the master steps down. During that window, the
|
||||
master may erroneously accept and acknowledge cluster state changes. To avoid this, we introduce
|
||||
a new phase to cluster state publishing where the proposed cluster state is sent to all nodes
|
||||
but is not yet committed. Only once enough nodes (`discovery.zen.minimum_master_nodes`) actively acknowledge
|
||||
the change, it is committed and commit messages are sent to the nodes. See {GIT}13062[#13062].
|
||||
|
||||
[float]
|
||||
=== Make index creation more user friendly (STATUS: ONGOING)
|
||||
|
||||
|
|
Loading…
Reference in New Issue