Merge branch 'master' into feature/client_aggs_parsing

This commit is contained in:
Christoph Büscher 2017-04-28 16:42:55 +02:00
commit db07a34718
73 changed files with 3396 additions and 2668 deletions

View File

@ -1047,7 +1047,7 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]Discovery.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]Discovery.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]DiscoveryModule.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]DiscoveryModule.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]DiscoverySettings.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]DiscoverySettings.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]NoneDiscovery.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]TribeDiscovery.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]zen[/\\]ElectMasterService.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]zen[/\\]ElectMasterService.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]zen[/\\]FaultDetection.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]zen[/\\]FaultDetection.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]zen[/\\]MasterFaultDetection.java" checks="LineLength" /> <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]zen[/\\]MasterFaultDetection.java" checks="LineLength" />

View File

@ -194,14 +194,14 @@ public class TransportClusterHealthAction extends TransportMasterNodeReadAction<
} }
private boolean validateRequest(final ClusterHealthRequest request, ClusterState clusterState, final int waitFor) { private boolean validateRequest(final ClusterHealthRequest request, ClusterState clusterState, final int waitFor) {
ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.numberOfPendingTasks(), ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.getMasterService().numberOfPendingTasks(),
gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMaxTaskWaitTime()); gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMasterService().getMaxTaskWaitTime());
return prepareResponse(request, response, clusterState, waitFor); return prepareResponse(request, response, clusterState, waitFor);
} }
private ClusterHealthResponse getResponse(final ClusterHealthRequest request, ClusterState clusterState, final int waitFor, boolean timedOut) { private ClusterHealthResponse getResponse(final ClusterHealthRequest request, ClusterState clusterState, final int waitFor, boolean timedOut) {
ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.numberOfPendingTasks(), ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.getMasterService().numberOfPendingTasks(),
gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMaxTaskWaitTime()); gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMasterService().getMaxTaskWaitTime());
boolean valid = prepareResponse(request, response, clusterState, waitFor); boolean valid = prepareResponse(request, response, clusterState, waitFor);
assert valid || timedOut; assert valid || timedOut;
// we check for a timeout here since this method might be called from the wait_for_events // we check for a timeout here since this method might be called from the wait_for_events

View File

@ -65,7 +65,7 @@ public class TransportPendingClusterTasksAction extends TransportMasterNodeReadA
@Override @Override
protected void masterOperation(PendingClusterTasksRequest request, ClusterState state, ActionListener<PendingClusterTasksResponse> listener) { protected void masterOperation(PendingClusterTasksRequest request, ClusterState state, ActionListener<PendingClusterTasksResponse> listener) {
logger.trace("fetching pending tasks from cluster service"); logger.trace("fetching pending tasks from cluster service");
final List<PendingClusterTask> pendingTasks = clusterService.pendingTasks(); final List<PendingClusterTask> pendingTasks = clusterService.getMasterService().pendingTasks();
logger.trace("done fetching pending tasks from cluster service"); logger.trace("done fetching pending tasks from cluster service");
listener.onResponse(new PendingClusterTasksResponse(pendingTasks)); listener.onResponse(new PendingClusterTasksResponse(pendingTasks));
} }

View File

@ -98,6 +98,12 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
@Override @Override
public ActionRequestValidationException validate() { public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = super.validate(); ActionRequestValidationException validationException = super.validate();
if (version != Versions.MATCH_ANY && upsertRequest != null) {
validationException = addValidationError("can't provide both upsert request and a version", validationException);
}
if(upsertRequest != null && upsertRequest.version() != Versions.MATCH_ANY) {
validationException = addValidationError("can't provide version in upsert request", validationException);
}
if (type == null) { if (type == null) {
validationException = addValidationError("type is missing", validationException); validationException = addValidationError("type is missing", validationException);
} }

View File

@ -22,7 +22,6 @@ package org.elasticsearch.cluster;
import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;

View File

@ -21,6 +21,7 @@ package org.elasticsearch.cluster;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.service.ClusterApplierService;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
@ -42,7 +43,7 @@ public class ClusterStateObserver {
private final Predicate<ClusterState> MATCH_ALL_CHANGES_PREDICATE = state -> true; private final Predicate<ClusterState> MATCH_ALL_CHANGES_PREDICATE = state -> true;
private final ClusterService clusterService; private final ClusterApplierService clusterApplierService;
private final ThreadContext contextHolder; private final ThreadContext contextHolder;
volatile TimeValue timeOutValue; volatile TimeValue timeOutValue;
@ -74,7 +75,12 @@ public class ClusterStateObserver {
*/ */
public ClusterStateObserver(ClusterState initialState, ClusterService clusterService, @Nullable TimeValue timeout, Logger logger, public ClusterStateObserver(ClusterState initialState, ClusterService clusterService, @Nullable TimeValue timeout, Logger logger,
ThreadContext contextHolder) { ThreadContext contextHolder) {
this.clusterService = clusterService; this(initialState, clusterService.getClusterApplierService(), timeout, logger, contextHolder);
}
public ClusterStateObserver(ClusterState initialState, ClusterApplierService clusterApplierService, @Nullable TimeValue timeout,
Logger logger, ThreadContext contextHolder) {
this.clusterApplierService = clusterApplierService;
this.lastObservedState = new AtomicReference<>(new StoredState(initialState)); this.lastObservedState = new AtomicReference<>(new StoredState(initialState));
this.timeOutValue = timeout; this.timeOutValue = timeout;
if (timeOutValue != null) { if (timeOutValue != null) {
@ -89,7 +95,7 @@ public class ClusterStateObserver {
if (observingContext.get() != null) { if (observingContext.get() != null) {
throw new ElasticsearchException("cannot set current cluster state while waiting for a cluster state change"); throw new ElasticsearchException("cannot set current cluster state while waiting for a cluster state change");
} }
ClusterState clusterState = clusterService.state(); ClusterState clusterState = clusterApplierService.state();
lastObservedState.set(new StoredState(clusterState)); lastObservedState.set(new StoredState(clusterState));
return clusterState; return clusterState;
} }
@ -135,7 +141,7 @@ public class ClusterStateObserver {
logger.trace("observer timed out. notifying listener. timeout setting [{}], time since start [{}]", timeOutValue, new TimeValue(timeSinceStartMS)); logger.trace("observer timed out. notifying listener. timeout setting [{}], time since start [{}]", timeOutValue, new TimeValue(timeSinceStartMS));
// update to latest, in case people want to retry // update to latest, in case people want to retry
timedOut = true; timedOut = true;
lastObservedState.set(new StoredState(clusterService.state())); lastObservedState.set(new StoredState(clusterApplierService.state()));
listener.onTimeout(timeOutValue); listener.onTimeout(timeOutValue);
return; return;
} }
@ -151,7 +157,7 @@ public class ClusterStateObserver {
// sample a new state. This state maybe *older* than the supplied state if we are called from an applier, // sample a new state. This state maybe *older* than the supplied state if we are called from an applier,
// which wants to wait for something else to happen // which wants to wait for something else to happen
ClusterState newState = clusterService.state(); ClusterState newState = clusterApplierService.state();
if (lastObservedState.get().isOlderOrDifferentMaster(newState) && statePredicate.test(newState)) { if (lastObservedState.get().isOlderOrDifferentMaster(newState) && statePredicate.test(newState)) {
// good enough, let's go. // good enough, let's go.
logger.trace("observer: sampled state accepted by predicate ({})", newState); logger.trace("observer: sampled state accepted by predicate ({})", newState);
@ -163,7 +169,7 @@ public class ClusterStateObserver {
if (!observingContext.compareAndSet(null, context)) { if (!observingContext.compareAndSet(null, context)) {
throw new ElasticsearchException("already waiting for a cluster state change"); throw new ElasticsearchException("already waiting for a cluster state change");
} }
clusterService.addTimeoutListener(timeoutTimeLeftMS == null ? null : new TimeValue(timeoutTimeLeftMS), clusterStateListener); clusterApplierService.addTimeoutListener(timeoutTimeLeftMS == null ? null : new TimeValue(timeoutTimeLeftMS), clusterStateListener);
} }
} }
@ -179,7 +185,7 @@ public class ClusterStateObserver {
final ClusterState state = event.state(); final ClusterState state = event.state();
if (context.statePredicate.test(state)) { if (context.statePredicate.test(state)) {
if (observingContext.compareAndSet(context, null)) { if (observingContext.compareAndSet(context, null)) {
clusterService.removeTimeoutListener(this); clusterApplierService.removeTimeoutListener(this);
logger.trace("observer: accepting cluster state change ({})", state); logger.trace("observer: accepting cluster state change ({})", state);
lastObservedState.set(new StoredState(state)); lastObservedState.set(new StoredState(state));
context.listener.onNewClusterState(state); context.listener.onNewClusterState(state);
@ -198,12 +204,12 @@ public class ClusterStateObserver {
// No need to remove listener as it is the responsibility of the thread that set observingContext to null // No need to remove listener as it is the responsibility of the thread that set observingContext to null
return; return;
} }
ClusterState newState = clusterService.state(); ClusterState newState = clusterApplierService.state();
if (lastObservedState.get().isOlderOrDifferentMaster(newState) && context.statePredicate.test(newState)) { if (lastObservedState.get().isOlderOrDifferentMaster(newState) && context.statePredicate.test(newState)) {
// double check we're still listening // double check we're still listening
if (observingContext.compareAndSet(context, null)) { if (observingContext.compareAndSet(context, null)) {
logger.trace("observer: post adding listener: accepting current cluster state ({})", newState); logger.trace("observer: post adding listener: accepting current cluster state ({})", newState);
clusterService.removeTimeoutListener(this); clusterApplierService.removeTimeoutListener(this);
lastObservedState.set(new StoredState(newState)); lastObservedState.set(new StoredState(newState));
context.listener.onNewClusterState(newState); context.listener.onNewClusterState(newState);
} else { } else {
@ -220,7 +226,7 @@ public class ClusterStateObserver {
if (context != null) { if (context != null) {
logger.trace("observer: cluster service closed. notifying listener."); logger.trace("observer: cluster service closed. notifying listener.");
clusterService.removeTimeoutListener(this); clusterApplierService.removeTimeoutListener(this);
context.listener.onClusterServiceClose(); context.listener.onClusterServiceClose();
} }
} }
@ -229,11 +235,11 @@ public class ClusterStateObserver {
public void onTimeout(TimeValue timeout) { public void onTimeout(TimeValue timeout) {
ObservingContext context = observingContext.getAndSet(null); ObservingContext context = observingContext.getAndSet(null);
if (context != null) { if (context != null) {
clusterService.removeTimeoutListener(this); clusterApplierService.removeTimeoutListener(this);
long timeSinceStartMS = TimeValue.nsecToMSec(System.nanoTime() - startTimeNS); long timeSinceStartMS = TimeValue.nsecToMSec(System.nanoTime() - startTimeNS);
logger.trace("observer: timeout notification from cluster service. timeout setting [{}], time since start [{}]", timeOutValue, new TimeValue(timeSinceStartMS)); logger.trace("observer: timeout notification from cluster service. timeout setting [{}], time since start [{}]", timeOutValue, new TimeValue(timeSinceStartMS));
// update to latest, in case people want to retry // update to latest, in case people want to retry
lastObservedState.set(new StoredState(clusterService.state())); lastObservedState.set(new StoredState(clusterApplierService.state()));
timedOut = true; timedOut = true;
context.listener.onTimeout(timeOutValue); context.listener.onTimeout(timeOutValue);
} }

View File

@ -71,21 +71,18 @@ public interface ClusterStateTaskExecutor<T> {
* @param <T> the type of the cluster state update task * @param <T> the type of the cluster state update task
*/ */
class ClusterTasksResult<T> { class ClusterTasksResult<T> {
public final boolean noMaster;
@Nullable @Nullable
public final ClusterState resultingState; public final ClusterState resultingState;
public final Map<T, TaskResult> executionResults; public final Map<T, TaskResult> executionResults;
/** /**
* Construct an execution result instance with a correspondence between the tasks and their execution result * Construct an execution result instance with a correspondence between the tasks and their execution result
* @param noMaster whether this node steps down as master or has lost connection to the master
* @param resultingState the resulting cluster state * @param resultingState the resulting cluster state
* @param executionResults the correspondence between tasks and their outcome * @param executionResults the correspondence between tasks and their outcome
*/ */
ClusterTasksResult(boolean noMaster, ClusterState resultingState, Map<T, TaskResult> executionResults) { ClusterTasksResult(ClusterState resultingState, Map<T, TaskResult> executionResults) {
this.resultingState = resultingState; this.resultingState = resultingState;
this.executionResults = executionResults; this.executionResults = executionResults;
this.noMaster = noMaster;
} }
public static <T> Builder<T> builder() { public static <T> Builder<T> builder() {
@ -124,11 +121,11 @@ public interface ClusterStateTaskExecutor<T> {
} }
public ClusterTasksResult<T> build(ClusterState resultingState) { public ClusterTasksResult<T> build(ClusterState resultingState) {
return new ClusterTasksResult<>(false, resultingState, executionResults); return new ClusterTasksResult<>(resultingState, executionResults);
} }
ClusterTasksResult<T> build(ClusterTasksResult<T> result, ClusterState previousState) { ClusterTasksResult<T> build(ClusterTasksResult<T> result, ClusterState previousState) {
return new ClusterTasksResult<>(result.noMaster, result.resultingState == null ? previousState : result.resultingState, return new ClusterTasksResult<>(result.resultingState == null ? previousState : result.resultingState,
executionResults); executionResults);
} }
} }

View File

@ -18,6 +18,8 @@
*/ */
package org.elasticsearch.cluster; package org.elasticsearch.cluster;
import org.elasticsearch.cluster.service.MasterService;
import java.util.List; import java.util.List;
public interface ClusterStateTaskListener { public interface ClusterStateTaskListener {
@ -28,7 +30,8 @@ public interface ClusterStateTaskListener {
void onFailure(String source, Exception e); void onFailure(String source, Exception e);
/** /**
* called when the task was rejected because the local node is no longer master * called when the task was rejected because the local node is no longer master.
* Used only for tasks submitted to {@link MasterService}.
*/ */
default void onNoLongerMaster(String source) { default void onNoLongerMaster(String source) {
onFailure(source, new NotMasterException("no longer master. source: [" + source + "]")); onFailure(source, new NotMasterException("no longer master. source: [" + source + "]"));

View File

@ -50,25 +50,11 @@ public abstract class LocalClusterUpdateTask implements ClusterStateTaskConfig,
return ClusterTasksResult.<LocalClusterUpdateTask>builder().successes(tasks).build(result, currentState); return ClusterTasksResult.<LocalClusterUpdateTask>builder().successes(tasks).build(result, currentState);
} }
/**
* node stepped down as master or has lost connection to the master
*/
public static ClusterTasksResult<LocalClusterUpdateTask> noMaster() {
return new ClusterTasksResult(true, null, null);
}
/** /**
* no changes were made to the cluster state. Useful to execute a runnable on the cluster state applier thread * no changes were made to the cluster state. Useful to execute a runnable on the cluster state applier thread
*/ */
public static ClusterTasksResult<LocalClusterUpdateTask> unchanged() { public static ClusterTasksResult<LocalClusterUpdateTask> unchanged() {
return new ClusterTasksResult(false, null, null); return new ClusterTasksResult<>(null, null);
}
/**
* locally apply cluster state received from a master
*/
public static ClusterTasksResult<LocalClusterUpdateTask> newState(ClusterState clusterState) {
return new ClusterTasksResult(false, clusterState, null);
} }
@Override @Override

View File

@ -178,8 +178,8 @@ public class DelayedAllocationService extends AbstractLifecycleComponent impleme
/** /**
* Figure out if an existing scheduled reroute is good enough or whether we need to cancel and reschedule. * Figure out if an existing scheduled reroute is good enough or whether we need to cancel and reschedule.
*/ */
private void scheduleIfNeeded(long currentNanoTime, ClusterState state) { private synchronized void scheduleIfNeeded(long currentNanoTime, ClusterState state) {
assertClusterStateThread(); assertClusterOrMasterStateThread();
long nextDelayNanos = UnassignedInfo.findNextDelayedAllocation(currentNanoTime, state); long nextDelayNanos = UnassignedInfo.findNextDelayedAllocation(currentNanoTime, state);
if (nextDelayNanos < 0) { if (nextDelayNanos < 0) {
logger.trace("no need to schedule reroute - no delayed unassigned shards"); logger.trace("no need to schedule reroute - no delayed unassigned shards");
@ -214,7 +214,7 @@ public class DelayedAllocationService extends AbstractLifecycleComponent impleme
} }
// protected so that it can be overridden (and disabled) by unit tests // protected so that it can be overridden (and disabled) by unit tests
protected void assertClusterStateThread() { protected void assertClusterOrMasterStateThread() {
ClusterService.assertClusterStateThread(); assert ClusterService.assertClusterOrMasterStateThread();
} }
} }

View File

@ -0,0 +1,37 @@
/*
* 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.service;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import java.util.function.Supplier;
@FunctionalInterface
public interface ClusterApplier {
/**
* Method to invoke when a new cluster state is available to be applied
*
* @param source information where the cluster state came from
* @param clusterStateSupplier the cluster state supplier which provides the latest cluster state to apply
* @param listener callback that is invoked after cluster state is applied
*/
void onNewClusterState(String source, Supplier<ClusterState> clusterStateSupplier, ClusterStateTaskListener listener);
}

View File

@ -0,0 +1,655 @@
/*
* 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.service;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateApplier;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.ClusterStateTaskConfig;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.LocalNodeMasterListener;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.TimeoutClusterStateListener;
import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor;
import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Stream;
import static org.elasticsearch.cluster.service.ClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING;
import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory;
public class ClusterApplierService extends AbstractLifecycleComponent implements ClusterApplier {
public static final String CLUSTER_UPDATE_THREAD_NAME = "clusterApplierService#updateTask";
private final ClusterSettings clusterSettings;
protected final ThreadPool threadPool;
private volatile TimeValue slowTaskLoggingThreshold;
private volatile PrioritizedEsThreadPoolExecutor threadPoolExecutor;
/**
* Those 3 state listeners are changing infrequently - CopyOnWriteArrayList is just fine
*/
private final Collection<ClusterStateApplier> highPriorityStateAppliers = new CopyOnWriteArrayList<>();
private final Collection<ClusterStateApplier> normalPriorityStateAppliers = new CopyOnWriteArrayList<>();
private final Collection<ClusterStateApplier> lowPriorityStateAppliers = new CopyOnWriteArrayList<>();
private final Iterable<ClusterStateApplier> clusterStateAppliers = Iterables.concat(highPriorityStateAppliers,
normalPriorityStateAppliers, lowPriorityStateAppliers);
private final Collection<ClusterStateListener> clusterStateListeners = new CopyOnWriteArrayList<>();
private final Collection<TimeoutClusterStateListener> timeoutClusterStateListeners =
Collections.newSetFromMap(new ConcurrentHashMap<TimeoutClusterStateListener, Boolean>());
private final LocalNodeMasterListeners localNodeMasterListeners;
private final Queue<NotifyTimeout> onGoingTimeouts = ConcurrentCollections.newQueue();
private final AtomicReference<ClusterState> state; // last applied state
private NodeConnectionsService nodeConnectionsService;
public ClusterApplierService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) {
super(settings);
this.clusterSettings = clusterSettings;
this.threadPool = threadPool;
this.state = new AtomicReference<>();
this.slowTaskLoggingThreshold = CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings);
this.localNodeMasterListeners = new LocalNodeMasterListeners(threadPool);
}
public void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
}
public synchronized void setNodeConnectionsService(NodeConnectionsService nodeConnectionsService) {
assert this.nodeConnectionsService == null : "nodeConnectionsService is already set";
this.nodeConnectionsService = nodeConnectionsService;
}
public void setInitialState(ClusterState initialState) {
if (lifecycle.started()) {
throw new IllegalStateException("can't set initial state when started");
}
assert state.get() == null : "state is already set";
state.set(initialState);
}
@Override
protected synchronized void doStart() {
Objects.requireNonNull(nodeConnectionsService, "please set the node connection service before starting");
Objects.requireNonNull(state.get(), "please set initial state before starting");
addListener(localNodeMasterListeners);
threadPoolExecutor = EsExecutors.newSinglePrioritizing(CLUSTER_UPDATE_THREAD_NAME,
daemonThreadFactory(settings, CLUSTER_UPDATE_THREAD_NAME), threadPool.getThreadContext(), threadPool.scheduler());
}
class UpdateTask extends SourcePrioritizedRunnable implements Function<ClusterState, ClusterState> {
final ClusterStateTaskListener listener;
final Function<ClusterState, ClusterState> updateFunction;
UpdateTask(Priority priority, String source, ClusterStateTaskListener listener,
Function<ClusterState, ClusterState> updateFunction) {
super(priority, source);
this.listener = listener;
this.updateFunction = updateFunction;
}
@Override
public ClusterState apply(ClusterState clusterState) {
return updateFunction.apply(clusterState);
}
@Override
public void run() {
runTask(this);
}
}
@Override
protected synchronized void doStop() {
for (NotifyTimeout onGoingTimeout : onGoingTimeouts) {
onGoingTimeout.cancel();
try {
onGoingTimeout.cancel();
onGoingTimeout.listener.onClose();
} catch (Exception ex) {
logger.debug("failed to notify listeners on shutdown", ex);
}
}
ThreadPool.terminate(threadPoolExecutor, 10, TimeUnit.SECONDS);
// close timeout listeners that did not have an ongoing timeout
timeoutClusterStateListeners.forEach(TimeoutClusterStateListener::onClose);
removeListener(localNodeMasterListeners);
}
@Override
protected synchronized void doClose() {
}
/**
* The current cluster state.
* Should be renamed to appliedClusterState
*/
public ClusterState state() {
assert assertNotCalledFromClusterStateApplier("the applied cluster state is not yet available");
ClusterState clusterState = this.state.get();
assert clusterState != null : "initial cluster state not set yet";
return clusterState;
}
/**
* Adds a high priority applier of updated cluster states.
*/
public void addHighPriorityApplier(ClusterStateApplier applier) {
highPriorityStateAppliers.add(applier);
}
/**
* Adds an applier which will be called after all high priority and normal appliers have been called.
*/
public void addLowPriorityApplier(ClusterStateApplier applier) {
lowPriorityStateAppliers.add(applier);
}
/**
* Adds a applier of updated cluster states.
*/
public void addStateApplier(ClusterStateApplier applier) {
normalPriorityStateAppliers.add(applier);
}
/**
* Removes an applier of updated cluster states.
*/
public void removeApplier(ClusterStateApplier applier) {
normalPriorityStateAppliers.remove(applier);
highPriorityStateAppliers.remove(applier);
lowPriorityStateAppliers.remove(applier);
}
/**
* Add a listener for updated cluster states
*/
public void addListener(ClusterStateListener listener) {
clusterStateListeners.add(listener);
}
/**
* Removes a listener for updated cluster states.
*/
public void removeListener(ClusterStateListener listener) {
clusterStateListeners.remove(listener);
}
/**
* Removes a timeout listener for updated cluster states.
*/
public void removeTimeoutListener(TimeoutClusterStateListener listener) {
timeoutClusterStateListeners.remove(listener);
for (Iterator<NotifyTimeout> it = onGoingTimeouts.iterator(); it.hasNext(); ) {
NotifyTimeout timeout = it.next();
if (timeout.listener.equals(listener)) {
timeout.cancel();
it.remove();
}
}
}
/**
* Add a listener for on/off local node master events
*/
public void addLocalNodeMasterListener(LocalNodeMasterListener listener) {
localNodeMasterListeners.add(listener);
}
/**
* Remove the given listener for on/off local master events
*/
public void removeLocalNodeMasterListener(LocalNodeMasterListener listener) {
localNodeMasterListeners.remove(listener);
}
/**
* Adds a cluster state listener that is expected to be removed during a short period of time.
* If provided, the listener will be notified once a specific time has elapsed.
*
* NOTE: the listener is not removed on timeout. This is the responsibility of the caller.
*/
public void addTimeoutListener(@Nullable final TimeValue timeout, final TimeoutClusterStateListener listener) {
if (lifecycle.stoppedOrClosed()) {
listener.onClose();
return;
}
// call the post added notification on the same event thread
try {
threadPoolExecutor.execute(new SourcePrioritizedRunnable(Priority.HIGH, "_add_listener_") {
@Override
public void run() {
if (timeout != null) {
NotifyTimeout notifyTimeout = new NotifyTimeout(listener, timeout);
notifyTimeout.future = threadPool.schedule(timeout, ThreadPool.Names.GENERIC, notifyTimeout);
onGoingTimeouts.add(notifyTimeout);
}
timeoutClusterStateListeners.add(listener);
listener.postAdded();
}
});
} catch (EsRejectedExecutionException e) {
if (lifecycle.stoppedOrClosed()) {
listener.onClose();
} else {
throw e;
}
}
}
public void runOnApplierThread(final String source, Consumer<ClusterState> clusterStateConsumer,
final ClusterStateTaskListener listener, Priority priority) {
submitStateUpdateTask(source, ClusterStateTaskConfig.build(priority),
(clusterState) -> {
clusterStateConsumer.accept(clusterState);
return clusterState;
},
listener);
}
public void runOnApplierThread(final String source, Consumer<ClusterState> clusterStateConsumer,
final ClusterStateTaskListener listener) {
runOnApplierThread(source, clusterStateConsumer, listener, Priority.HIGH);
}
@Override
public void onNewClusterState(final String source, final java.util.function.Supplier<ClusterState> clusterStateSupplier,
final ClusterStateTaskListener listener) {
Function<ClusterState, ClusterState> applyFunction = currentState -> {
ClusterState nextState = clusterStateSupplier.get();
if (nextState != null) {
return nextState;
} else {
return currentState;
}
};
submitStateUpdateTask(source, ClusterStateTaskConfig.build(Priority.HIGH), applyFunction, listener);
}
private void submitStateUpdateTask(final String source, final ClusterStateTaskConfig config,
final Function<ClusterState, ClusterState> executor,
final ClusterStateTaskListener listener) {
if (!lifecycle.started()) {
return;
}
try {
UpdateTask updateTask = new UpdateTask(config.priority(), source, new SafeClusterStateTaskListener(listener, logger), executor);
if (config.timeout() != null) {
threadPoolExecutor.execute(updateTask, config.timeout(),
() -> threadPool.generic().execute(
() -> listener.onFailure(source, new ProcessClusterEventTimeoutException(config.timeout(), source))));
} else {
threadPoolExecutor.execute(updateTask);
}
} catch (EsRejectedExecutionException e) {
// ignore cases where we are shutting down..., there is really nothing interesting
// to be done here...
if (!lifecycle.stoppedOrClosed()) {
throw e;
}
}
}
/** asserts that the current thread is the cluster state update thread */
public static boolean assertClusterStateUpdateThread() {
assert Thread.currentThread().getName().contains(ClusterApplierService.CLUSTER_UPDATE_THREAD_NAME) :
"not called from the cluster state update thread";
return true;
}
/** asserts that the current thread is <b>NOT</b> the cluster state update thread */
public static boolean assertNotClusterStateUpdateThread(String reason) {
assert Thread.currentThread().getName().contains(CLUSTER_UPDATE_THREAD_NAME) == false :
"Expected current thread [" + Thread.currentThread() + "] to not be the cluster state update thread. Reason: [" + reason + "]";
return true;
}
/** asserts that the current stack trace does <b>NOT</b> involve a cluster state applier */
private static boolean assertNotCalledFromClusterStateApplier(String reason) {
if (Thread.currentThread().getName().contains(CLUSTER_UPDATE_THREAD_NAME)) {
for (StackTraceElement element : Thread.currentThread().getStackTrace()) {
final String className = element.getClassName();
final String methodName = element.getMethodName();
if (className.equals(ClusterStateObserver.class.getName())) {
// people may start an observer from an applier
return true;
} else if (className.equals(ClusterApplierService.class.getName())
&& methodName.equals("callClusterStateAppliers")) {
throw new AssertionError("should not be called by a cluster state applier. reason [" + reason + "]");
}
}
}
return true;
}
protected void runTask(UpdateTask task) {
if (!lifecycle.started()) {
logger.debug("processing [{}]: ignoring, cluster applier service not started", task.source);
return;
}
logger.debug("processing [{}]: execute", task.source);
final ClusterState previousClusterState = state.get();
long startTimeNS = currentTimeInNanos();
final ClusterState newClusterState;
try {
newClusterState = task.apply(previousClusterState);
} catch (Exception e) {
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
if (logger.isTraceEnabled()) {
logger.trace(
(Supplier<?>) () -> new ParameterizedMessage(
"failed to execute cluster state applier in [{}], state:\nversion [{}], source [{}]\n{}{}{}",
executionTime,
previousClusterState.version(),
task.source,
previousClusterState.nodes(),
previousClusterState.routingTable(),
previousClusterState.getRoutingNodes()),
e);
}
warnAboutSlowTaskIfNeeded(executionTime, task.source);
task.listener.onFailure(task.source, e);
return;
}
if (previousClusterState == newClusterState) {
task.listener.clusterStateProcessed(task.source, newClusterState, newClusterState);
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
logger.debug("processing [{}]: took [{}] no change in cluster state", task.source, executionTime);
warnAboutSlowTaskIfNeeded(executionTime, task.source);
} else {
if (logger.isTraceEnabled()) {
logger.trace("cluster state updated, source [{}]\n{}", task.source, newClusterState);
} else if (logger.isDebugEnabled()) {
logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), task.source);
}
try {
applyChanges(task, previousClusterState, newClusterState);
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
logger.debug("processing [{}]: took [{}] done applying updated cluster state (version: {}, uuid: {})", task.source,
executionTime, newClusterState.version(),
newClusterState.stateUUID());
warnAboutSlowTaskIfNeeded(executionTime, task.source);
} catch (Exception e) {
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
final long version = newClusterState.version();
final String stateUUID = newClusterState.stateUUID();
final String fullState = newClusterState.toString();
logger.warn(
(Supplier<?>) () -> new ParameterizedMessage(
"failed to apply updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}",
executionTime,
version,
stateUUID,
task.source,
fullState),
e);
// TODO: do we want to call updateTask.onFailure here?
}
}
}
private void applyChanges(UpdateTask task, ClusterState previousClusterState, ClusterState newClusterState) {
ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent(task.source, newClusterState, previousClusterState);
// new cluster state, notify all listeners
final DiscoveryNodes.Delta nodesDelta = clusterChangedEvent.nodesDelta();
if (nodesDelta.hasChanges() && logger.isInfoEnabled()) {
String summary = nodesDelta.shortSummary();
if (summary.length() > 0) {
logger.info("{}, reason: {}", summary, task.source);
}
}
nodeConnectionsService.connectToNodes(newClusterState.nodes());
logger.debug("applying cluster state version {}", newClusterState.version());
try {
// nothing to do until we actually recover from the gateway or any other block indicates we need to disable persistency
if (clusterChangedEvent.state().blocks().disableStatePersistence() == false && clusterChangedEvent.metaDataChanged()) {
final Settings incomingSettings = clusterChangedEvent.state().metaData().settings();
clusterSettings.applySettings(incomingSettings);
}
} catch (Exception ex) {
logger.warn("failed to apply cluster settings", ex);
}
logger.debug("apply cluster state with version {}", newClusterState.version());
callClusterStateAppliers(clusterChangedEvent);
nodeConnectionsService.disconnectFromNodesExcept(newClusterState.nodes());
logger.debug("set locally applied cluster state to version {}", newClusterState.version());
state.set(newClusterState);
callClusterStateListeners(clusterChangedEvent);
task.listener.clusterStateProcessed(task.source, previousClusterState, newClusterState);
}
private void callClusterStateAppliers(ClusterChangedEvent clusterChangedEvent) {
clusterStateAppliers.forEach(applier -> {
try {
logger.trace("calling [{}] with change to version [{}]", applier, clusterChangedEvent.state().version());
applier.applyClusterState(clusterChangedEvent);
} catch (Exception ex) {
logger.warn("failed to notify ClusterStateApplier", ex);
}
});
}
private void callClusterStateListeners(ClusterChangedEvent clusterChangedEvent) {
Stream.concat(clusterStateListeners.stream(), timeoutClusterStateListeners.stream()).forEach(listener -> {
try {
logger.trace("calling [{}] with change to version [{}]", listener, clusterChangedEvent.state().version());
listener.clusterChanged(clusterChangedEvent);
} catch (Exception ex) {
logger.warn("failed to notify ClusterStateListener", ex);
}
});
}
private static class SafeClusterStateTaskListener implements ClusterStateTaskListener {
private final ClusterStateTaskListener listener;
private final Logger logger;
SafeClusterStateTaskListener(ClusterStateTaskListener listener, Logger logger) {
this.listener = listener;
this.logger = logger;
}
@Override
public void onFailure(String source, Exception e) {
try {
listener.onFailure(source, e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.error(
(Supplier<?>) () -> new ParameterizedMessage(
"exception thrown by listener notifying of failure from [{}]", source), inner);
}
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
try {
listener.clusterStateProcessed(source, oldState, newState);
} catch (Exception e) {
logger.error(
(Supplier<?>) () -> new ParameterizedMessage(
"exception thrown by listener while notifying of cluster state processed from [{}], old cluster state:\n" +
"{}\nnew cluster state:\n{}",
source, oldState, newState),
e);
}
}
}
protected void warnAboutSlowTaskIfNeeded(TimeValue executionTime, String source) {
if (executionTime.getMillis() > slowTaskLoggingThreshold.getMillis()) {
logger.warn("cluster state applier task [{}] took [{}] above the warn threshold of {}", source, executionTime,
slowTaskLoggingThreshold);
}
}
class NotifyTimeout implements Runnable {
final TimeoutClusterStateListener listener;
final TimeValue timeout;
volatile ScheduledFuture future;
NotifyTimeout(TimeoutClusterStateListener listener, TimeValue timeout) {
this.listener = listener;
this.timeout = timeout;
}
public void cancel() {
FutureUtils.cancel(future);
}
@Override
public void run() {
if (future != null && future.isCancelled()) {
return;
}
if (lifecycle.stoppedOrClosed()) {
listener.onClose();
} else {
listener.onTimeout(this.timeout);
}
// note, we rely on the listener to remove itself in case of timeout if needed
}
}
private static class LocalNodeMasterListeners implements ClusterStateListener {
private final List<LocalNodeMasterListener> listeners = new CopyOnWriteArrayList<>();
private final ThreadPool threadPool;
private volatile boolean master = false;
private LocalNodeMasterListeners(ThreadPool threadPool) {
this.threadPool = threadPool;
}
@Override
public void clusterChanged(ClusterChangedEvent event) {
if (!master && event.localNodeMaster()) {
master = true;
for (LocalNodeMasterListener listener : listeners) {
java.util.concurrent.Executor executor = threadPool.executor(listener.executorName());
executor.execute(new OnMasterRunnable(listener));
}
return;
}
if (master && !event.localNodeMaster()) {
master = false;
for (LocalNodeMasterListener listener : listeners) {
java.util.concurrent.Executor executor = threadPool.executor(listener.executorName());
executor.execute(new OffMasterRunnable(listener));
}
}
}
private void add(LocalNodeMasterListener listener) {
listeners.add(listener);
}
private void remove(LocalNodeMasterListener listener) {
listeners.remove(listener);
}
private void clear() {
listeners.clear();
}
}
private static class OnMasterRunnable implements Runnable {
private final LocalNodeMasterListener listener;
private OnMasterRunnable(LocalNodeMasterListener listener) {
this.listener = listener;
}
@Override
public void run() {
listener.onMaster();
}
}
private static class OffMasterRunnable implements Runnable {
private final LocalNodeMasterListener listener;
private OffMasterRunnable(LocalNodeMasterListener listener) {
this.listener = listener;
}
@Override
public void run() {
listener.offMaster();
}
}
// this one is overridden in tests so we can control time
protected long currentTimeInNanos() {
return System.nanoTime();
}
}

View File

@ -0,0 +1,752 @@
/*
* 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.service;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.cluster.AckedClusterStateTaskListener;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterState.Builder;
import org.elasticsearch.cluster.ClusterStateTaskConfig;
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterStateTaskExecutor.ClusterTasksResult;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.CountDown;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;
import static org.elasticsearch.cluster.service.ClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING;
import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory;
public class MasterService extends AbstractLifecycleComponent {
public static final String MASTER_UPDATE_THREAD_NAME = "masterService#updateTask";
private BiConsumer<ClusterChangedEvent, Discovery.AckListener> clusterStatePublisher;
private java.util.function.Supplier<ClusterState> clusterStateSupplier;
private volatile TimeValue slowTaskLoggingThreshold;
protected final ThreadPool threadPool;
private volatile PrioritizedEsThreadPoolExecutor threadPoolExecutor;
private volatile Batcher taskBatcher;
public MasterService(Settings settings, ThreadPool threadPool) {
super(settings);
// TODO: introduce a dedicated setting for master service
this.slowTaskLoggingThreshold = CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings);
this.threadPool = threadPool;
}
public void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
}
public synchronized void setClusterStatePublisher(BiConsumer<ClusterChangedEvent, Discovery.AckListener> publisher) {
clusterStatePublisher = publisher;
}
public synchronized void setClusterStateSupplier(java.util.function.Supplier<ClusterState> clusterStateSupplier) {
this.clusterStateSupplier = clusterStateSupplier;
}
@Override
protected synchronized void doStart() {
Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting");
Objects.requireNonNull(clusterStateSupplier, "please set a cluster state supplier before starting");
threadPoolExecutor = EsExecutors.newSinglePrioritizing(MASTER_UPDATE_THREAD_NAME,
daemonThreadFactory(settings, MASTER_UPDATE_THREAD_NAME), threadPool.getThreadContext(), threadPool.scheduler());
taskBatcher = new Batcher(logger, threadPoolExecutor);
}
class Batcher extends TaskBatcher {
Batcher(Logger logger, PrioritizedEsThreadPoolExecutor threadExecutor) {
super(logger, threadExecutor);
}
@Override
protected void onTimeout(List<? extends BatchedTask> tasks, TimeValue timeout) {
threadPool.generic().execute(
() -> tasks.forEach(
task -> ((UpdateTask) task).listener.onFailure(task.source,
new ProcessClusterEventTimeoutException(timeout, task.source))));
}
@Override
protected void run(Object batchingKey, List<? extends BatchedTask> tasks, String tasksSummary) {
ClusterStateTaskExecutor<Object> taskExecutor = (ClusterStateTaskExecutor<Object>) batchingKey;
List<UpdateTask> updateTasks = (List<UpdateTask>) tasks;
runTasks(new TaskInputs(taskExecutor, updateTasks, tasksSummary));
}
class UpdateTask extends BatchedTask {
final ClusterStateTaskListener listener;
UpdateTask(Priority priority, String source, Object task, ClusterStateTaskListener listener,
ClusterStateTaskExecutor<?> executor) {
super(priority, source, executor, task);
this.listener = listener;
}
@Override
public String describeTasks(List<? extends BatchedTask> tasks) {
return ((ClusterStateTaskExecutor<Object>) batchingKey).describeTasks(
tasks.stream().map(BatchedTask::getTask).collect(Collectors.toList()));
}
}
}
@Override
protected synchronized void doStop() {
ThreadPool.terminate(threadPoolExecutor, 10, TimeUnit.SECONDS);
}
@Override
protected synchronized void doClose() {
}
/**
* The current cluster state exposed by the discovery layer. Package-visible for tests.
*/
ClusterState state() {
return clusterStateSupplier.get();
}
public static boolean assertMasterUpdateThread() {
assert Thread.currentThread().getName().contains(MASTER_UPDATE_THREAD_NAME) :
"not called from the master service thread";
return true;
}
public static boolean assertNotMasterUpdateThread(String reason) {
assert Thread.currentThread().getName().contains(MASTER_UPDATE_THREAD_NAME) == false :
"Expected current thread [" + Thread.currentThread() + "] to not be the master service thread. Reason: [" + reason + "]";
return true;
}
protected void runTasks(TaskInputs taskInputs) {
final String summary = taskInputs.summary;
if (!lifecycle.started()) {
logger.debug("processing [{}]: ignoring, master service not started", summary);
return;
}
logger.debug("processing [{}]: execute", summary);
final ClusterState previousClusterState = state();
if (!previousClusterState.nodes().isLocalNodeElectedMaster() && taskInputs.runOnlyWhenMaster()) {
logger.debug("failing [{}]: local node is no longer master", summary);
taskInputs.onNoLongerMaster();
return;
}
long startTimeNS = currentTimeInNanos();
TaskOutputs taskOutputs = calculateTaskOutputs(taskInputs, previousClusterState, startTimeNS);
taskOutputs.notifyFailedTasks();
if (taskOutputs.clusterStateUnchanged()) {
taskOutputs.notifySuccessfulTasksOnUnchangedClusterState();
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
logger.debug("processing [{}]: took [{}] no change in cluster state", summary, executionTime);
warnAboutSlowTaskIfNeeded(executionTime, summary);
} else {
ClusterState newClusterState = taskOutputs.newClusterState;
if (logger.isTraceEnabled()) {
logger.trace("cluster state updated, source [{}]\n{}", summary, newClusterState);
} else if (logger.isDebugEnabled()) {
logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), summary);
}
try {
ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent(summary, newClusterState, previousClusterState);
// new cluster state, notify all listeners
final DiscoveryNodes.Delta nodesDelta = clusterChangedEvent.nodesDelta();
if (nodesDelta.hasChanges() && logger.isInfoEnabled()) {
String nodeSummary = nodesDelta.shortSummary();
if (nodeSummary.length() > 0) {
logger.info("{}, reason: {}", summary, nodeSummary);
}
}
logger.debug("publishing cluster state version [{}]", newClusterState.version());
try {
clusterStatePublisher.accept(clusterChangedEvent, taskOutputs.createAckListener(threadPool, newClusterState));
} catch (Discovery.FailedToCommitClusterStateException t) {
final long version = newClusterState.version();
logger.warn(
(Supplier<?>) () -> new ParameterizedMessage(
"failing [{}]: failed to commit cluster state version [{}]", summary, version),
t);
taskOutputs.publishingFailed(t);
return;
}
taskOutputs.processedDifferentClusterState(previousClusterState, newClusterState);
try {
taskOutputs.clusterStatePublished(clusterChangedEvent);
} catch (Exception e) {
logger.error(
(Supplier<?>) () -> new ParameterizedMessage(
"exception thrown while notifying executor of new cluster state publication [{}]",
summary),
e);
}
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
logger.debug("processing [{}]: took [{}] done publishing updated cluster state (version: {}, uuid: {})", summary,
executionTime, newClusterState.version(),
newClusterState.stateUUID());
warnAboutSlowTaskIfNeeded(executionTime, summary);
} catch (Exception e) {
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
final long version = newClusterState.version();
final String stateUUID = newClusterState.stateUUID();
final String fullState = newClusterState.toString();
logger.warn(
(Supplier<?>) () -> new ParameterizedMessage(
"failed to publish updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}",
executionTime,
version,
stateUUID,
summary,
fullState),
e);
// TODO: do we want to call updateTask.onFailure here?
}
}
}
public TaskOutputs calculateTaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState, long startTimeNS) {
ClusterTasksResult<Object> clusterTasksResult = executeTasks(taskInputs, startTimeNS, previousClusterState);
ClusterState newClusterState = patchVersions(previousClusterState, clusterTasksResult);
return new TaskOutputs(taskInputs, previousClusterState, newClusterState, getNonFailedTasks(taskInputs, clusterTasksResult),
clusterTasksResult.executionResults);
}
private ClusterState patchVersions(ClusterState previousClusterState, ClusterTasksResult<?> executionResult) {
ClusterState newClusterState = executionResult.resultingState;
if (previousClusterState != newClusterState) {
// only the master controls the version numbers
Builder builder = ClusterState.builder(newClusterState).incrementVersion();
if (previousClusterState.routingTable() != newClusterState.routingTable()) {
builder.routingTable(RoutingTable.builder(newClusterState.routingTable())
.version(newClusterState.routingTable().version() + 1).build());
}
if (previousClusterState.metaData() != newClusterState.metaData()) {
builder.metaData(MetaData.builder(newClusterState.metaData()).version(newClusterState.metaData().version() + 1));
}
newClusterState = builder.build();
}
return newClusterState;
}
/**
* Submits a cluster state update task; unlike {@link #submitStateUpdateTask(String, Object, ClusterStateTaskConfig,
* ClusterStateTaskExecutor, ClusterStateTaskListener)}, submitted updates will not be batched.
*
* @param source the source of the cluster state update task
* @param updateTask the full context for the cluster state update
* task
*
*/
public <T extends ClusterStateTaskConfig & ClusterStateTaskExecutor<T> & ClusterStateTaskListener>
void submitStateUpdateTask(
String source, T updateTask) {
submitStateUpdateTask(source, updateTask, updateTask, updateTask, updateTask);
}
/**
* Submits a cluster state update task; submitted updates will be
* batched across the same instance of executor. The exact batching
* semantics depend on the underlying implementation but a rough
* guideline is that if the update task is submitted while there
* are pending update tasks for the same executor, these update
* tasks will all be executed on the executor in a single batch
*
* @param source the source of the cluster state update task
* @param task the state needed for the cluster state update task
* @param config the cluster state update task configuration
* @param executor the cluster state update task executor; tasks
* that share the same executor will be executed
* batches on this executor
* @param listener callback after the cluster state update task
* completes
* @param <T> the type of the cluster state update task state
*
*/
public <T> void submitStateUpdateTask(String source, T task,
ClusterStateTaskConfig config,
ClusterStateTaskExecutor<T> executor,
ClusterStateTaskListener listener) {
submitStateUpdateTasks(source, Collections.singletonMap(task, listener), config, executor);
}
/**
* Output created by executing a set of tasks provided as TaskInputs
*/
class TaskOutputs {
public final TaskInputs taskInputs;
public final ClusterState previousClusterState;
public final ClusterState newClusterState;
public final List<Batcher.UpdateTask> nonFailedTasks;
public final Map<Object, ClusterStateTaskExecutor.TaskResult> executionResults;
TaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState,
ClusterState newClusterState,
List<Batcher.UpdateTask> nonFailedTasks,
Map<Object, ClusterStateTaskExecutor.TaskResult> executionResults) {
this.taskInputs = taskInputs;
this.previousClusterState = previousClusterState;
this.newClusterState = newClusterState;
this.nonFailedTasks = nonFailedTasks;
this.executionResults = executionResults;
}
public void publishingFailed(Discovery.FailedToCommitClusterStateException t) {
nonFailedTasks.forEach(task -> task.listener.onFailure(task.source(), t));
}
public void processedDifferentClusterState(ClusterState previousClusterState, ClusterState newClusterState) {
nonFailedTasks.forEach(task -> task.listener.clusterStateProcessed(task.source(), previousClusterState, newClusterState));
}
public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) {
taskInputs.executor.clusterStatePublished(clusterChangedEvent);
}
public Discovery.AckListener createAckListener(ThreadPool threadPool, ClusterState newClusterState) {
ArrayList<Discovery.AckListener> ackListeners = new ArrayList<>();
//timeout straightaway, otherwise we could wait forever as the timeout thread has not started
nonFailedTasks.stream().filter(task -> task.listener instanceof AckedClusterStateTaskListener).forEach(task -> {
final AckedClusterStateTaskListener ackedListener = (AckedClusterStateTaskListener) task.listener;
if (ackedListener.ackTimeout() == null || ackedListener.ackTimeout().millis() == 0) {
ackedListener.onAckTimeout();
} else {
try {
ackListeners.add(new AckCountDownListener(ackedListener, newClusterState.version(), newClusterState.nodes(),
threadPool));
} catch (EsRejectedExecutionException ex) {
if (logger.isDebugEnabled()) {
logger.debug("Couldn't schedule timeout thread - node might be shutting down", ex);
}
//timeout straightaway, otherwise we could wait forever as the timeout thread has not started
ackedListener.onAckTimeout();
}
}
});
return new DelegetingAckListener(ackListeners);
}
public boolean clusterStateUnchanged() {
return previousClusterState == newClusterState;
}
public void notifyFailedTasks() {
// fail all tasks that have failed
for (Batcher.UpdateTask updateTask : taskInputs.updateTasks) {
assert executionResults.containsKey(updateTask.task) : "missing " + updateTask;
final ClusterStateTaskExecutor.TaskResult taskResult = executionResults.get(updateTask.task);
if (taskResult.isSuccess() == false) {
updateTask.listener.onFailure(updateTask.source(), taskResult.getFailure());
}
}
}
public void notifySuccessfulTasksOnUnchangedClusterState() {
nonFailedTasks.forEach(task -> {
if (task.listener instanceof AckedClusterStateTaskListener) {
//no need to wait for ack if nothing changed, the update can be counted as acknowledged
((AckedClusterStateTaskListener) task.listener).onAllNodesAcked(null);
}
task.listener.clusterStateProcessed(task.source(), newClusterState, newClusterState);
});
}
}
/**
* Returns the tasks that are pending.
*/
public List<PendingClusterTask> pendingTasks() {
return Arrays.stream(threadPoolExecutor.getPending()).map(pending -> {
assert pending.task instanceof SourcePrioritizedRunnable :
"thread pool executor should only use SourcePrioritizedRunnable instances but found: " + pending.task.getClass().getName();
SourcePrioritizedRunnable task = (SourcePrioritizedRunnable) pending.task;
return new PendingClusterTask(pending.insertionOrder, pending.priority, new Text(task.source()),
task.getAgeInMillis(), pending.executing);
}).collect(Collectors.toList());
}
/**
* Returns the number of currently pending tasks.
*/
public int numberOfPendingTasks() {
return threadPoolExecutor.getNumberOfPendingTasks();
}
/**
* Returns the maximum wait time for tasks in the queue
*
* @return A zero time value if the queue is empty, otherwise the time value oldest task waiting in the queue
*/
public TimeValue getMaxTaskWaitTime() {
return threadPoolExecutor.getMaxTaskWaitTime();
}
private SafeClusterStateTaskListener safe(ClusterStateTaskListener listener) {
if (listener instanceof AckedClusterStateTaskListener) {
return new SafeAckedClusterStateTaskListener((AckedClusterStateTaskListener) listener, logger);
} else {
return new SafeClusterStateTaskListener(listener, logger);
}
}
private static class SafeClusterStateTaskListener implements ClusterStateTaskListener {
private final ClusterStateTaskListener listener;
private final Logger logger;
SafeClusterStateTaskListener(ClusterStateTaskListener listener, Logger logger) {
this.listener = listener;
this.logger = logger;
}
@Override
public void onFailure(String source, Exception e) {
try {
listener.onFailure(source, e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.error(
(Supplier<?>) () -> new ParameterizedMessage(
"exception thrown by listener notifying of failure from [{}]", source), inner);
}
}
@Override
public void onNoLongerMaster(String source) {
try {
listener.onNoLongerMaster(source);
} catch (Exception e) {
logger.error(
(Supplier<?>) () -> new ParameterizedMessage(
"exception thrown by listener while notifying no longer master from [{}]", source), e);
}
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
try {
listener.clusterStateProcessed(source, oldState, newState);
} catch (Exception e) {
logger.error(
(Supplier<?>) () -> new ParameterizedMessage(
"exception thrown by listener while notifying of cluster state processed from [{}], old cluster state:\n" +
"{}\nnew cluster state:\n{}",
source, oldState, newState),
e);
}
}
}
private static class SafeAckedClusterStateTaskListener extends SafeClusterStateTaskListener implements AckedClusterStateTaskListener {
private final AckedClusterStateTaskListener listener;
private final Logger logger;
SafeAckedClusterStateTaskListener(AckedClusterStateTaskListener listener, Logger logger) {
super(listener, logger);
this.listener = listener;
this.logger = logger;
}
@Override
public boolean mustAck(DiscoveryNode discoveryNode) {
return listener.mustAck(discoveryNode);
}
@Override
public void onAllNodesAcked(@Nullable Exception e) {
try {
listener.onAllNodesAcked(e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.error("exception thrown by listener while notifying on all nodes acked", inner);
}
}
@Override
public void onAckTimeout() {
try {
listener.onAckTimeout();
} catch (Exception e) {
logger.error("exception thrown by listener while notifying on ack timeout", e);
}
}
@Override
public TimeValue ackTimeout() {
return listener.ackTimeout();
}
}
protected void warnAboutSlowTaskIfNeeded(TimeValue executionTime, String source) {
if (executionTime.getMillis() > slowTaskLoggingThreshold.getMillis()) {
logger.warn("cluster state update task [{}] took [{}] above the warn threshold of {}", source, executionTime,
slowTaskLoggingThreshold);
}
}
private static class DelegetingAckListener implements Discovery.AckListener {
private final List<Discovery.AckListener> listeners;
private DelegetingAckListener(List<Discovery.AckListener> listeners) {
this.listeners = listeners;
}
@Override
public void onNodeAck(DiscoveryNode node, @Nullable Exception e) {
for (Discovery.AckListener listener : listeners) {
listener.onNodeAck(node, e);
}
}
@Override
public void onTimeout() {
throw new UnsupportedOperationException("no timeout delegation");
}
}
private static class AckCountDownListener implements Discovery.AckListener {
private static final Logger logger = Loggers.getLogger(AckCountDownListener.class);
private final AckedClusterStateTaskListener ackedTaskListener;
private final CountDown countDown;
private final DiscoveryNodes nodes;
private final long clusterStateVersion;
private final Future<?> ackTimeoutCallback;
private Exception lastFailure;
AckCountDownListener(AckedClusterStateTaskListener ackedTaskListener, long clusterStateVersion, DiscoveryNodes nodes,
ThreadPool threadPool) {
this.ackedTaskListener = ackedTaskListener;
this.clusterStateVersion = clusterStateVersion;
this.nodes = nodes;
int countDown = 0;
for (DiscoveryNode node : nodes) {
if (ackedTaskListener.mustAck(node)) {
countDown++;
}
}
//we always wait for at least 1 node (the master)
countDown = Math.max(1, countDown);
logger.trace("expecting {} acknowledgements for cluster_state update (version: {})", countDown, clusterStateVersion);
this.countDown = new CountDown(countDown);
this.ackTimeoutCallback = threadPool.schedule(ackedTaskListener.ackTimeout(), ThreadPool.Names.GENERIC, () -> onTimeout());
}
@Override
public void onNodeAck(DiscoveryNode node, @Nullable Exception e) {
if (!ackedTaskListener.mustAck(node)) {
//we always wait for the master ack anyway
if (!node.equals(nodes.getMasterNode())) {
return;
}
}
if (e == null) {
logger.trace("ack received from node [{}], cluster_state update (version: {})", node, clusterStateVersion);
} else {
this.lastFailure = e;
logger.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"ack received from node [{}], cluster_state update (version: {})", node, clusterStateVersion),
e);
}
if (countDown.countDown()) {
logger.trace("all expected nodes acknowledged cluster_state update (version: {})", clusterStateVersion);
FutureUtils.cancel(ackTimeoutCallback);
ackedTaskListener.onAllNodesAcked(lastFailure);
}
}
@Override
public void onTimeout() {
if (countDown.fastForward()) {
logger.trace("timeout waiting for acknowledgement for cluster_state update (version: {})", clusterStateVersion);
ackedTaskListener.onAckTimeout();
}
}
}
protected ClusterTasksResult<Object> executeTasks(TaskInputs taskInputs, long startTimeNS, ClusterState previousClusterState) {
ClusterTasksResult<Object> clusterTasksResult;
try {
List<Object> inputs = taskInputs.updateTasks.stream().map(tUpdateTask -> tUpdateTask.task).collect(Collectors.toList());
clusterTasksResult = taskInputs.executor.execute(previousClusterState, inputs);
if (previousClusterState != clusterTasksResult.resultingState &&
previousClusterState.nodes().isLocalNodeElectedMaster() &&
(clusterTasksResult.resultingState.nodes().isLocalNodeElectedMaster() == false)) {
throw new AssertionError("update task submitted to MasterService cannot remove master");
}
} catch (Exception e) {
TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
if (logger.isTraceEnabled()) {
logger.trace(
(Supplier<?>) () -> new ParameterizedMessage(
"failed to execute cluster state update in [{}], state:\nversion [{}], source [{}]\n{}{}{}",
executionTime,
previousClusterState.version(),
taskInputs.summary,
previousClusterState.nodes(),
previousClusterState.routingTable(),
previousClusterState.getRoutingNodes()),
e);
}
warnAboutSlowTaskIfNeeded(executionTime, taskInputs.summary);
clusterTasksResult = ClusterTasksResult.builder()
.failures(taskInputs.updateTasks.stream().map(updateTask -> updateTask.task)::iterator, e)
.build(previousClusterState);
}
assert clusterTasksResult.executionResults != null;
assert clusterTasksResult.executionResults.size() == taskInputs.updateTasks.size()
: String.format(Locale.ROOT, "expected [%d] task result%s but was [%d]", taskInputs.updateTasks.size(),
taskInputs.updateTasks.size() == 1 ? "" : "s", clusterTasksResult.executionResults.size());
boolean assertsEnabled = false;
assert (assertsEnabled = true);
if (assertsEnabled) {
ClusterTasksResult<Object> finalClusterTasksResult = clusterTasksResult;
taskInputs.updateTasks.forEach(updateTask -> {
assert finalClusterTasksResult.executionResults.containsKey(updateTask.task) :
"missing task result for " + updateTask;
});
}
return clusterTasksResult;
}
public List<Batcher.UpdateTask> getNonFailedTasks(TaskInputs taskInputs,
ClusterTasksResult<Object> clusterTasksResult) {
return taskInputs.updateTasks.stream().filter(updateTask -> {
assert clusterTasksResult.executionResults.containsKey(updateTask.task) : "missing " + updateTask;
final ClusterStateTaskExecutor.TaskResult taskResult =
clusterTasksResult.executionResults.get(updateTask.task);
return taskResult.isSuccess();
}).collect(Collectors.toList());
}
/**
* Represents a set of tasks to be processed together with their executor
*/
protected class TaskInputs {
public final String summary;
public final List<Batcher.UpdateTask> updateTasks;
public final ClusterStateTaskExecutor<Object> executor;
TaskInputs(ClusterStateTaskExecutor<Object> executor, List<Batcher.UpdateTask> updateTasks, String summary) {
this.summary = summary;
this.executor = executor;
this.updateTasks = updateTasks;
}
public boolean runOnlyWhenMaster() {
return executor.runOnlyOnMaster();
}
public void onNoLongerMaster() {
updateTasks.forEach(task -> task.listener.onNoLongerMaster(task.source()));
}
}
/**
* Submits a batch of cluster state update tasks; submitted updates are guaranteed to be processed together,
* potentially with more tasks of the same executor.
*
* @param source the source of the cluster state update task
* @param tasks a map of update tasks and their corresponding listeners
* @param config the cluster state update task configuration
* @param executor the cluster state update task executor; tasks
* that share the same executor will be executed
* batches on this executor
* @param <T> the type of the cluster state update task state
*
*/
public <T> void submitStateUpdateTasks(final String source,
final Map<T, ClusterStateTaskListener> tasks, final ClusterStateTaskConfig config,
final ClusterStateTaskExecutor<T> executor) {
if (!lifecycle.started()) {
return;
}
try {
List<Batcher.UpdateTask> safeTasks = tasks.entrySet().stream()
.map(e -> taskBatcher.new UpdateTask(config.priority(), source, e.getKey(), safe(e.getValue()), executor))
.collect(Collectors.toList());
taskBatcher.submitTasks(safeTasks, config.timeout());
} catch (EsRejectedExecutionException e) {
// ignore cases where we are shutting down..., there is really nothing interesting
// to be done here...
if (!lifecycle.stoppedOrClosed()) {
throw e;
}
}
}
// this one is overridden in tests so we can control time
protected long currentTimeInNanos() {
return System.nanoTime();
}
}

View File

@ -41,7 +41,7 @@ import java.util.concurrent.ThreadLocalRandom;
* setting a reproducible seed. When running the Elasticsearch server * setting a reproducible seed. When running the Elasticsearch server
* process, non-reproducible sources of randomness are provided (unless * process, non-reproducible sources of randomness are provided (unless
* a setting is provided for a module that exposes a seed setting (e.g., * a setting is provided for a module that exposes a seed setting (e.g.,
* DiscoveryService#NODE_ID_SEED_SETTING)). * NodeEnvironment#NODE_ID_SEED_SETTING)).
*/ */
public final class Randomness { public final class Randomness {
private static final Method currentMethod; private static final Method currentMethod;

View File

@ -19,7 +19,8 @@
package org.elasticsearch.common.util.concurrent; package org.elasticsearch.common.util.concurrent;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplierService;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.Transports; import org.elasticsearch.transport.Transports;
@ -63,7 +64,8 @@ public abstract class BaseFuture<V> implements Future<V> {
assert timeout <= 0 || assert timeout <= 0 ||
(Transports.assertNotTransportThread(BLOCKING_OP_REASON) && (Transports.assertNotTransportThread(BLOCKING_OP_REASON) &&
ThreadPool.assertNotScheduleThread(BLOCKING_OP_REASON) && ThreadPool.assertNotScheduleThread(BLOCKING_OP_REASON) &&
ClusterService.assertNotClusterStateUpdateThread(BLOCKING_OP_REASON)); ClusterApplierService.assertNotClusterStateUpdateThread(BLOCKING_OP_REASON) &&
MasterService.assertNotMasterUpdateThread(BLOCKING_OP_REASON));
return sync.get(unit.toNanos(timeout)); return sync.get(unit.toNanos(timeout));
} }
@ -87,7 +89,8 @@ public abstract class BaseFuture<V> implements Future<V> {
public V get() throws InterruptedException, ExecutionException { public V get() throws InterruptedException, ExecutionException {
assert Transports.assertNotTransportThread(BLOCKING_OP_REASON) && assert Transports.assertNotTransportThread(BLOCKING_OP_REASON) &&
ThreadPool.assertNotScheduleThread(BLOCKING_OP_REASON) && ThreadPool.assertNotScheduleThread(BLOCKING_OP_REASON) &&
ClusterService.assertNotClusterStateUpdateThread(BLOCKING_OP_REASON); ClusterApplierService.assertNotClusterStateUpdateThread(BLOCKING_OP_REASON) &&
MasterService.assertNotMasterUpdateThread(BLOCKING_OP_REASON);
return sync.get(); return sync.get();
} }

View File

@ -21,6 +21,7 @@ package org.elasticsearch.discovery;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
@ -36,10 +37,6 @@ import java.io.IOException;
*/ */
public interface Discovery extends LifecycleComponent { public interface Discovery extends LifecycleComponent {
DiscoveryNode localNode();
String nodeDescription();
/** /**
* Another hack to solve dep injection problem..., note, this will be called before * Another hack to solve dep injection problem..., note, this will be called before
* any start is called. * any start is called.
@ -48,7 +45,7 @@ public interface Discovery extends LifecycleComponent {
/** /**
* Publish all the changes to the cluster from the master (can be called just by the master). The publish * Publish all the changes to the cluster from the master (can be called just by the master). The publish
* process should not publish this state to the master as well! (the master is sending it...). * process should apply this state to the master as well!
* *
* The {@link AckListener} allows to keep track of the ack received from nodes, and verify whether * The {@link AckListener} allows to keep track of the ack received from nodes, and verify whether
* they updated their own cluster state or not. * they updated their own cluster state or not.
@ -58,6 +55,18 @@ public interface Discovery extends LifecycleComponent {
*/ */
void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener); void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener);
/**
* Returns the initial cluster state provided by the discovery module. Used by
* {@link org.elasticsearch.cluster.service.ClusterApplierService} as initial applied state.
*/
ClusterState getInitialClusterState();
/**
* Returns latest cluster state used by the discovery module. Used by {@link org.elasticsearch.cluster.service.MasterService} to
* calculate the next prospective state to publish.
*/
ClusterState clusterState();
interface AckListener { interface AckListener {
void onNodeAck(DiscoveryNode node, @Nullable Exception e); void onNodeAck(DiscoveryNode node, @Nullable Exception e);
void onTimeout(); void onTimeout();
@ -83,8 +92,6 @@ public interface Discovery extends LifecycleComponent {
*/ */
DiscoveryStats stats(); DiscoveryStats stats();
DiscoverySettings getDiscoverySettings();
/** /**
* Triggers the first join cycle * Triggers the first join cycle
*/ */

View File

@ -19,10 +19,12 @@
package org.elasticsearch.discovery; package org.elasticsearch.discovery;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -55,8 +57,8 @@ public class DiscoveryModule {
private final Discovery discovery; private final Discovery discovery;
public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportService transportService, public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService, ClusterService clusterService, NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService, MasterService masterService,
List<DiscoveryPlugin> plugins) { ClusterApplier clusterApplier, ClusterSettings clusterSettings, List<DiscoveryPlugin> plugins) {
final UnicastHostsProvider hostsProvider; final UnicastHostsProvider hostsProvider;
Map<String, Supplier<UnicastHostsProvider>> hostProviders = new HashMap<>(); Map<String, Supplier<UnicastHostsProvider>> hostProviders = new HashMap<>();
@ -80,12 +82,13 @@ public class DiscoveryModule {
Map<String, Supplier<Discovery>> discoveryTypes = new HashMap<>(); Map<String, Supplier<Discovery>> discoveryTypes = new HashMap<>();
discoveryTypes.put("zen", discoveryTypes.put("zen",
() -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider)); () -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
discoveryTypes.put("none", () -> new NoneDiscovery(settings, clusterService, clusterService.getClusterSettings())); clusterSettings, hostsProvider));
discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, clusterService)); discoveryTypes.put("tribe", () -> new TribeDiscovery(settings, transportService, clusterApplier));
discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, transportService, clusterApplier));
for (DiscoveryPlugin plugin : plugins) { for (DiscoveryPlugin plugin : plugins) {
plugin.getDiscoveryTypes(threadPool, transportService, namedWriteableRegistry, plugin.getDiscoveryTypes(threadPool, transportService, namedWriteableRegistry,
clusterService, hostsProvider).entrySet().forEach(entry -> { masterService, clusterApplier, clusterSettings, hostsProvider).entrySet().forEach(entry -> {
if (discoveryTypes.put(entry.getKey(), entry.getValue()) != null) { if (discoveryTypes.put(entry.getKey(), entry.getValue()) != null) {
throw new IllegalArgumentException("Cannot register discovery type [" + entry.getKey() + "] twice"); throw new IllegalArgumentException("Cannot register discovery type [" + entry.getKey() + "] twice");
} }

View File

@ -1,102 +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.discovery;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.zen.ElectMasterService;
/**
* A {@link Discovery} implementation that is used by {@link org.elasticsearch.tribe.TribeService}. This implementation
* doesn't support any clustering features. Most notably {@link #startInitialJoin()} does nothing and
* {@link #publish(ClusterChangedEvent, AckListener)} is not supported.
*/
public class NoneDiscovery extends AbstractLifecycleComponent implements Discovery {
private final ClusterService clusterService;
private final DiscoverySettings discoverySettings;
@Inject
public NoneDiscovery(Settings settings, ClusterService clusterService, ClusterSettings clusterSettings) {
super(settings);
this.clusterService = clusterService;
this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
}
@Override
public DiscoveryNode localNode() {
return clusterService.localNode();
}
@Override
public String nodeDescription() {
return clusterService.getClusterName().value() + "/" + clusterService.localNode().getId();
}
@Override
public void setAllocationService(AllocationService allocationService) {
}
@Override
public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) {
throw new UnsupportedOperationException();
}
@Override
public DiscoveryStats stats() {
return null;
}
@Override
public DiscoverySettings getDiscoverySettings() {
return discoverySettings;
}
@Override
public void startInitialJoin() {
}
@Override
public int getMinimumMasterNodes() {
return ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings);
}
@Override
protected void doStart() {
}
@Override
protected void doStop() {
}
@Override
protected void doClose() {
}
}

View File

@ -0,0 +1,73 @@
/*
* 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;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.single.SingleNodeDiscovery;
import org.elasticsearch.transport.TransportService;
import static org.elasticsearch.tribe.TribeService.BLOCKS_METADATA_SETTING;
import static org.elasticsearch.tribe.TribeService.BLOCKS_WRITE_SETTING;
import static org.elasticsearch.tribe.TribeService.TRIBE_METADATA_BLOCK;
import static org.elasticsearch.tribe.TribeService.TRIBE_WRITE_BLOCK;
/**
* A {@link Discovery} implementation that is used by {@link org.elasticsearch.tribe.TribeService}. This implementation
* doesn't support any clustering features. Most notably {@link #startInitialJoin()} does nothing and
* {@link #publish(ClusterChangedEvent, AckListener)} delegates state updates directly to the
* {@link org.elasticsearch.cluster.service.ClusterApplierService}.
*/
public class TribeDiscovery extends SingleNodeDiscovery implements Discovery {
@Inject
public TribeDiscovery(Settings settings, TransportService transportService, ClusterApplier clusterApplier) {
super(settings, transportService, clusterApplier);
}
@Override
public synchronized ClusterState getInitialClusterState() {
if (initialState == null) {
ClusterBlocks.Builder clusterBlocks = ClusterBlocks.builder(); // don't add no_master / state recovery block
if (BLOCKS_WRITE_SETTING.get(settings)) {
clusterBlocks.addGlobalBlock(TRIBE_WRITE_BLOCK);
}
if (BLOCKS_METADATA_SETTING.get(settings)) {
clusterBlocks.addGlobalBlock(TRIBE_METADATA_BLOCK);
}
DiscoveryNode localNode = transportService.getLocalNode();
initialState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings))
.nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).build())
.blocks(clusterBlocks).build();
}
return initialState;
}
@Override
public synchronized void startInitialJoin() {
// no state recovery required as tribe nodes don't persist cluster state
}
}

View File

@ -19,53 +19,44 @@
package org.elasticsearch.discovery.single; package org.elasticsearch.discovery.single;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.DiscoveryStats; import org.elasticsearch.discovery.DiscoveryStats;
import org.elasticsearch.discovery.zen.PendingClusterStateStats; import org.elasticsearch.discovery.zen.PendingClusterStateStats;
import org.elasticsearch.discovery.zen.PendingClusterStatesQueue; import org.elasticsearch.transport.TransportService;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.concurrent.CountDownLatch;
import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
/** /**
* A discovery implementation where the only member of the cluster is the local node. * A discovery implementation where the only member of the cluster is the local node.
*/ */
public class SingleNodeDiscovery extends AbstractLifecycleComponent implements Discovery { public class SingleNodeDiscovery extends AbstractLifecycleComponent implements Discovery {
private final ClusterService clusterService; protected final TransportService transportService;
private final DiscoverySettings discoverySettings; private final ClusterApplier clusterApplier;
protected volatile ClusterState initialState;
private volatile ClusterState clusterState;
public SingleNodeDiscovery(final Settings settings, final ClusterService clusterService) { public SingleNodeDiscovery(final Settings settings, final TransportService transportService,
ClusterApplier clusterApplier) {
super(Objects.requireNonNull(settings)); super(Objects.requireNonNull(settings));
this.clusterService = Objects.requireNonNull(clusterService); this.transportService = Objects.requireNonNull(transportService);
final ClusterSettings clusterSettings = this.clusterApplier = clusterApplier;
Objects.requireNonNull(clusterService.getClusterSettings());
this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
}
@Override
public DiscoveryNode localNode() {
return clusterService.localNode();
}
@Override
public String nodeDescription() {
return clusterService.getClusterName().value() + "/" + clusterService.localNode().getId();
} }
@Override @Override
@ -74,8 +65,57 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
} }
@Override @Override
public void publish(final ClusterChangedEvent event, final AckListener listener) { public synchronized void publish(final ClusterChangedEvent event,
final AckListener ackListener) {
clusterState = event.state();
CountDownLatch latch = new CountDownLatch(1);
ClusterStateTaskListener listener = new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
ackListener.onNodeAck(transportService.getLocalNode(), null);
}
@Override
public void onFailure(String source, Exception e) {
latch.countDown();
ackListener.onNodeAck(transportService.getLocalNode(), e);
logger.warn(
(org.apache.logging.log4j.util.Supplier<?>) () -> new ParameterizedMessage(
"failed while applying cluster state locally [{}]",
event.source()),
e);
}
};
clusterApplier.onNewClusterState("apply-locally-on-node[" + event.source() + "]", this::clusterState, listener);
try {
latch.await();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
@Override
public synchronized ClusterState getInitialClusterState() {
if (initialState == null) {
DiscoveryNode localNode = transportService.getLocalNode();
initialState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings))
.nodes(DiscoveryNodes.builder().add(localNode)
.localNodeId(localNode.getId())
.masterNodeId(localNode.getId())
.build())
.blocks(ClusterBlocks.builder()
.addGlobalBlock(STATE_NOT_RECOVERED_BLOCK))
.build();
}
return initialState;
}
@Override
public ClusterState clusterState() {
return clusterState;
} }
@Override @Override
@ -84,41 +124,11 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
} }
@Override @Override
public DiscoverySettings getDiscoverySettings() { public synchronized void startInitialJoin() {
return discoverySettings; // apply a fresh cluster state just so that state recovery gets triggered by GatewayService
} // TODO: give discovery module control over GatewayService
clusterState = ClusterState.builder(getInitialClusterState()).build();
@Override clusterApplier.onNewClusterState("single-node-start-initial-join", this::clusterState, (source, e) -> {});
public void startInitialJoin() {
final ClusterStateTaskExecutor<DiscoveryNode> executor =
new ClusterStateTaskExecutor<DiscoveryNode>() {
@Override
public ClusterTasksResult<DiscoveryNode> execute(
final ClusterState current,
final List<DiscoveryNode> tasks) throws Exception {
assert tasks.size() == 1;
final DiscoveryNodes.Builder nodes =
DiscoveryNodes.builder(current.nodes());
// always set the local node as master, there will not be other nodes
nodes.masterNodeId(localNode().getId());
final ClusterState next =
ClusterState.builder(current).nodes(nodes).build();
final ClusterTasksResult.Builder<DiscoveryNode> result =
ClusterTasksResult.builder();
return result.successes(tasks).build(next);
}
@Override
public boolean runOnlyOnMaster() {
return false;
}
};
final ClusterStateTaskConfig config = ClusterStateTaskConfig.build(Priority.URGENT);
clusterService.submitStateUpdateTasks(
"single-node-start-initial-join",
Collections.singletonMap(localNode(), (s, e) -> {}), config, executor);
} }
@Override @Override
@ -127,8 +137,9 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
} }
@Override @Override
protected void doStart() { protected synchronized void doStart() {
initialState = getInitialClusterState();
clusterState = initialState;
} }
@Override @Override

View File

@ -28,7 +28,7 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.NotMasterException;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
@ -64,7 +64,8 @@ public class MasterFaultDetection extends FaultDetection {
} }
private final ClusterService clusterService; private final MasterService masterService;
private final java.util.function.Supplier<ClusterState> clusterStateSupplier;
private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<>(); private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<>();
private volatile MasterPinger masterPinger; private volatile MasterPinger masterPinger;
@ -78,9 +79,11 @@ public class MasterFaultDetection extends FaultDetection {
private final AtomicBoolean notifiedMasterFailure = new AtomicBoolean(); private final AtomicBoolean notifiedMasterFailure = new AtomicBoolean();
public MasterFaultDetection(Settings settings, ThreadPool threadPool, TransportService transportService, public MasterFaultDetection(Settings settings, ThreadPool threadPool, TransportService transportService,
ClusterService clusterService) { java.util.function.Supplier<ClusterState> clusterStateSupplier, MasterService masterService,
super(settings, threadPool, transportService, clusterService.getClusterName()); ClusterName clusterName) {
this.clusterService = clusterService; super(settings, threadPool, transportService, clusterName);
this.clusterStateSupplier = clusterStateSupplier;
this.masterService = masterService;
logger.debug("[master] uses ping_interval [{}], ping_timeout [{}], ping_retries [{}]", pingInterval, pingRetryTimeout, logger.debug("[master] uses ping_interval [{}], ping_timeout [{}], ping_retries [{}]", pingInterval, pingRetryTimeout,
pingRetryCount); pingRetryCount);
@ -215,7 +218,8 @@ public class MasterFaultDetection extends FaultDetection {
return; return;
} }
final MasterPingRequest request = new MasterPingRequest(clusterService.localNode(), masterToPing, clusterName); final MasterPingRequest request = new MasterPingRequest(
clusterStateSupplier.get().nodes().getLocalNode(), masterToPing, clusterName);
final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING) final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING)
.withTimeout(pingRetryTimeout).build(); .withTimeout(pingRetryTimeout).build();
transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options,
@ -323,7 +327,7 @@ public class MasterFaultDetection extends FaultDetection {
@Override @Override
public void messageReceived(final MasterPingRequest request, final TransportChannel channel) throws Exception { public void messageReceived(final MasterPingRequest request, final TransportChannel channel) throws Exception {
final DiscoveryNodes nodes = clusterService.state().nodes(); final DiscoveryNodes nodes = clusterStateSupplier.get().nodes();
// check if we are really the same master as the one we seemed to be think we are // check if we are really the same master as the one we seemed to be think we are
// this can happen if the master got "kill -9" and then another node started using the same port // this can happen if the master got "kill -9" and then another node started using the same port
if (!request.masterNode.equals(nodes.getLocalNode())) { if (!request.masterNode.equals(nodes.getLocalNode())) {
@ -348,7 +352,7 @@ public class MasterFaultDetection extends FaultDetection {
if (!nodes.isLocalNodeElectedMaster() || !nodes.nodeExists(request.sourceNode)) { if (!nodes.isLocalNodeElectedMaster() || !nodes.nodeExists(request.sourceNode)) {
logger.trace("checking ping from {} under a cluster state thread", request.sourceNode); logger.trace("checking ping from {} under a cluster state thread", request.sourceNode);
clusterService.submitStateUpdateTask("master ping (from: " + request.sourceNode + ")", new ClusterStateUpdateTask() { masterService.submitStateUpdateTask("master ping (from: " + request.sourceNode + ")", new ClusterStateUpdateTask() {
@Override @Override
public ClusterState execute(ClusterState currentState) throws Exception { public ClusterState execute(ClusterState currentState) throws Exception {

View File

@ -24,20 +24,21 @@ import org.apache.logging.log4j.util.Supplier;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskConfig;
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.NotMasterException;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.DiscoverySettings;
import java.util.ArrayList; import java.util.ArrayList;
@ -55,7 +56,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
*/ */
public class NodeJoinController extends AbstractComponent { public class NodeJoinController extends AbstractComponent {
private final ClusterService clusterService; private final MasterService masterService;
private final AllocationService allocationService; private final AllocationService allocationService;
private final ElectMasterService electMaster; private final ElectMasterService electMaster;
private final JoinTaskExecutor joinTaskExecutor = new JoinTaskExecutor(); private final JoinTaskExecutor joinTaskExecutor = new JoinTaskExecutor();
@ -65,10 +66,10 @@ public class NodeJoinController extends AbstractComponent {
private ElectionContext electionContext = null; private ElectionContext electionContext = null;
public NodeJoinController(ClusterService clusterService, AllocationService allocationService, ElectMasterService electMaster, public NodeJoinController(MasterService masterService, AllocationService allocationService, ElectMasterService electMaster,
Settings settings) { Settings settings) {
super(settings); super(settings);
this.clusterService = clusterService; this.masterService = masterService;
this.allocationService = allocationService; this.allocationService = allocationService;
this.electMaster = electMaster; this.electMaster = electMaster;
} }
@ -176,7 +177,7 @@ public class NodeJoinController extends AbstractComponent {
electionContext.addIncomingJoin(node, callback); electionContext.addIncomingJoin(node, callback);
checkPendingJoinsAndElectIfNeeded(); checkPendingJoinsAndElectIfNeeded();
} else { } else {
clusterService.submitStateUpdateTask("zen-disco-node-join", masterService.submitStateUpdateTask("zen-disco-node-join",
node, ClusterStateTaskConfig.build(Priority.URGENT), node, ClusterStateTaskConfig.build(Priority.URGENT),
joinTaskExecutor, new JoinTaskListener(callback, logger)); joinTaskExecutor, new JoinTaskListener(callback, logger));
} }
@ -279,7 +280,7 @@ public class NodeJoinController extends AbstractComponent {
tasks.put(BECOME_MASTER_TASK, (source1, e) -> {}); // noop listener, the election finished listener determines result tasks.put(BECOME_MASTER_TASK, (source1, e) -> {}); // noop listener, the election finished listener determines result
tasks.put(FINISH_ELECTION_TASK, electionFinishedListener); tasks.put(FINISH_ELECTION_TASK, electionFinishedListener);
clusterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); masterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor);
} }
public synchronized void closeAndProcessPending(String reason) { public synchronized void closeAndProcessPending(String reason) {
@ -287,7 +288,7 @@ public class NodeJoinController extends AbstractComponent {
Map<DiscoveryNode, ClusterStateTaskListener> tasks = getPendingAsTasks(); Map<DiscoveryNode, ClusterStateTaskListener> tasks = getPendingAsTasks();
final String source = "zen-disco-election-stop [" + reason + "]"; final String source = "zen-disco-election-stop [" + reason + "]";
tasks.put(FINISH_ELECTION_TASK, electionFinishedListener); tasks.put(FINISH_ELECTION_TASK, electionFinishedListener);
clusterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); masterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor);
} }
private void innerClose() { private void innerClose() {
@ -307,7 +308,7 @@ public class NodeJoinController extends AbstractComponent {
} }
private void onElectedAsMaster(ClusterState state) { private void onElectedAsMaster(ClusterState state) {
ClusterService.assertClusterStateThread(); assert MasterService.assertMasterUpdateThread();
assert state.nodes().isLocalNodeElectedMaster() : "onElectedAsMaster called but local node is not master"; assert state.nodes().isLocalNodeElectedMaster() : "onElectedAsMaster called but local node is not master";
ElectionCallback callback = getCallback(); // get under lock ElectionCallback callback = getCallback(); // get under lock
if (callback != null) { if (callback != null) {
@ -316,7 +317,7 @@ public class NodeJoinController extends AbstractComponent {
} }
private void onFailure(Throwable t) { private void onFailure(Throwable t) {
ClusterService.assertClusterStateThread(); assert MasterService.assertMasterUpdateThread();
ElectionCallback callback = getCallback(); // get under lock ElectionCallback callback = getCallback(); // get under lock
if (callback != null) { if (callback != null) {
callback.onFailure(t); callback.onFailure(t);
@ -469,6 +470,7 @@ public class NodeJoinController extends AbstractComponent {
DiscoveryNodes currentNodes = currentState.nodes(); DiscoveryNodes currentNodes = currentState.nodes();
DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(currentNodes); DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(currentNodes);
nodesBuilder.masterNodeId(currentState.nodes().getLocalNodeId()); nodesBuilder.masterNodeId(currentState.nodes().getLocalNodeId());
for (final DiscoveryNode joiningNode : joiningNodes) { for (final DiscoveryNode joiningNode : joiningNodes) {
final DiscoveryNode nodeWithSameId = nodesBuilder.get(joiningNode.getId()); final DiscoveryNode nodeWithSameId = nodesBuilder.get(joiningNode.getId());
if (nodeWithSameId != null && nodeWithSameId.equals(joiningNode) == false) { if (nodeWithSameId != null && nodeWithSameId.equals(joiningNode) == false) {
@ -486,7 +488,9 @@ public class NodeJoinController extends AbstractComponent {
// now trim any left over dead nodes - either left there when the previous master stepped down // now trim any left over dead nodes - either left there when the previous master stepped down
// or removed by us above // or removed by us above
ClusterState tmpState = ClusterState.builder(currentState).nodes(nodesBuilder).build(); ClusterState tmpState = ClusterState.builder(currentState).nodes(nodesBuilder).blocks(ClusterBlocks.builder()
.blocks(currentState.blocks())
.removeGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID)).build();
return ClusterState.builder(allocationService.deassociateDeadNodes(tmpState, false, return ClusterState.builder(allocationService.deassociateDeadNodes(tmpState, false,
"removed dead nodes on election")); "removed dead nodes on election"));
} }

View File

@ -31,28 +31,28 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskConfig;
import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.LocalClusterUpdateTask;
import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.NotMasterException;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.logging.LoggerMessageFormat;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.DiscoveryStats; import org.elasticsearch.discovery.DiscoveryStats;
@ -68,8 +68,10 @@ import org.elasticsearch.transport.TransportService;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Locale;
import java.util.Set; import java.util.Set;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -78,6 +80,7 @@ import java.util.function.Consumer;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
public class ZenDiscovery extends AbstractLifecycleComponent implements Discovery, PingContextProvider { public class ZenDiscovery extends AbstractLifecycleComponent implements Discovery, PingContextProvider {
@ -105,8 +108,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
public static final String DISCOVERY_REJOIN_ACTION_NAME = "internal:discovery/zen/rejoin"; public static final String DISCOVERY_REJOIN_ACTION_NAME = "internal:discovery/zen/rejoin";
private final TransportService transportService; private final TransportService transportService;
private final NamedWriteableRegistry namedWriteableRegistry; private final MasterService masterService;
private final ClusterService clusterService;
private AllocationService allocationService; private AllocationService allocationService;
private final ClusterName clusterName; private final ClusterName clusterName;
private final DiscoverySettings discoverySettings; private final DiscoverySettings discoverySettings;
@ -142,15 +144,19 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
private volatile NodeJoinController nodeJoinController; private volatile NodeJoinController nodeJoinController;
private volatile NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor; private volatile NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor;
private final ClusterApplier clusterApplier;
private final AtomicReference<ClusterState> state; // last committed cluster state
private final Object stateMutex = new Object();
private volatile ClusterState initialState; // set lazily when discovery layer is started
public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService, public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NamedWriteableRegistry namedWriteableRegistry, MasterService masterService, ClusterApplier clusterApplier,
ClusterService clusterService, UnicastHostsProvider hostsProvider) { ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
super(settings); super(settings);
this.clusterService = clusterService; this.masterService = masterService;
this.clusterName = clusterService.getClusterName(); this.clusterApplier = clusterApplier;
this.transportService = transportService; this.transportService = transportService;
this.namedWriteableRegistry = namedWriteableRegistry; this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
this.discoverySettings = new DiscoverySettings(settings, clusterService.getClusterSettings());
this.zenPing = newZenPing(settings, threadPool, transportService, hostsProvider); this.zenPing = newZenPing(settings, threadPool, transportService, hostsProvider);
this.electMaster = new ElectMasterService(settings); this.electMaster = new ElectMasterService(settings);
this.pingTimeout = PING_TIMEOUT_SETTING.get(settings); this.pingTimeout = PING_TIMEOUT_SETTING.get(settings);
@ -160,6 +166,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
this.maxPingsFromAnotherMaster = MAX_PINGS_FROM_ANOTHER_MASTER_SETTING.get(settings); this.maxPingsFromAnotherMaster = MAX_PINGS_FROM_ANOTHER_MASTER_SETTING.get(settings);
this.sendLeaveRequest = SEND_LEAVE_REQUEST_SETTING.get(settings); this.sendLeaveRequest = SEND_LEAVE_REQUEST_SETTING.get(settings);
this.threadPool = threadPool; this.threadPool = threadPool;
this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);
this.state = new AtomicReference<>();
this.masterElectionIgnoreNonMasters = MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING.get(settings); this.masterElectionIgnoreNonMasters = MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING.get(settings);
this.masterElectionWaitForJoinsTimeout = MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.get(settings); this.masterElectionWaitForJoinsTimeout = MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.get(settings);
@ -167,9 +175,9 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
logger.debug("using ping_timeout [{}], join.timeout [{}], master_election.ignore_non_master [{}]", logger.debug("using ping_timeout [{}], join.timeout [{}], master_election.ignore_non_master [{}]",
this.pingTimeout, joinTimeout, masterElectionIgnoreNonMasters); this.pingTimeout, joinTimeout, masterElectionIgnoreNonMasters);
clusterService.getClusterSettings().addSettingsUpdateConsumer(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, clusterSettings.addSettingsUpdateConsumer(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING,
this::handleMinimumMasterNodesChanged, (value) -> { this::handleMinimumMasterNodesChanged, (value) -> {
final ClusterState clusterState = clusterService.state(); final ClusterState clusterState = this.clusterState();
int masterNodes = clusterState.nodes().getMasterNodes().size(); int masterNodes = clusterState.nodes().getMasterNodes().size();
// the purpose of this validation is to make sure that the master doesn't step down // the purpose of this validation is to make sure that the master doesn't step down
// due to a change in master nodes, which also means that there is no way to revert // due to a change in master nodes, which also means that there is no way to revert
@ -188,9 +196,9 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
} }
}); });
this.masterFD = new MasterFaultDetection(settings, threadPool, transportService, clusterService); this.masterFD = new MasterFaultDetection(settings, threadPool, transportService, this::clusterState, masterService, clusterName);
this.masterFD.addListener(new MasterNodeFailureListener()); this.masterFD.addListener(new MasterNodeFailureListener());
this.nodesFD = new NodesFaultDetection(settings, threadPool, transportService, clusterService.getClusterName()); this.nodesFD = new NodesFaultDetection(settings, threadPool, transportService, clusterName);
this.nodesFD.addListener(new NodeFaultDetectionListener()); this.nodesFD.addListener(new NodeFaultDetectionListener());
this.publishClusterState = this.publishClusterState =
@ -198,10 +206,10 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
settings, settings,
transportService, transportService,
namedWriteableRegistry, namedWriteableRegistry,
clusterService::state, this::clusterState,
new NewPendingClusterStateListener(), new NewPendingClusterStateListener(),
discoverySettings, discoverySettings,
clusterService.getClusterName()); clusterName);
this.membership = new MembershipAction(settings, transportService, new MembershipListener()); this.membership = new MembershipAction(settings, transportService, new MembershipListener());
this.joinThreadControl = new JoinThreadControl(); this.joinThreadControl = new JoinThreadControl();
@ -222,30 +230,26 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
@Override @Override
protected void doStart() { protected void doStart() {
nodesFD.setLocalNode(clusterService.localNode()); DiscoveryNode localNode = transportService.getLocalNode();
assert localNode != null;
synchronized (stateMutex) {
initialState = getInitialClusterState();
state.set(initialState);
nodesFD.setLocalNode(localNode);
joinThreadControl.start(); joinThreadControl.start();
}
zenPing.start(this); zenPing.start(this);
this.nodeJoinController = new NodeJoinController(clusterService, allocationService, electMaster, settings); this.nodeJoinController = new NodeJoinController(masterService, allocationService, electMaster, settings);
this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, electMaster, this::submitRejoin, logger); this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, electMaster, this::submitRejoin, logger);
} }
@Override @Override
public void startInitialJoin() { public void startInitialJoin() {
// start the join thread from a cluster state update. See {@link JoinThreadControl} for details. // start the join thread from a cluster state update. See {@link JoinThreadControl} for details.
clusterService.submitStateUpdateTask("initial_join", new LocalClusterUpdateTask() { synchronized (stateMutex) {
// do the join on a different thread, the caller of this method waits for 30s anyhow till it is discovered
@Override
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
// do the join on a different thread, the DiscoveryService waits for 30s anyhow till it is discovered
joinThreadControl.startNewThreadIfNotRunning(); joinThreadControl.startNewThreadIfNotRunning();
return unchanged();
} }
@Override
public void onFailure(String source, @org.elasticsearch.common.Nullable Exception e) {
logger.warn("failed to start initial join process", e);
}
});
} }
@Override @Override
@ -286,58 +290,122 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
IOUtils.close(masterFD, nodesFD); IOUtils.close(masterFD, nodesFD);
} }
@Override
public DiscoveryNode localNode() {
return clusterService.localNode();
}
@Override
public String nodeDescription() {
return clusterName.value() + "/" + clusterService.localNode().getId();
}
/** start of {@link PingContextProvider } implementation */ /** start of {@link PingContextProvider } implementation */
@Override @Override
public DiscoveryNodes nodes() { public DiscoveryNodes nodes() {
return clusterService.state().nodes(); return clusterState().nodes();
} }
@Override @Override
public ClusterState clusterState() { public ClusterState clusterState() {
return clusterService.state(); ClusterState clusterState = state.get();
assert clusterState != null : "accessing cluster state before it is set";
return clusterState;
} }
/** end of {@link PingContextProvider } implementation */ /** end of {@link PingContextProvider } implementation */
@Override @Override
public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) { public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) {
if (!clusterChangedEvent.state().getNodes().isLocalNodeElectedMaster()) { ClusterState newState = clusterChangedEvent.state();
throw new IllegalStateException("Shouldn't publish state when not master"); assert newState.getNodes().isLocalNodeElectedMaster() : "Shouldn't publish state when not master " + clusterChangedEvent.source();
// state got changed locally (maybe because another master published to us)
if (clusterChangedEvent.previousState() != this.state.get()) {
throw new FailedToCommitClusterStateException("state was mutated while calculating new CS update");
} }
publishClusterState.pendingStatesQueue().addPending(newState);
try { try {
publishClusterState.publish(clusterChangedEvent, electMaster.minimumMasterNodes(), ackListener); publishClusterState.publish(clusterChangedEvent, electMaster.minimumMasterNodes(), ackListener);
} catch (FailedToCommitClusterStateException t) { } catch (FailedToCommitClusterStateException t) {
// cluster service logs a WARN message // 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()); logger.debug("failed to publish cluster state version [{}] (not enough nodes acknowledged, min master nodes [{}])",
submitRejoin("zen-disco-failed-to-publish"); newState.version(), electMaster.minimumMasterNodes());
synchronized (stateMutex) {
publishClusterState.pendingStatesQueue().failAllStatesAndClear(
new ElasticsearchException("failed to publish cluster state"));
rejoin("zen-disco-failed-to-publish");
}
throw t; throw t;
} }
// update the set of nodes to ping after the new cluster state has been published final DiscoveryNode localNode = newState.getNodes().getLocalNode();
nodesFD.updateNodesAndPing(clusterChangedEvent.state()); final CountDownLatch latch = new CountDownLatch(1);
final AtomicBoolean processedOrFailed = new AtomicBoolean();
publishClusterState.pendingStatesQueue().markAsCommitted(newState.stateUUID(),
new PendingClusterStatesQueue.StateProcessedListener() {
@Override
public void onNewClusterStateProcessed() {
processedOrFailed.set(true);
latch.countDown();
ackListener.onNodeAck(localNode, null);
}
// clean the pending cluster queue - we are currently master, so any pending cluster state should be failed @Override
// note that we also clean the queue on master failure (see handleMasterGone) but a delayed cluster state publish public void onNewClusterStateFailed(Exception e) {
// from a stale master can still make it in the queue during the election (but not be committed) processedOrFailed.set(true);
publishClusterState.pendingStatesQueue().failAllStatesAndClear(new ElasticsearchException("elected as master")); latch.countDown();
ackListener.onNodeAck(localNode, e);
logger.warn(
(org.apache.logging.log4j.util.Supplier<?>) () -> new ParameterizedMessage(
"failed while applying cluster state locally [{}]",
clusterChangedEvent.source()),
e);
}
});
synchronized (stateMutex) {
if (clusterChangedEvent.previousState() != this.state.get()) {
throw new FailedToCommitClusterStateException("local state was mutated while CS update was published to other nodes");
}
boolean sentToApplier = processNextCommittedClusterState("master " + newState.nodes().getMasterNode() +
" committed version [" + newState.version() + "] source [" + clusterChangedEvent.source() + "]");
if (sentToApplier == false && processedOrFailed.get() == false) {
assert false : "cluster state published locally neither processed nor failed: " + newState;
logger.warn("cluster state with version [{}] that is published locally has neither been processed nor failed",
newState.version());
return;
}
}
// indefinitely wait for cluster state to be applied locally
try {
latch.await();
} catch (InterruptedException e) {
logger.debug(
(org.apache.logging.log4j.util.Supplier<?>) () -> new ParameterizedMessage(
"interrupted while applying cluster state locally [{}]",
clusterChangedEvent.source()),
e);
Thread.currentThread().interrupt();
}
}
@Override
public synchronized ClusterState getInitialClusterState() {
if (initialState == null) {
assert state.get() == null;
DiscoveryNode localNode = transportService.getLocalNode();
assert localNode != null;
initialState = ClusterState.builder(clusterName)
.blocks(ClusterBlocks.builder()
.addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)
.addGlobalBlock(discoverySettings.getNoMasterBlock()))
.nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()))
.build();
}
return initialState;
} }
/** /**
* Gets the current set of nodes involved in the node fault detection. * Gets the current set of nodes involved in the node fault detection.
* NB: for testing purposes * NB: for testing purposes
*/ */
public Set<DiscoveryNode> getFaultDetectionNodes() { Set<DiscoveryNode> getFaultDetectionNodes() {
return nodesFD.getNodes(); return nodesFD.getNodes();
} }
@ -347,7 +415,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
return new DiscoveryStats(queueStats); return new DiscoveryStats(queueStats);
} }
@Override
public DiscoverySettings getDiscoverySettings() { public DiscoverySettings getDiscoverySettings() {
return discoverySettings; return discoverySettings;
} }
@ -391,24 +458,26 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
return; return;
} }
if (clusterService.localNode().equals(masterNode)) { if (transportService.getLocalNode().equals(masterNode)) {
final int requiredJoins = Math.max(0, electMaster.minimumMasterNodes() - 1); // we count as one final int requiredJoins = Math.max(0, electMaster.minimumMasterNodes() - 1); // we count as one
logger.debug("elected as master, waiting for incoming joins ([{}] needed)", requiredJoins); logger.debug("elected as master, waiting for incoming joins ([{}] needed)", requiredJoins);
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, masterElectionWaitForJoinsTimeout, nodeJoinController.waitToBeElectedAsMaster(requiredJoins, masterElectionWaitForJoinsTimeout,
new NodeJoinController.ElectionCallback() { new NodeJoinController.ElectionCallback() {
@Override @Override
public void onElectedAsMaster(ClusterState state) { public void onElectedAsMaster(ClusterState state) {
synchronized (stateMutex) {
joinThreadControl.markThreadAsDone(currentThread); joinThreadControl.markThreadAsDone(currentThread);
// we only starts nodesFD if we are master (it may be that we received a cluster state while pinging) }
nodesFD.updateNodesAndPing(state); // start the nodes FD
} }
@Override @Override
public void onFailure(Throwable t) { public void onFailure(Throwable t) {
logger.trace("failed while waiting for nodes to join, rejoining", t); logger.trace("failed while waiting for nodes to join, rejoining", t);
synchronized (stateMutex) {
joinThreadControl.markThreadAsDoneAndStartNew(currentThread); joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
} }
} }
}
); );
} else { } else {
@ -418,41 +487,25 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
// send join request // send join request
final boolean success = joinElectedMaster(masterNode); final boolean success = joinElectedMaster(masterNode);
// finalize join through the cluster state update thread synchronized (stateMutex) {
final DiscoveryNode finalMasterNode = masterNode; if (success) {
clusterService.submitStateUpdateTask("finalize_join (" + masterNode + ")", new LocalClusterUpdateTask() { DiscoveryNode currentMasterNode = this.clusterState().getNodes().getMasterNode();
@Override if (currentMasterNode == null) {
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
if (!success) {
// failed to join. Try again...
joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
return unchanged();
}
if (currentState.getNodes().getMasterNode() == null) {
// Post 1.3.0, the master should publish a new cluster state before acking our join request. we now should have // Post 1.3.0, the master should publish a new cluster state before acking our join request. we now should have
// a valid master. // a valid master.
logger.debug("no master node is set, despite of join request completing. retrying pings."); logger.debug("no master node is set, despite of join request completing. retrying pings.");
joinThreadControl.markThreadAsDoneAndStartNew(currentThread); joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
return unchanged(); } else if (currentMasterNode.equals(masterNode) == false) {
// update cluster state
joinThreadControl.stopRunningThreadAndRejoin("master_switched_while_finalizing_join");
} }
if (!currentState.getNodes().getMasterNode().equals(finalMasterNode)) {
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 #processNextPendingClusterState }
// when the first cluster state arrives.
joinThreadControl.markThreadAsDone(currentThread); joinThreadControl.markThreadAsDone(currentThread);
return unchanged(); } else {
} // failed to join. Try again...
@Override
public void onFailure(String source, @Nullable Exception e) {
logger.error("unexpected error while trying to finalize cluster join", e);
joinThreadControl.markThreadAsDoneAndStartNew(currentThread); joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
} }
}); }
} }
} }
@ -473,7 +526,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
while (true) { while (true) {
try { try {
logger.trace("joining master {}", masterNode); logger.trace("joining master {}", masterNode);
membership.sendJoinRequestBlocking(masterNode, clusterService.localNode(), joinTimeout); membership.sendJoinRequestBlocking(masterNode, transportService.getLocalNode(), joinTimeout);
return true; return true;
} catch (Exception e) { } catch (Exception e) {
final Throwable unwrap = ExceptionsHelper.unwrapCause(e); final Throwable unwrap = ExceptionsHelper.unwrapCause(e);
@ -503,18 +556,16 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
} }
private void submitRejoin(String source) { private void submitRejoin(String source) {
clusterService.submitStateUpdateTask(source, new LocalClusterUpdateTask(Priority.IMMEDIATE) { synchronized (stateMutex) {
@Override rejoin(source);
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) { }
return rejoin(currentState, source);
} }
@Override // visible for testing
public void onFailure(String source, Exception e) { void setState(ClusterState clusterState) {
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e); synchronized (stateMutex) {
state.set(clusterState);
} }
});
} }
// visible for testing // visible for testing
@ -611,7 +662,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
} }
private void removeNode(final DiscoveryNode node, final String source, final String reason) { private void removeNode(final DiscoveryNode node, final String source, final String reason) {
clusterService.submitStateUpdateTask( masterService.submitStateUpdateTask(
source + "(" + node + "), reason(" + reason + ")", source + "(" + node + "), reason(" + reason + ")",
new NodeRemovalClusterStateTaskExecutor.Task(node, reason), new NodeRemovalClusterStateTaskExecutor.Task(node, reason),
ClusterStateTaskConfig.build(Priority.IMMEDIATE), ClusterStateTaskConfig.build(Priority.IMMEDIATE),
@ -654,32 +705,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
// We only set the new value. If the master doesn't see enough nodes it will revoke it's mastership. // We only set the new value. If the master doesn't see enough nodes it will revoke it's mastership.
return; return;
} }
clusterService.submitStateUpdateTask("zen-disco-min-master-nodes-changed", new LocalClusterUpdateTask(Priority.IMMEDIATE) { synchronized (stateMutex) {
@Override
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) {
// check if we have enough master nodes, if not, we need to move into joining the cluster again // check if we have enough master nodes, if not, we need to move into joining the cluster again
if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) { if (!electMaster.hasEnoughMasterNodes(state.get().nodes())) {
return rejoin(currentState, "not enough master nodes on change of minimum_master_nodes from [" + prevMinimumMasterNode + "] to [" + minimumMasterNodes + "]"); rejoin("not enough master nodes on change of minimum_master_nodes from [" + prevMinimumMasterNode + "] to [" + minimumMasterNodes + "]");
} }
return unchanged();
} }
@Override
public void onNoLongerMaster(String source) {
// ignoring (already logged)
}
@Override
public void onFailure(String source, Exception e) {
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
electMaster.logMinimumMasterNodesWarningIfNecessary(oldState, newState);
}
});
} }
private void handleMasterGone(final DiscoveryNode masterNode, final Throwable cause, final String reason) { private void handleMasterGone(final DiscoveryNode masterNode, final Throwable cause, final String reason) {
@ -694,59 +725,71 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
logger.info((Supplier<?>) () -> new ParameterizedMessage("master_left [{}], reason [{}]", masterNode, reason), cause); logger.info((Supplier<?>) () -> new ParameterizedMessage("master_left [{}], reason [{}]", masterNode, reason), cause);
clusterService.submitStateUpdateTask("master_failed (" + masterNode + ")", new LocalClusterUpdateTask(Priority.IMMEDIATE) { synchronized (stateMutex) {
if (localNodeMaster() == false && masterNode.equals(state.get().nodes().getMasterNode())) {
@Override
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) {
if (!masterNode.equals(currentState.nodes().getMasterNode())) {
// master got switched on us, no need to send anything
return unchanged();
}
// flush any pending cluster states from old master, so it will not be set as master again // flush any pending cluster states from old master, so it will not be set as master again
publishClusterState.pendingStatesQueue().failAllStatesAndClear(new ElasticsearchException("master left [{}]", reason)); publishClusterState.pendingStatesQueue().failAllStatesAndClear(new ElasticsearchException("master left [{}]", reason));
rejoin("master left (reason = " + reason + ")");
return rejoin(currentState, "master left (reason = " + reason + ")"); }
}
} }
@Override // return true if state has been sent to applier
public void onFailure(String source, Exception e) { boolean processNextCommittedClusterState(String reason) {
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e); assert Thread.holdsLock(stateMutex);
}
});
}
void processNextPendingClusterState(String reason) {
clusterService.submitStateUpdateTask("zen-disco-receive(from master [" + reason + "])", new LocalClusterUpdateTask(Priority.URGENT) {
ClusterState newClusterState = null;
@Override
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) {
newClusterState = publishClusterState.pendingStatesQueue().getNextClusterStateToProcess();
final ClusterState newClusterState = publishClusterState.pendingStatesQueue().getNextClusterStateToProcess();
final ClusterState currentState = state.get();
final ClusterState adaptedNewClusterState;
// all pending states have been processed // all pending states have been processed
if (newClusterState == null) { if (newClusterState == null) {
return unchanged(); return false;
} }
assert newClusterState.nodes().getMasterNode() != null : "received a cluster state without a master"; assert newClusterState.nodes().getMasterNode() != null : "received a cluster state without a master";
assert !newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block"; assert !newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block";
if (currentState.nodes().isLocalNodeElectedMaster()) { if (currentState.nodes().isLocalNodeElectedMaster() && newClusterState.nodes().isLocalNodeElectedMaster() == false) {
return handleAnotherMaster(currentState, newClusterState.nodes().getMasterNode(), newClusterState.version(), "via a new cluster state"); handleAnotherMaster(currentState, newClusterState.nodes().getMasterNode(), newClusterState.version(), "via a new cluster state");
return false;
} }
try {
if (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) { if (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) {
return unchanged(); String message = String.format(
Locale.ROOT,
"rejecting cluster state version [%d] uuid [%s] received from [%s]",
newClusterState.version(),
newClusterState.stateUUID(),
newClusterState.nodes().getMasterNodeId()
);
throw new IllegalStateException(message);
} }
} catch (Exception e) {
try {
publishClusterState.pendingStatesQueue().markAsFailed(newClusterState, e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected exception while failing [{}]", reason), inner);
}
return false;
}
if (newClusterState.nodes().isLocalNodeElectedMaster()) {
// update the set of nodes to ping
nodesFD.updateNodesAndPing(newClusterState);
} else {
// check to see that we monitor the correct master of the cluster
if (masterFD.masterNode() == null || !masterFD.masterNode().equals(newClusterState.nodes().getMasterNode())) {
masterFD.restart(newClusterState.nodes().getMasterNode(), "new cluster state received and we are monitoring the wrong master [" + masterFD.masterNode() + "]");
}
}
if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) { 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 // 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().getMasterNodeId()); logger.debug("got first state from fresh master [{}]", newClusterState.nodes().getMasterNodeId());
return newState(newClusterState); adaptedNewClusterState = newClusterState;
} } else if (newClusterState.nodes().isLocalNodeElectedMaster() == false) {
// some optimizations to make sure we keep old objects where possible // some optimizations to make sure we keep old objects where possible
ClusterState.Builder builder = ClusterState.builder(newClusterState); ClusterState.Builder builder = ClusterState.builder(newClusterState);
@ -773,37 +816,44 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
builder.metaData(metaDataBuilder); builder.metaData(metaDataBuilder);
} }
return newState(builder.build()); adaptedNewClusterState = builder.build();
} else {
adaptedNewClusterState = newClusterState;
} }
@Override if (currentState == adaptedNewClusterState) {
public void onFailure(String source, Exception e) { return false;
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
if (newClusterState != null) {
try {
publishClusterState.pendingStatesQueue().markAsFailed(newClusterState, e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected exception while failing [{}]", source), inner);
}
}
} }
state.set(adaptedNewClusterState);
clusterApplier.onNewClusterState("apply cluster state (from master [" + reason + "])",
this::clusterState,
new ClusterStateTaskListener() {
@Override @Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
try { try {
if (newClusterState != null) {
// check to see that we monitor the correct master of the cluster
if (masterFD.masterNode() == null || !masterFD.masterNode().equals(newClusterState.nodes().getMasterNode())) {
masterFD.restart(newClusterState.nodes().getMasterNode(), "new cluster state received and we are monitoring the wrong master [" + masterFD.masterNode() + "]");
}
publishClusterState.pendingStatesQueue().markAsProcessed(newClusterState); publishClusterState.pendingStatesQueue().markAsProcessed(newClusterState);
}
} catch (Exception e) { } catch (Exception e) {
onFailure(source, e); onFailure(source, e);
} }
} }
@Override
public void onFailure(String source, Exception e) {
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure applying [{}]", reason), e);
try {
// TODO: use cluster state uuid instead of full cluster state so that we don't keep reference to CS around
// for too long.
publishClusterState.pendingStatesQueue().markAsFailed(newClusterState, e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected exception while failing [{}]", reason), inner);
}
}
}); });
return true;
} }
/** /**
@ -888,13 +938,13 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
logger.trace("full ping responses:{}", sb); logger.trace("full ping responses:{}", sb);
} }
final DiscoveryNode localNode = clusterService.localNode(); final DiscoveryNode localNode = transportService.getLocalNode();
// add our selves // add our selves
assert fullPingResponses.stream().map(ZenPing.PingResponse::node) assert fullPingResponses.stream().map(ZenPing.PingResponse::node)
.filter(n -> n.equals(localNode)).findAny().isPresent() == false; .filter(n -> n.equals(localNode)).findAny().isPresent() == false;
fullPingResponses.add(new ZenPing.PingResponse(localNode, null, clusterService.state())); fullPingResponses.add(new ZenPing.PingResponse(localNode, null, this.clusterState()));
// filter responses // filter responses
final List<ZenPing.PingResponse> pingResponses = filterPingResponses(fullPingResponses, masterElectionIgnoreNonMasters, logger); final List<ZenPing.PingResponse> pingResponses = filterPingResponses(fullPingResponses, masterElectionIgnoreNonMasters, logger);
@ -956,10 +1006,9 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
return pingResponses; return pingResponses;
} }
protected ClusterStateTaskExecutor.ClusterTasksResult<LocalClusterUpdateTask> rejoin(ClusterState clusterState, String reason) { protected void rejoin(String reason) {
assert Thread.holdsLock(stateMutex);
// *** called from within an cluster state update task *** // ClusterState clusterState = state.get();
assert Thread.currentThread().getName().contains(ClusterService.UPDATE_THREAD_NAME);
logger.warn("{}, current nodes: {}", reason, clusterState.nodes()); logger.warn("{}, current nodes: {}", reason, clusterState.nodes());
nodesFD.stop(); nodesFD.stop();
@ -968,44 +1017,54 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
// TODO: do we want to force a new thread if we actively removed the master? this is to give a full pinging cycle // TODO: do we want to force a new thread if we actively removed the master? this is to give a full pinging cycle
// before a decision is made. // before a decision is made.
joinThreadControl.startNewThreadIfNotRunning(); joinThreadControl.startNewThreadIfNotRunning();
return LocalClusterUpdateTask.noMaster();
if (clusterState.nodes().getMasterNodeId() != null) {
// remove block if it already exists before adding new one
assert clusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock().id()) == false :
"NO_MASTER_BLOCK should only be added by ZenDiscovery";
ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(clusterState.blocks())
.addGlobalBlock(discoverySettings.getNoMasterBlock())
.build();
DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder(clusterState.nodes()).masterNodeId(null).build();
clusterState = ClusterState.builder(clusterState)
.blocks(clusterBlocks)
.nodes(discoveryNodes)
.build();
state.set(clusterState);
clusterApplier.onNewClusterState(reason, this::clusterState, (source, e) -> {}); // don't wait for state to be applied
}
} }
private boolean localNodeMaster() { private boolean localNodeMaster() {
return nodes().isLocalNodeElectedMaster(); return nodes().isLocalNodeElectedMaster();
} }
private ClusterStateTaskExecutor.ClusterTasksResult handleAnotherMaster(ClusterState localClusterState, final DiscoveryNode otherMaster, long otherClusterStateVersion, String reason) { private void handleAnotherMaster(ClusterState localClusterState, final DiscoveryNode otherMaster, long otherClusterStateVersion, String reason) {
assert localClusterState.nodes().isLocalNodeElectedMaster() : "handleAnotherMaster called but current node is not a master"; assert localClusterState.nodes().isLocalNodeElectedMaster() : "handleAnotherMaster called but current node is not a master";
assert Thread.currentThread().getName().contains(ClusterService.UPDATE_THREAD_NAME) : "not called from the cluster state update thread"; assert Thread.holdsLock(stateMutex);
if (otherClusterStateVersion > localClusterState.version()) { if (otherClusterStateVersion > localClusterState.version()) {
return rejoin(localClusterState, "zen-disco-discovered another master with a new cluster_state [" + otherMaster + "][" + reason + "]"); rejoin("zen-disco-discovered another master with a new cluster_state [" + otherMaster + "][" + reason + "]");
} else { } else {
// TODO: do this outside mutex
logger.warn("discovered [{}] which is also master but with an older cluster_state, telling [{}] to rejoin the cluster ([{}])", otherMaster, otherMaster, reason); logger.warn("discovered [{}] which is also master but with an older cluster_state, telling [{}] to rejoin the cluster ([{}])", otherMaster, otherMaster, reason);
// spawn to a background thread to not do blocking operations on the cluster state thread try {
threadPool.generic().execute(new AbstractRunnable() {
@Override
public void onFailure(Exception e) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), e);
}
@Override
protected void doRun() throws Exception {
// make sure we're connected to this node (connect to node does nothing if we're already connected) // make sure we're connected to this node (connect to node does nothing if we're already connected)
// since the network connections are asymmetric, it may be that we received a state but have disconnected from the node // since the network connections are asymmetric, it may be that we received a state but have disconnected from the node
// in the past (after a master failure, for example) // in the past (after a master failure, for example)
transportService.connectToNode(otherMaster); transportService.connectToNode(otherMaster);
transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, new RejoinClusterRequest(localNode().getId()), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, new RejoinClusterRequest(localClusterState.nodes().getLocalNodeId()), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@Override @Override
public void handleException(TransportException exp) { public void handleException(TransportException exp) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), exp); logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), exp);
} }
}); });
} catch (Exception e) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), e);
} }
});
return LocalClusterUpdateTask.unchanged();
} }
} }
@ -1033,14 +1092,16 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
@Override @Override
public void onNewClusterState(String reason) { public void onNewClusterState(String reason) {
processNextPendingClusterState(reason); synchronized (stateMutex) {
processNextCommittedClusterState(reason);
}
} }
} }
private class MembershipListener implements MembershipAction.MembershipListener { private class MembershipListener implements MembershipAction.MembershipListener {
@Override @Override
public void onJoin(DiscoveryNode node, MembershipAction.JoinCallback callback) { public void onJoin(DiscoveryNode node, MembershipAction.JoinCallback callback) {
handleJoinRequest(node, clusterService.state(), callback); handleJoinRequest(node, ZenDiscovery.this.clusterState(), callback);
} }
@Override @Override
@ -1072,23 +1133,13 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
return; return;
} }
logger.debug("got a ping from another master {}. resolving who should rejoin. current ping count: [{}]", pingRequest.masterNode(), pingsWhileMaster.get()); logger.debug("got a ping from another master {}. resolving who should rejoin. current ping count: [{}]", pingRequest.masterNode(), pingsWhileMaster.get());
clusterService.submitStateUpdateTask("ping from another master", new LocalClusterUpdateTask(Priority.IMMEDIATE) { synchronized (stateMutex) {
ClusterState currentState = state.get();
@Override
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
if (currentState.nodes().isLocalNodeElectedMaster()) { if (currentState.nodes().isLocalNodeElectedMaster()) {
pingsWhileMaster.set(0); pingsWhileMaster.set(0);
return handleAnotherMaster(currentState, pingRequest.masterNode(), pingRequest.clusterStateVersion(), "node fd ping"); handleAnotherMaster(currentState, pingRequest.masterNode(), pingRequest.clusterStateVersion(), "node fd ping");
} else {
return unchanged();
} }
} }
@Override
public void onFailure(String source, Exception e) {
logger.debug("unexpected error during cluster state update task after pings from another master", e);
}
});
} }
} }
@ -1127,23 +1178,14 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
class RejoinClusterRequestHandler implements TransportRequestHandler<RejoinClusterRequest> { class RejoinClusterRequestHandler implements TransportRequestHandler<RejoinClusterRequest> {
@Override @Override
public void messageReceived(final RejoinClusterRequest request, final TransportChannel channel) throws Exception { public void messageReceived(final RejoinClusterRequest request, final TransportChannel channel) throws Exception {
clusterService.submitStateUpdateTask("received a request to rejoin the cluster from [" + request.fromNodeId + "]", new LocalClusterUpdateTask(Priority.IMMEDIATE) {
@Override
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) {
try { try {
channel.sendResponse(TransportResponse.Empty.INSTANCE); channel.sendResponse(TransportResponse.Empty.INSTANCE);
} catch (Exception e) { } catch (Exception e) {
logger.warn("failed to send response on rejoin cluster request handling", e); logger.warn("failed to send response on rejoin cluster request handling", e);
} }
return rejoin(currentState, "received a request to rejoin the cluster from [" + request.fromNodeId + "]"); synchronized (stateMutex) {
rejoin("received a request to rejoin the cluster from [" + request.fromNodeId + "]");
} }
@Override
public void onFailure(String source, Exception e) {
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
}
});
} }
} }
@ -1169,15 +1211,15 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
} }
/** cleans any running joining thread and calls {@link #rejoin} */ /** cleans any running joining thread and calls {@link #rejoin} */
public ClusterStateTaskExecutor.ClusterTasksResult<LocalClusterUpdateTask> stopRunningThreadAndRejoin(ClusterState clusterState, String reason) { public void stopRunningThreadAndRejoin(String reason) {
ClusterService.assertClusterStateThread(); assert Thread.holdsLock(stateMutex);
currentJoinThread.set(null); currentJoinThread.set(null);
return rejoin(clusterState, reason); rejoin(reason);
} }
/** starts a new joining thread if there is no currently active one and join thread controlling is started */ /** starts a new joining thread if there is no currently active one and join thread controlling is started */
public void startNewThreadIfNotRunning() { public void startNewThreadIfNotRunning() {
ClusterService.assertClusterStateThread(); assert Thread.holdsLock(stateMutex);
if (joinThreadActive()) { if (joinThreadActive()) {
return; return;
} }
@ -1210,7 +1252,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
* If the given thread is not the currently running join thread, the command is ignored. * If the given thread is not the currently running join thread, the command is ignored.
*/ */
public void markThreadAsDoneAndStartNew(Thread joinThread) { public void markThreadAsDoneAndStartNew(Thread joinThread) {
ClusterService.assertClusterStateThread(); assert Thread.holdsLock(stateMutex);
if (!markThreadAsDone(joinThread)) { if (!markThreadAsDone(joinThread)) {
return; return;
} }
@ -1219,7 +1261,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
/** marks the given joinThread as completed. Returns false if the supplied thread is not the currently active join thread */ /** marks the given joinThread as completed. Returns false if the supplied thread is not the currently active join thread */
public boolean markThreadAsDone(Thread joinThread) { public boolean markThreadAsDone(Thread joinThread) {
ClusterService.assertClusterStateThread(); assert Thread.holdsLock(stateMutex);
return currentJoinThread.compareAndSet(joinThread, null); return currentJoinThread.compareAndSet(joinThread, null);
} }

View File

@ -122,9 +122,6 @@ public class GatewayService extends AbstractLifecycleComponent implements Cluste
// TODO: change me once the minimum_master_nodes is changed too // TODO: change me once the minimum_master_nodes is changed too
recoverAfterMasterNodes = settings.getAsInt("discovery.zen.minimum_master_nodes", -1); recoverAfterMasterNodes = settings.getAsInt("discovery.zen.minimum_master_nodes", -1);
} }
// Add the not recovered as initial state block, we don't allow anything until
this.clusterService.addInitialStateBlock(STATE_NOT_RECOVERED_BLOCK);
} }
@Override @Override

View File

@ -26,7 +26,6 @@ import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.LocalClusterUpdateTask;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexRoutingTable;
@ -280,26 +279,20 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe
return; return;
} }
clusterService.submitStateUpdateTask("indices_store ([" + shardId + "] active fully on other nodes)", new LocalClusterUpdateTask() { clusterService.getClusterApplierService().runOnApplierThread("indices_store ([" + shardId + "] active fully on other nodes)",
@Override currentState -> {
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
if (clusterStateVersion != currentState.getVersion()) { if (clusterStateVersion != currentState.getVersion()) {
logger.trace("not deleting shard {}, the update task state version[{}] is not equal to cluster state before shard active api call [{}]", shardId, currentState.getVersion(), clusterStateVersion); logger.trace("not deleting shard {}, the update task state version[{}] is not equal to cluster state before shard active api call [{}]", shardId, currentState.getVersion(), clusterStateVersion);
return unchanged(); return;
} }
try { try {
indicesService.deleteShardStore("no longer used", shardId, currentState); indicesService.deleteShardStore("no longer used", shardId, currentState);
} catch (Exception ex) { } catch (Exception ex) {
logger.debug((Supplier<?>) () -> new ParameterizedMessage("{} failed to delete unallocated shard, ignoring", shardId), ex); logger.debug((Supplier<?>) () -> new ParameterizedMessage("{} failed to delete unallocated shard, ignoring", shardId), ex);
} }
return unchanged(); },
} (source, e) -> logger.error((Supplier<?>) () -> new ParameterizedMessage("{} unexpected error during deletion of unallocated shard", shardId), e)
);
@Override
public void onFailure(String source, Exception e) {
logger.error((Supplier<?>) () -> new ParameterizedMessage("{} unexpected error during deletion of unallocated shard", shardId), e);
}
});
} }
} }

View File

@ -260,7 +260,7 @@ public class Node implements Closeable {
nodeEnvironment = new NodeEnvironment(tmpSettings, environment); nodeEnvironment = new NodeEnvironment(tmpSettings, environment);
resourcesToClose.add(nodeEnvironment); resourcesToClose.add(nodeEnvironment);
} catch (IOException ex) { } catch (IOException ex) {
throw new IllegalStateException("Failed to created node environment", ex); throw new IllegalStateException("Failed to create node environment", ex);
} }
final boolean hadPredefinedNodeName = NODE_NAME_SETTING.exists(tmpSettings); final boolean hadPredefinedNodeName = NODE_NAME_SETTING.exists(tmpSettings);
Logger logger = Loggers.getLogger(Node.class, tmpSettings); Logger logger = Loggers.getLogger(Node.class, tmpSettings);
@ -336,8 +336,7 @@ public class Node implements Closeable {
resourcesToClose.add(resourceWatcherService); resourcesToClose.add(resourceWatcherService);
final NetworkService networkService = new NetworkService(settings, final NetworkService networkService = new NetworkService(settings,
getCustomNameResolvers(pluginsService.filterPlugins(DiscoveryPlugin.class))); getCustomNameResolvers(pluginsService.filterPlugins(DiscoveryPlugin.class)));
final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool, final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool);
localNodeFactory::getNode);
clusterService.addListener(scriptModule.getScriptService()); clusterService.addListener(scriptModule.getScriptService());
resourcesToClose.add(clusterService); resourcesToClose.add(clusterService);
final IngestService ingestService = new IngestService(settings, threadPool, this.environment, final IngestService ingestService = new IngestService(settings, threadPool, this.environment,
@ -428,12 +427,13 @@ public class Node implements Closeable {
}; };
httpServerTransport = null; httpServerTransport = null;
} }
final DiscoveryModule discoveryModule = new DiscoveryModule(this.settings, threadPool, transportService,
namedWriteableRegistry, networkService, clusterService, pluginsService.filterPlugins(DiscoveryPlugin.class)); final DiscoveryModule discoveryModule = new DiscoveryModule(this.settings, threadPool, transportService, namedWriteableRegistry,
networkService, clusterService.getMasterService(), clusterService.getClusterApplierService(),
clusterService.getClusterSettings(), pluginsService.filterPlugins(DiscoveryPlugin.class));
NodeService nodeService = new NodeService(settings, threadPool, monitorService, discoveryModule.getDiscovery(), NodeService nodeService = new NodeService(settings, threadPool, monitorService, discoveryModule.getDiscovery(),
transportService, indicesService, pluginsService, circuitBreakerService, scriptModule.getScriptService(), transportService, indicesService, pluginsService, circuitBreakerService, scriptModule.getScriptService(),
httpServerTransport, ingestService, clusterService, settingsModule.getSettingsFilter()); httpServerTransport, ingestService, clusterService, settingsModule.getSettingsFilter());
modules.add(b -> { modules.add(b -> {
b.bind(NodeService.class).toInstance(nodeService); b.bind(NodeService.class).toInstance(nodeService);
b.bind(NamedXContentRegistry.class).toInstance(xContentRegistry); b.bind(NamedXContentRegistry.class).toInstance(xContentRegistry);
@ -662,9 +662,7 @@ public class Node implements Closeable {
injector.getInstance(ResourceWatcherService.class).start(); injector.getInstance(ResourceWatcherService.class).start();
injector.getInstance(GatewayService.class).start(); injector.getInstance(GatewayService.class).start();
Discovery discovery = injector.getInstance(Discovery.class); Discovery discovery = injector.getInstance(Discovery.class);
clusterService.setDiscoverySettings(discovery.getDiscoverySettings()); clusterService.getMasterService().setClusterStatePublisher(discovery::publish);
clusterService.addInitialStateBlock(discovery.getDiscoverySettings().getNoMasterBlock());
clusterService.setClusterStatePublisher(discovery::publish);
// start before the cluster service since it adds/removes initial Cluster state blocks // start before the cluster service since it adds/removes initial Cluster state blocks
final TribeService tribeService = injector.getInstance(TribeService.class); final TribeService tribeService = injector.getInstance(TribeService.class);
@ -674,18 +672,20 @@ public class Node implements Closeable {
TransportService transportService = injector.getInstance(TransportService.class); TransportService transportService = injector.getInstance(TransportService.class);
transportService.getTaskManager().setTaskResultsService(injector.getInstance(TaskResultsService.class)); transportService.getTaskManager().setTaskResultsService(injector.getInstance(TaskResultsService.class));
transportService.start(); transportService.start();
assert localNodeFactory.getNode() != null;
assert transportService.getLocalNode().equals(localNodeFactory.getNode())
: "transportService has a different local node than the factory provided";
validateNodeBeforeAcceptingRequests(settings, transportService.boundAddress(), pluginsService.filterPlugins(Plugin.class).stream() validateNodeBeforeAcceptingRequests(settings, transportService.boundAddress(), pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.getBootstrapChecks().stream()).collect(Collectors.toList())); .flatMap(p -> p.getBootstrapChecks().stream()).collect(Collectors.toList()));
clusterService.addStateApplier(transportService.getTaskManager()); clusterService.addStateApplier(transportService.getTaskManager());
clusterService.getMasterService().setClusterStateSupplier(discovery::clusterState);
clusterService.getClusterApplierService().setInitialState(discovery.getInitialClusterState());
// start after transport service so the local disco is known
clusterService.start(); clusterService.start();
assert localNodeFactory.getNode() != null; discovery.start();
assert transportService.getLocalNode().equals(localNodeFactory.getNode())
: "transportService has a different local node than the factory provided";
assert clusterService.localNode().equals(localNodeFactory.getNode()) assert clusterService.localNode().equals(localNodeFactory.getNode())
: "clusterService has a different local node than the factory provided"; : "clusterService has a different local node than the factory provided";
// start after cluster service so the local disco is known
discovery.start();
transportService.acceptIncomingRequests(); transportService.acceptIncomingRequests();
discovery.startInitialJoin(); discovery.startInitialJoin();
// tribe nodes don't have a master so we shouldn't register an observer s // tribe nodes don't have a master so we shouldn't register an observer s

View File

@ -82,7 +82,7 @@ public class NodeService extends AbstractComponent implements Closeable {
public NodeInfo info(boolean settings, boolean os, boolean process, boolean jvm, boolean threadPool, public NodeInfo info(boolean settings, boolean os, boolean process, boolean jvm, boolean threadPool,
boolean transport, boolean http, boolean plugin, boolean ingest, boolean indices) { boolean transport, boolean http, boolean plugin, boolean ingest, boolean indices) {
return new NodeInfo(Version.CURRENT, Build.CURRENT, discovery.localNode(), return new NodeInfo(Version.CURRENT, Build.CURRENT, transportService.getLocalNode(),
settings ? settingsFilter.filter(this.settings) : null, settings ? settingsFilter.filter(this.settings) : null,
os ? monitorService.osService().info() : null, os ? monitorService.osService().info() : null,
process ? monitorService.processService().info() : null, process ? monitorService.processService().info() : null,
@ -101,7 +101,7 @@ public class NodeService extends AbstractComponent implements Closeable {
boolean script, boolean discoveryStats, boolean ingest) { boolean script, boolean discoveryStats, boolean ingest) {
// for indices stats we want to include previous allocated shards stats as well (it will // for indices stats we want to include previous allocated shards stats as well (it will
// only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats) // only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats)
return new NodeStats(discovery.localNode(), System.currentTimeMillis(), return new NodeStats(transportService.getLocalNode(), System.currentTimeMillis(),
indices.anySet() ? indicesService.stats(true, indices) : null, indices.anySet() ? indicesService.stats(true, indices) : null,
os ? monitorService.osService().stats() : null, os ? monitorService.osService().stats() : null,
process ? monitorService.processService().stats() : null, process ? monitorService.processService().stats() : null,

View File

@ -23,13 +23,14 @@ import java.util.Collections;
import java.util.Map; import java.util.Map;
import java.util.function.Supplier; import java.util.function.Supplier;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.zen.UnicastHostsProvider; import org.elasticsearch.discovery.zen.UnicastHostsProvider;
import org.elasticsearch.discovery.zen.ZenPing;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
@ -57,12 +58,17 @@ public interface DiscoveryPlugin {
* *
* @param threadPool Use to schedule ping actions * @param threadPool Use to schedule ping actions
* @param transportService Use to communicate with other nodes * @param transportService Use to communicate with other nodes
* @param clusterService Use to find current nodes in the cluster * @param masterService Use to submit cluster state update tasks
* @param clusterApplier Use to locally apply cluster state updates
* @param clusterSettings Use to get cluster settings
* @param hostsProvider Use to find configured hosts which should be pinged for initial discovery * @param hostsProvider Use to find configured hosts which should be pinged for initial discovery
*/ */
default Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, default Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) { MasterService masterService,
ClusterApplier clusterApplier,
ClusterSettings clusterSettings,
UnicastHostsProvider hostsProvider) {
return Collections.emptyMap(); return Collections.emptyMap();
} }

View File

@ -148,7 +148,8 @@ public class RepositoriesService extends AbstractComponent implements ClusterSta
@Override @Override
public boolean mustAck(DiscoveryNode discoveryNode) { public boolean mustAck(DiscoveryNode discoveryNode) {
return discoveryNode.isMasterNode(); // repository is created on both master and data nodes
return discoveryNode.isMasterNode() || discoveryNode.isDataNode();
} }
}); });
} }
@ -198,8 +199,8 @@ public class RepositoriesService extends AbstractComponent implements ClusterSta
@Override @Override
public boolean mustAck(DiscoveryNode discoveryNode) { public boolean mustAck(DiscoveryNode discoveryNode) {
// Since operation occurs only on masters, it's enough that only master-eligible nodes acked // repository was created on both master and data nodes
return discoveryNode.isMasterNode(); return discoveryNode.isMasterNode() || discoveryNode.isDataNode();
} }
}); });
} }

View File

@ -46,7 +46,7 @@ public class RestAliasAction extends AbstractCatAction {
@Override @Override
protected RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) { protected RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) {
final GetAliasesRequest getAliasesRequest = request.hasParam("alias") ? final GetAliasesRequest getAliasesRequest = request.hasParam("alias") ?
new GetAliasesRequest(request.param("alias")) : new GetAliasesRequest(Strings.commaDelimitedListToStringArray(request.param("alias"))) :
new GetAliasesRequest(); new GetAliasesRequest();
getAliasesRequest.local(request.paramAsBoolean("local", getAliasesRequest.local())); getAliasesRequest.local(request.paramAsBoolean("local", getAliasesRequest.local()));

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.aggregations.metrics.geocentroid; package org.elasticsearch.search.aggregations.metrics.geocentroid;
import org.apache.lucene.geo.GeoEncodingUtils; import org.apache.lucene.geo.GeoEncodingUtils;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
@ -36,8 +37,8 @@ import java.util.Objects;
* Serialization and merge logic for {@link GeoCentroidAggregator}. * Serialization and merge logic for {@link GeoCentroidAggregator}.
*/ */
public class InternalGeoCentroid extends InternalAggregation implements GeoCentroid { public class InternalGeoCentroid extends InternalAggregation implements GeoCentroid {
protected final GeoPoint centroid; private final GeoPoint centroid;
protected final long count; private final long count;
public static long encodeLatLon(double lat, double lon) { public static long encodeLatLon(double lat, double lon) {
return (Integer.toUnsignedLong(GeoEncodingUtils.encodeLatitude(lat)) << 32) | Integer.toUnsignedLong(GeoEncodingUtils.encodeLongitude(lon)); return (Integer.toUnsignedLong(GeoEncodingUtils.encodeLatitude(lat)) << 32) | Integer.toUnsignedLong(GeoEncodingUtils.encodeLongitude(lon));
@ -136,6 +137,8 @@ public class InternalGeoCentroid extends InternalAggregation implements GeoCentr
return centroid.lat(); return centroid.lat();
case "lon": case "lon":
return centroid.lon(); return centroid.lon();
case "count":
return count;
default: default:
throw new IllegalArgumentException("Found unknown path element [" + coordinate + "] in [" + getName() + "]"); throw new IllegalArgumentException("Found unknown path element [" + coordinate + "] in [" + getName() + "]");
} }
@ -145,14 +148,27 @@ public class InternalGeoCentroid extends InternalAggregation implements GeoCentr
} }
static class Fields { static class Fields {
static final String CENTROID = "location"; static final ParseField CENTROID = new ParseField("location");
static final ParseField CENTROID_LAT = new ParseField("lat");
static final ParseField CENTROID_LON = new ParseField("lon");
static final ParseField COUNT = new ParseField("count");
} }
@Override @Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
if (centroid != null) { return renderXContent(builder, params, centroid, count);
builder.startObject(Fields.CENTROID).field("lat", centroid.lat()).field("lon", centroid.lon()).endObject();
} }
static XContentBuilder renderXContent(XContentBuilder builder, Params params, GeoPoint centroid, long count) throws IOException {
if (centroid != null) {
builder.startObject(Fields.CENTROID.getPreferredName());
{
builder.field(Fields.CENTROID_LAT.getPreferredName(), centroid.lat());
builder.field(Fields.CENTROID_LON.getPreferredName(), centroid.lon());
}
builder.endObject();
}
builder.field(Fields.COUNT.getPreferredName(), count);
return builder; return builder;
} }

View File

@ -43,6 +43,7 @@ import org.elasticsearch.common.Priority;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.hash.MurmurHash3; import org.elasticsearch.common.hash.MurmurHash3;
import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
@ -73,6 +74,7 @@ import java.util.EnumSet;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
import java.util.function.Function; import java.util.function.Function;
@ -129,7 +131,7 @@ public class TribeService extends AbstractLifecycleComponent {
if (!NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.exists(settings)) { if (!NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.exists(settings)) {
sb.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), nodesSettings.size()); sb.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), nodesSettings.size());
} }
sb.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "none"); // a tribe node should not use zen discovery sb.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "tribe"); // there is a special discovery implementation for tribe
// nothing is going to be discovered, since no master will be elected // nothing is going to be discovered, since no master will be elected
sb.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0); sb.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
if (sb.get("cluster.name") == null) { if (sb.get("cluster.name") == null) {
@ -230,16 +232,6 @@ public class TribeService extends AbstractLifecycleComponent {
this.blockIndicesMetadata = BLOCKS_METADATA_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY); this.blockIndicesMetadata = BLOCKS_METADATA_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
this.blockIndicesRead = BLOCKS_READ_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY); this.blockIndicesRead = BLOCKS_READ_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
this.blockIndicesWrite = BLOCKS_WRITE_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY); this.blockIndicesWrite = BLOCKS_WRITE_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
if (!nodes.isEmpty()) {
if (BLOCKS_WRITE_SETTING.get(settings)) {
clusterService.addInitialStateBlock(TRIBE_WRITE_BLOCK);
}
if (BLOCKS_METADATA_SETTING.get(settings)) {
clusterService.addInitialStateBlock(TRIBE_METADATA_BLOCK);
}
}
this.onConflict = ON_CONFLICT_SETTING.get(settings); this.onConflict = ON_CONFLICT_SETTING.get(settings);
} }
@ -290,12 +282,7 @@ public class TribeService extends AbstractLifecycleComponent {
@Override @Override
protected void doStart() { protected void doStart() {
if (nodes.isEmpty() == false) {
// remove the initial election / recovery blocks since we are not going to have a
// master elected in this single tribe node local "cluster"
clusterService.removeInitialStateBlock(DiscoverySettings.NO_MASTER_BLOCK_ID);
clusterService.removeInitialStateBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK);
}
} }
public void startNodes() { public void startNodes() {
@ -516,7 +503,10 @@ public class TribeService extends AbstractLifecycleComponent {
final List<Node> tribeClientNodes = TribeService.this.nodes; final List<Node> tribeClientNodes = TribeService.this.nodes;
Map<String, MetaData.Custom> mergedCustomMetaDataMap = mergeChangedCustomMetaData(changedCustomMetaDataTypeSet, Map<String, MetaData.Custom> mergedCustomMetaDataMap = mergeChangedCustomMetaData(changedCustomMetaDataTypeSet,
customMetaDataType -> tribeClientNodes.stream() customMetaDataType -> tribeClientNodes.stream()
.map(TribeService::getClusterService).map(ClusterService::state) .map(TribeService::getClusterService)
// cluster service might not have initial state yet (as tribeClientNodes are started after main node)
.filter(cs -> cs.lifecycleState() == Lifecycle.State.STARTED)
.map(ClusterService::state)
.map(ClusterState::metaData) .map(ClusterState::metaData)
.map(clusterMetaData -> ((MetaData.Custom) clusterMetaData.custom(customMetaDataType))) .map(clusterMetaData -> ((MetaData.Custom) clusterMetaData.custom(customMetaDataType)))
.filter(custom1 -> custom1 != null && custom1 instanceof MergableCustomMetaData) .filter(custom1 -> custom1 != null && custom1 instanceof MergableCustomMetaData)

View File

@ -40,6 +40,7 @@ import org.elasticsearch.test.ESTestCase;
import java.io.IOException; import java.io.IOException;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -255,4 +256,39 @@ public class BulkRequestTests extends ESTestCase {
assertEquals(1, request.sourceAsMap().size()); assertEquals(1, request.sourceAsMap().size());
assertEquals("value", request.sourceAsMap().get("field")); assertEquals("value", request.sourceAsMap().get("field"));
} }
public void testToValidateUpsertRequestAndVersionInBulkRequest() throws IOException {
XContentType xContentType = XContentType.SMILE;
BytesReference data;
try (BytesStreamOutput out = new BytesStreamOutput()) {
try (XContentBuilder builder = XContentFactory.contentBuilder(xContentType, out)) {
builder.startObject();
builder.startObject("update");
builder.field("_index", "index");
builder.field("_type", "type");
builder.field("_id", "id");
builder.field("_version", 1L);
builder.endObject();
builder.endObject();
}
out.write(xContentType.xContent().streamSeparator());
try(XContentBuilder builder = XContentFactory.contentBuilder(xContentType, out)) {
builder.startObject();
builder.field("doc", "{}");
Map<String,Object> values = new HashMap<>();
values.put("_version", 2L);
values.put("_index", "index");
values.put("_type", "type");
builder.field("upsert", values);
builder.endObject();
}
out.write(xContentType.xContent().streamSeparator());
data = out.bytes();
}
BulkRequest bulkRequest = new BulkRequest();
bulkRequest.add(data, null, null, xContentType);
assertThat(bulkRequest.validate().validationErrors(), contains("can't provide both upsert request and a version",
"can't provide version in upsert request"));
}
} }

View File

@ -59,9 +59,11 @@ import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonList; import static java.util.Collections.singletonList;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.common.xcontent.XContentHelper.toXContent; import static org.elasticsearch.common.xcontent.XContentHelper.toXContent;
import static org.elasticsearch.common.xcontent.XContentHelper.update;
import static org.elasticsearch.script.MockScriptEngine.mockInlineScript; import static org.elasticsearch.script.MockScriptEngine.mockInlineScript;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.arrayContaining;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -485,4 +487,19 @@ public class UpdateRequestTests extends ESTestCase {
BytesReference finalBytes = toXContent(parsedUpdateRequest, xContentType, humanReadable); BytesReference finalBytes = toXContent(parsedUpdateRequest, xContentType, humanReadable);
assertToXContentEquivalent(originalBytes, finalBytes, xContentType); assertToXContentEquivalent(originalBytes, finalBytes, xContentType);
} }
public void testToValidateUpsertRequestAndVersion() {
UpdateRequest updateRequest = new UpdateRequest("index", "type", "id");
updateRequest.version(1L);
updateRequest.doc("{}", XContentType.JSON);
updateRequest.upsert(new IndexRequest("index","type", "id"));
assertThat(updateRequest.validate().validationErrors(), contains("can't provide both upsert request and a version"));
}
public void testToValidateUpsertRequestWithVersion() {
UpdateRequest updateRequest = new UpdateRequest("index", "type", "id");
updateRequest.doc("{}", XContentType.JSON);
updateRequest.upsert(new IndexRequest("index", "type", "1").version(1L));
assertThat(updateRequest.validate().validationErrors(), contains("can't provide version in upsert request"));
}
} }

View File

@ -19,8 +19,6 @@
package org.elasticsearch.cluster; package org.elasticsearch.cluster;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.ShardAllocationDecision; import org.elasticsearch.cluster.routing.allocation.ShardAllocationDecision;
@ -41,7 +39,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocatio
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.inject.ModuleTestCase; import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.IndexScopedSettings;
@ -61,8 +58,7 @@ import java.util.function.Supplier;
public class ClusterModuleTests extends ModuleTestCase { public class ClusterModuleTests extends ModuleTestCase {
private ClusterService clusterService = new ClusterService(Settings.EMPTY, private ClusterService clusterService = new ClusterService(Settings.EMPTY,
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null, () -> new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null);
new DiscoveryNode(UUIDs.randomBase64UUID(), buildNewFakeTransportAddress(), Version.CURRENT));
static class FakeAllocationDecider extends AllocationDecider { static class FakeAllocationDecider extends AllocationDecider {
protected FakeAllocationDecider(Settings settings) { protected FakeAllocationDecider(Settings settings) {
super(settings); super(settings);

View File

@ -29,7 +29,6 @@ import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.LocalClusterUpdateTask;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
@ -130,18 +129,11 @@ public class ClusterStateHealthTests extends ESTestCase {
}); });
logger.info("--> submit task to restore master"); logger.info("--> submit task to restore master");
clusterService.submitStateUpdateTask("restore master", new LocalClusterUpdateTask() { ClusterState currentState = clusterService.getClusterApplierService().state();
@Override clusterService.getClusterApplierService().onNewClusterState("restore master",
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception { () -> ClusterState.builder(currentState)
return newState(ClusterState.builder(currentState).nodes( .nodes(DiscoveryNodes.builder(currentState.nodes()).masterNodeId(currentState.nodes().getLocalNodeId())).build(),
DiscoveryNodes.builder(currentState.nodes()).masterNodeId(currentState.nodes().getLocalNodeId())).build()); (source, e) -> {});
}
@Override
public void onFailure(String source, Exception e) {
logger.warn("unexpected failure", e);
}
});
logger.info("--> waiting for listener to be called and cluster state being blocked"); logger.info("--> waiting for listener to be called and cluster state being blocked");
listenerCalled.await(); listenerCalled.await();

View File

@ -470,7 +470,7 @@ public class DelayedAllocationServiceTests extends ESAllocationTestCase {
} }
@Override @Override
protected void assertClusterStateThread() { protected void assertClusterOrMasterStateThread() {
// do not check this in the unit tests // do not check this in the unit tests
} }

View File

@ -0,0 +1,427 @@
/*
* 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.service;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.LocalNodeMasterListener;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.MockLogAppender;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static org.elasticsearch.test.ClusterServiceUtils.setState;
import static org.hamcrest.Matchers.is;
public class ClusterApplierServiceTests extends ESTestCase {
protected static ThreadPool threadPool;
protected TimedClusterApplierService clusterApplierService;
@BeforeClass
public static void createThreadPool() {
threadPool = new TestThreadPool(ClusterApplierServiceTests.class.getName());
}
@AfterClass
public static void stopThreadPool() {
if (threadPool != null) {
threadPool.shutdownNow();
threadPool = null;
}
}
@Before
public void setUp() throws Exception {
super.setUp();
clusterApplierService = createTimedClusterService(true);
}
@After
public void tearDown() throws Exception {
clusterApplierService.close();
super.tearDown();
}
TimedClusterApplierService createTimedClusterService(boolean makeMaster) throws InterruptedException {
DiscoveryNode localNode = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(),
emptySet(), Version.CURRENT);
TimedClusterApplierService timedClusterApplierService = new TimedClusterApplierService(Settings.builder().put("cluster.name",
"ClusterApplierServiceTests").build(), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
threadPool);
timedClusterApplierService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
@Override
public void connectToNodes(DiscoveryNodes discoveryNodes) {
// skip
}
@Override
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
// skip
}
});
timedClusterApplierService.setInitialState(ClusterState.builder(new ClusterName("ClusterApplierServiceTests"))
.nodes(DiscoveryNodes.builder()
.add(localNode)
.localNodeId(localNode.getId())
.masterNodeId(makeMaster ? localNode.getId() : null))
.blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build());
timedClusterApplierService.start();
return timedClusterApplierService;
}
@TestLogging("org.elasticsearch.cluster.service:TRACE") // To ensure that we log cluster state events on TRACE level
public void testClusterStateUpdateLogging() throws Exception {
MockLogAppender mockAppender = new MockLogAppender();
mockAppender.start();
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test1",
clusterApplierService.getClass().getName(),
Level.DEBUG,
"*processing [test1]: took [1s] no change in cluster state"));
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test2",
clusterApplierService.getClass().getName(),
Level.TRACE,
"*failed to execute cluster state applier in [2s]*"));
Logger clusterLogger = Loggers.getLogger("org.elasticsearch.cluster.service");
Loggers.addAppender(clusterLogger, mockAppender);
try {
final CountDownLatch latch = new CountDownLatch(3);
clusterApplierService.currentTimeOverride = System.nanoTime();
clusterApplierService.runOnApplierThread("test1",
currentState -> clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos(),
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
clusterApplierService.runOnApplierThread("test2",
currentState -> {
clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(2).nanos();
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
},
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
fail();
}
@Override
public void onFailure(String source, Exception e) {
latch.countDown();
}
});
// Additional update task to make sure all previous logging made it to the loggerName
// We don't check logging for this on since there is no guarantee that it will occur before our check
clusterApplierService.runOnApplierThread("test3",
currentState -> {},
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
latch.await();
} finally {
Loggers.removeAppender(clusterLogger, mockAppender);
mockAppender.stop();
}
mockAppender.assertAllExpectationsMatched();
}
@TestLogging("org.elasticsearch.cluster.service:WARN") // To ensure that we log cluster state events on WARN level
public void testLongClusterStateUpdateLogging() throws Exception {
MockLogAppender mockAppender = new MockLogAppender();
mockAppender.start();
mockAppender.addExpectation(
new MockLogAppender.UnseenEventExpectation(
"test1 shouldn't see because setting is too low",
clusterApplierService.getClass().getName(),
Level.WARN,
"*cluster state applier task [test1] took [*] above the warn threshold of *"));
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test2",
clusterApplierService.getClass().getName(),
Level.WARN,
"*cluster state applier task [test2] took [32s] above the warn threshold of *"));
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test4",
clusterApplierService.getClass().getName(),
Level.WARN,
"*cluster state applier task [test3] took [34s] above the warn threshold of *"));
Logger clusterLogger = Loggers.getLogger("org.elasticsearch.cluster.service");
Loggers.addAppender(clusterLogger, mockAppender);
try {
final CountDownLatch latch = new CountDownLatch(4);
final CountDownLatch processedFirstTask = new CountDownLatch(1);
clusterApplierService.currentTimeOverride = System.nanoTime();
clusterApplierService.runOnApplierThread("test1",
currentState -> clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos(),
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
processedFirstTask.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
processedFirstTask.await();
clusterApplierService.runOnApplierThread("test2",
currentState -> {
clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(32).nanos();
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
},
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
fail();
}
@Override
public void onFailure(String source, Exception e) {
latch.countDown();
}
});
clusterApplierService.runOnApplierThread("test3",
currentState -> clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(34).nanos(),
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
// Additional update task to make sure all previous logging made it to the loggerName
// We don't check logging for this on since there is no guarantee that it will occur before our check
clusterApplierService.runOnApplierThread("test4",
currentState -> {},
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
latch.await();
} finally {
Loggers.removeAppender(clusterLogger, mockAppender);
mockAppender.stop();
}
mockAppender.assertAllExpectationsMatched();
}
public void testLocalNodeMasterListenerCallbacks() throws Exception {
TimedClusterApplierService timedClusterApplierService = createTimedClusterService(false);
AtomicBoolean isMaster = new AtomicBoolean();
timedClusterApplierService.addLocalNodeMasterListener(new LocalNodeMasterListener() {
@Override
public void onMaster() {
isMaster.set(true);
}
@Override
public void offMaster() {
isMaster.set(false);
}
@Override
public String executorName() {
return ThreadPool.Names.SAME;
}
});
ClusterState state = timedClusterApplierService.state();
DiscoveryNodes nodes = state.nodes();
DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(nodes).masterNodeId(nodes.getLocalNodeId());
state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).nodes(nodesBuilder).build();
setState(timedClusterApplierService, state);
assertThat(isMaster.get(), is(true));
nodes = state.nodes();
nodesBuilder = DiscoveryNodes.builder(nodes).masterNodeId(null);
state = ClusterState.builder(state).blocks(ClusterBlocks.builder().addGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_WRITES))
.nodes(nodesBuilder).build();
setState(timedClusterApplierService, state);
assertThat(isMaster.get(), is(false));
nodesBuilder = DiscoveryNodes.builder(nodes).masterNodeId(nodes.getLocalNodeId());
state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).nodes(nodesBuilder).build();
setState(timedClusterApplierService, state);
assertThat(isMaster.get(), is(true));
timedClusterApplierService.close();
}
public void testClusterStateApplierCantSampleClusterState() throws InterruptedException {
AtomicReference<Throwable> error = new AtomicReference<>();
AtomicBoolean applierCalled = new AtomicBoolean();
clusterApplierService.addStateApplier(event -> {
try {
applierCalled.set(true);
clusterApplierService.state();
error.set(new AssertionError("successfully sampled state"));
} catch (AssertionError e) {
if (e.getMessage().contains("should not be called by a cluster state applier") == false) {
error.set(e);
}
}
});
CountDownLatch latch = new CountDownLatch(1);
clusterApplierService.onNewClusterState("test", () -> ClusterState.builder(clusterApplierService.state()).build(),
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
error.compareAndSet(null, e);
}
}
);
latch.await();
assertNull(error.get());
assertTrue(applierCalled.get());
}
public void testClusterStateApplierCanCreateAnObserver() throws InterruptedException {
AtomicReference<Throwable> error = new AtomicReference<>();
AtomicBoolean applierCalled = new AtomicBoolean();
clusterApplierService.addStateApplier(event -> {
try {
applierCalled.set(true);
ClusterStateObserver observer = new ClusterStateObserver(event.state(),
clusterApplierService, null, logger, threadPool.getThreadContext());
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
}
@Override
public void onClusterServiceClose() {
}
@Override
public void onTimeout(TimeValue timeout) {
}
});
} catch (AssertionError e) {
error.set(e);
}
});
CountDownLatch latch = new CountDownLatch(1);
clusterApplierService.onNewClusterState("test", () -> ClusterState.builder(clusterApplierService.state()).build(),
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
error.compareAndSet(null, e);
}
});
latch.await();
assertNull(error.get());
assertTrue(applierCalled.get());
}
static class TimedClusterApplierService extends ClusterApplierService {
public volatile Long currentTimeOverride = null;
TimedClusterApplierService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) {
super(settings, clusterSettings, threadPool);
}
@Override
protected long currentTimeInNanos() {
if (currentTimeOverride != null) {
return currentTimeOverride;
}
return super.currentTimeInNanos();
}
}
}

View File

@ -382,7 +382,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
// The tasks can be re-ordered, so we need to check out-of-order // The tasks can be re-ordered, so we need to check out-of-order
Set<String> controlSources = new HashSet<>(Arrays.asList("1", "2", "3", "4", "5", "6", "7", "8", "9", "10")); Set<String> controlSources = new HashSet<>(Arrays.asList("1", "2", "3", "4", "5", "6", "7", "8", "9", "10"));
List<PendingClusterTask> pendingClusterTasks = clusterService.pendingTasks(); List<PendingClusterTask> pendingClusterTasks = clusterService.getMasterService().pendingTasks();
assertThat(pendingClusterTasks.size(), greaterThanOrEqualTo(10)); assertThat(pendingClusterTasks.size(), greaterThanOrEqualTo(10));
assertThat(pendingClusterTasks.get(0).getSource().string(), equalTo("1")); assertThat(pendingClusterTasks.get(0).getSource().string(), equalTo("1"));
assertThat(pendingClusterTasks.get(0).isExecuting(), equalTo(true)); assertThat(pendingClusterTasks.get(0).isExecuting(), equalTo(true));
@ -404,7 +404,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
invoked2.await(); invoked2.await();
// whenever we test for no tasks, we need to awaitBusy since this is a live node // whenever we test for no tasks, we need to awaitBusy since this is a live node
assertTrue(awaitBusy(() -> clusterService.pendingTasks().isEmpty())); assertTrue(awaitBusy(() -> clusterService.getMasterService().pendingTasks().isEmpty()));
waitNoPendingTasksOnAll(); waitNoPendingTasksOnAll();
final CountDownLatch block2 = new CountDownLatch(1); final CountDownLatch block2 = new CountDownLatch(1);
@ -444,7 +444,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
} }
Thread.sleep(100); Thread.sleep(100);
pendingClusterTasks = clusterService.pendingTasks(); pendingClusterTasks = clusterService.getMasterService().pendingTasks();
assertThat(pendingClusterTasks.size(), greaterThanOrEqualTo(5)); assertThat(pendingClusterTasks.size(), greaterThanOrEqualTo(5));
controlSources = new HashSet<>(Arrays.asList("1", "2", "3", "4", "5")); controlSources = new HashSet<>(Arrays.asList("1", "2", "3", "4", "5"));
for (PendingClusterTask task : pendingClusterTasks) { for (PendingClusterTask task : pendingClusterTasks) {

View File

@ -20,32 +20,25 @@ package org.elasticsearch.cluster.service;
import org.apache.logging.log4j.Level; import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskConfig;
import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.LocalClusterUpdateTask; import org.elasticsearch.cluster.LocalClusterUpdateTask;
import org.elasticsearch.cluster.LocalNodeMasterListener;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.BaseFuture; import org.elasticsearch.common.util.concurrent.BaseFuture;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.MockLogAppender; import org.elasticsearch.test.MockLogAppender;
import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.junit.annotations.TestLogging;
@ -75,21 +68,19 @@ import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.emptyMap; import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet; import static java.util.Collections.emptySet;
import static org.elasticsearch.test.ClusterServiceUtils.setState;
import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasKey;
import static org.hamcrest.Matchers.is;
public class ClusterServiceTests extends ESTestCase { public class MasterServiceTests extends ESTestCase {
static ThreadPool threadPool; private static ThreadPool threadPool;
TimedClusterService clusterService; private TimedMasterService masterService;
@BeforeClass @BeforeClass
public static void createThreadPool() { public static void createThreadPool() {
threadPool = new TestThreadPool(ClusterServiceTests.class.getName()); threadPool = new TestThreadPool(MasterServiceTests.class.getName());
} }
@AfterClass @AfterClass
@ -103,48 +94,35 @@ public class ClusterServiceTests extends ESTestCase {
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
super.setUp(); super.setUp();
clusterService = createTimedClusterService(true); masterService = createTimedMasterService(true);
} }
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
clusterService.close(); masterService.close();
super.tearDown(); super.tearDown();
} }
TimedClusterService createTimedClusterService(boolean makeMaster) throws InterruptedException { private TimedMasterService createTimedMasterService(boolean makeMaster) throws InterruptedException {
TimedClusterService timedClusterService = new TimedClusterService(Settings.builder().put("cluster.name", DiscoveryNode localNode = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(),
"ClusterServiceTests").build(), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), emptySet(), Version.CURRENT);
threadPool, () -> new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), TimedMasterService timedMasterService = new TimedMasterService(Settings.builder().put("cluster.name",
emptySet(), Version.CURRENT)); MasterServiceTests.class.getSimpleName()).build(), threadPool);
timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) { ClusterState initialClusterState = ClusterState.builder(new ClusterName(MasterServiceTests.class.getSimpleName()))
@Override .nodes(DiscoveryNodes.builder()
public void connectToNodes(DiscoveryNodes discoveryNodes) { .add(localNode)
// skip .localNodeId(localNode.getId())
} .masterNodeId(makeMaster ? localNode.getId() : null))
.blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build();
@Override AtomicReference<ClusterState> clusterStateRef = new AtomicReference<>(initialClusterState);
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) { timedMasterService.setClusterStatePublisher((event, ackListener) -> clusterStateRef.set(event.state()));
// skip timedMasterService.setClusterStateSupplier(clusterStateRef::get);
} timedMasterService.start();
}); return timedMasterService;
timedClusterService.setClusterStatePublisher((event, ackListener) -> {
});
timedClusterService.setDiscoverySettings(new DiscoverySettings(Settings.EMPTY,
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)));
timedClusterService.start();
ClusterState state = timedClusterService.state();
final DiscoveryNodes nodes = state.nodes();
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(nodes)
.masterNodeId(makeMaster ? nodes.getLocalNodeId() : null);
state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK)
.nodes(nodesBuilder).build();
setState(timedClusterService, state);
return timedClusterService;
} }
public void testMasterAwareExecution() throws Exception { public void testMasterAwareExecution() throws Exception {
ClusterService nonMaster = createTimedClusterService(false); TimedMasterService nonMaster = createTimedMasterService(false);
final boolean[] taskFailed = {false}; final boolean[] taskFailed = {false};
final CountDownLatch latch1 = new CountDownLatch(1); final CountDownLatch latch1 = new CountDownLatch(1);
@ -196,7 +174,7 @@ public class ClusterServiceTests extends ESTestCase {
final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch latch = new CountDownLatch(1);
AtomicBoolean published = new AtomicBoolean(); AtomicBoolean published = new AtomicBoolean();
clusterService.submitStateUpdateTask( masterService.submitStateUpdateTask(
"testClusterStateTaskListenerThrowingExceptionIsOkay", "testClusterStateTaskListenerThrowingExceptionIsOkay",
new Object(), new Object(),
ClusterStateTaskConfig.build(Priority.NORMAL), ClusterStateTaskConfig.build(Priority.NORMAL),
@ -229,49 +207,109 @@ public class ClusterServiceTests extends ESTestCase {
assertTrue(published.get()); assertTrue(published.get());
} }
public void testBlockingCallInClusterStateTaskListenerFails() throws InterruptedException { @TestLogging("org.elasticsearch.cluster.service:TRACE") // To ensure that we log cluster state events on TRACE level
assumeTrue("assertions must be enabled for this test to work", BaseFuture.class.desiredAssertionStatus()); public void testClusterStateUpdateLogging() throws Exception {
final CountDownLatch latch = new CountDownLatch(1); MockLogAppender mockAppender = new MockLogAppender();
final AtomicReference<AssertionError> assertionRef = new AtomicReference<>(); mockAppender.start();
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test1",
masterService.getClass().getName(),
Level.DEBUG,
"*processing [test1]: took [1s] no change in cluster state"));
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test2",
masterService.getClass().getName(),
Level.TRACE,
"*failed to execute cluster state update in [2s]*"));
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test3",
masterService.getClass().getName(),
Level.DEBUG,
"*processing [test3]: took [3s] done publishing updated cluster state (version: *, uuid: *)"));
clusterService.submitStateUpdateTask( Logger clusterLogger = Loggers.getLogger(masterService.getClass().getPackage().getName());
"testBlockingCallInClusterStateTaskListenerFails", Loggers.addAppender(clusterLogger, mockAppender);
new Object(), try {
ClusterStateTaskConfig.build(Priority.NORMAL), final CountDownLatch latch = new CountDownLatch(4);
new ClusterStateTaskExecutor<Object>() { masterService.currentTimeOverride = System.nanoTime();
masterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
@Override @Override
public ClusterTasksResult<Object> execute(ClusterState currentState, List<Object> tasks) throws Exception { public ClusterState execute(ClusterState currentState) throws Exception {
ClusterState newClusterState = ClusterState.builder(currentState).build(); masterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
return ClusterTasksResult.builder().successes(tasks).build(newClusterState); return currentState;
} }
},
new ClusterStateTaskListener() {
@Override @Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
BaseFuture<Void> future = new BaseFuture<Void>() {};
try {
if (randomBoolean()) {
future.get(1L, TimeUnit.SECONDS);
} else {
future.get();
}
} catch (Exception e) {
throw new RuntimeException(e);
} catch (AssertionError e) {
assertionRef.set(e);
latch.countDown(); latch.countDown();
} }
}
@Override @Override
public void onFailure(String source, Exception e) { public void onFailure(String source, Exception e) {
fail();
} }
});
masterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
masterService.currentTimeOverride += TimeValue.timeValueSeconds(2).nanos();
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
} }
);
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
fail();
}
@Override
public void onFailure(String source, Exception e) {
latch.countDown();
}
});
masterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
masterService.currentTimeOverride += TimeValue.timeValueSeconds(3).nanos();
return ClusterState.builder(currentState).incrementVersion().build();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
// Additional update task to make sure all previous logging made it to the loggerName
// We don't check logging for this on since there is no guarantee that it will occur before our check
masterService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
latch.await(); latch.await();
assertNotNull(assertionRef.get()); } finally {
assertThat(assertionRef.get().getMessage(), containsString("not be the cluster state update thread. Reason: [Blocking operation]")); Loggers.removeAppender(clusterLogger, mockAppender);
mockAppender.stop();
}
mockAppender.assertAllExpectationsMatched();
} }
public void testClusterStateBatchedUpdates() throws BrokenBarrierException, InterruptedException { public void testClusterStateBatchedUpdates() throws BrokenBarrierException, InterruptedException {
@ -414,7 +452,7 @@ public class ClusterServiceTests extends ESTestCase {
submittedTasksPerThread.computeIfAbsent(threadName, key -> new AtomicInteger()).addAndGet(tasks.size()); submittedTasksPerThread.computeIfAbsent(threadName, key -> new AtomicInteger()).addAndGet(tasks.size());
final TaskExecutor executor = assignment.v1(); final TaskExecutor executor = assignment.v1();
if (tasks.size() == 1) { if (tasks.size() == 1) {
clusterService.submitStateUpdateTask( masterService.submitStateUpdateTask(
threadName, threadName,
tasks.stream().findFirst().get(), tasks.stream().findFirst().get(),
ClusterStateTaskConfig.build(randomFrom(Priority.values())), ClusterStateTaskConfig.build(randomFrom(Priority.values())),
@ -423,7 +461,7 @@ public class ClusterServiceTests extends ESTestCase {
} else { } else {
Map<Task, ClusterStateTaskListener> taskListeners = new HashMap<>(); Map<Task, ClusterStateTaskListener> taskListeners = new HashMap<>();
tasks.stream().forEach(t -> taskListeners.put(t, listener)); tasks.stream().forEach(t -> taskListeners.put(t, listener));
clusterService.submitStateUpdateTasks( masterService.submitStateUpdateTasks(
threadName, threadName,
taskListeners, ClusterStateTaskConfig.build(randomFrom(Priority.values())), taskListeners, ClusterStateTaskConfig.build(randomFrom(Priority.values())),
executor executor
@ -467,109 +505,47 @@ public class ClusterServiceTests extends ESTestCase {
} }
} }
@TestLogging("org.elasticsearch.cluster.service:TRACE") // To ensure that we log cluster state events on TRACE level public void testBlockingCallInClusterStateTaskListenerFails() throws InterruptedException {
public void testClusterStateUpdateLogging() throws Exception { assumeTrue("assertions must be enabled for this test to work", BaseFuture.class.desiredAssertionStatus());
MockLogAppender mockAppender = new MockLogAppender(); final CountDownLatch latch = new CountDownLatch(1);
mockAppender.start(); final AtomicReference<AssertionError> assertionRef = new AtomicReference<>();
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test1",
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService",
Level.DEBUG,
"*processing [test1]: took [1s] no change in cluster_state"));
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test2",
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService",
Level.TRACE,
"*failed to execute cluster state update in [2s]*"));
mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation(
"test3",
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService",
Level.DEBUG,
"*processing [test3]: took [3s] done applying updated cluster_state (version: *, uuid: *)"));
Logger clusterLogger = Loggers.getLogger("org.elasticsearch.cluster.service"); masterService.submitStateUpdateTask(
Loggers.addAppender(clusterLogger, mockAppender); "testBlockingCallInClusterStateTaskListenerFails",
new Object(),
ClusterStateTaskConfig.build(Priority.NORMAL),
(currentState, tasks) -> {
ClusterState newClusterState = ClusterState.builder(currentState).build();
return ClusterStateTaskExecutor.ClusterTasksResult.builder().successes(tasks).build(newClusterState);
},
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
BaseFuture<Void> future = new BaseFuture<Void>() {};
try { try {
final CountDownLatch latch = new CountDownLatch(4); if (randomBoolean()) {
clusterService.currentTimeOverride = System.nanoTime(); future.get(1L, TimeUnit.SECONDS);
clusterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() { } else {
@Override future.get();
public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
return currentState;
} }
} catch (Exception e) {
@Override throw new RuntimeException(e);
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { } catch (AssertionError e) {
assertionRef.set(e);
latch.countDown(); latch.countDown();
} }
}
@Override @Override
public void onFailure(String source, Exception e) { public void onFailure(String source, Exception e) {
fail();
} }
});
clusterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(2).nanos();
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
} }
);
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
fail();
}
@Override
public void onFailure(String source, Exception e) {
latch.countDown();
}
});
clusterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(3).nanos();
return ClusterState.builder(currentState).incrementVersion().build();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
// Additional update task to make sure all previous logging made it to the loggerName
// We don't check logging for this on since there is no guarantee that it will occur before our check
clusterService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
return currentState;
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
fail();
}
});
latch.await(); latch.await();
} finally { assertNotNull(assertionRef.get());
Loggers.removeAppender(clusterLogger, mockAppender); assertThat(assertionRef.get().getMessage(),
mockAppender.stop(); containsString("Reason: [Blocking operation]"));
}
mockAppender.assertAllExpectationsMatched();
} }
@TestLogging("org.elasticsearch.cluster.service:WARN") // To ensure that we log cluster state events on WARN level @TestLogging("org.elasticsearch.cluster.service:WARN") // To ensure that we log cluster state events on WARN level
@ -579,38 +555,38 @@ public class ClusterServiceTests extends ESTestCase {
mockAppender.addExpectation( mockAppender.addExpectation(
new MockLogAppender.UnseenEventExpectation( new MockLogAppender.UnseenEventExpectation(
"test1 shouldn't see because setting is too low", "test1 shouldn't see because setting is too low",
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService", masterService.getClass().getName(),
Level.WARN, Level.WARN,
"*cluster state update task [test1] took [*] above the warn threshold of *")); "*cluster state update task [test1] took [*] above the warn threshold of *"));
mockAppender.addExpectation( mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation( new MockLogAppender.SeenEventExpectation(
"test2", "test2",
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService", masterService.getClass().getName(),
Level.WARN, Level.WARN,
"*cluster state update task [test2] took [32s] above the warn threshold of *")); "*cluster state update task [test2] took [32s] above the warn threshold of *"));
mockAppender.addExpectation( mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation( new MockLogAppender.SeenEventExpectation(
"test3", "test3",
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService", masterService.getClass().getName(),
Level.WARN, Level.WARN,
"*cluster state update task [test3] took [33s] above the warn threshold of *")); "*cluster state update task [test3] took [33s] above the warn threshold of *"));
mockAppender.addExpectation( mockAppender.addExpectation(
new MockLogAppender.SeenEventExpectation( new MockLogAppender.SeenEventExpectation(
"test4", "test4",
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService", masterService.getClass().getName(),
Level.WARN, Level.WARN,
"*cluster state update task [test4] took [34s] above the warn threshold of *")); "*cluster state update task [test4] took [34s] above the warn threshold of *"));
Logger clusterLogger = Loggers.getLogger("org.elasticsearch.cluster.service"); Logger clusterLogger = Loggers.getLogger(masterService.getClass().getPackage().getName());
Loggers.addAppender(clusterLogger, mockAppender); Loggers.addAppender(clusterLogger, mockAppender);
try { try {
final CountDownLatch latch = new CountDownLatch(5); final CountDownLatch latch = new CountDownLatch(5);
final CountDownLatch processedFirstTask = new CountDownLatch(1); final CountDownLatch processedFirstTask = new CountDownLatch(1);
clusterService.currentTimeOverride = System.nanoTime(); masterService.currentTimeOverride = System.nanoTime();
clusterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() { masterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
@Override @Override
public ClusterState execute(ClusterState currentState) throws Exception { public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos(); masterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
return currentState; return currentState;
} }
@ -627,10 +603,10 @@ public class ClusterServiceTests extends ESTestCase {
}); });
processedFirstTask.await(); processedFirstTask.await();
clusterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() { masterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override @Override
public ClusterState execute(ClusterState currentState) throws Exception { public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(32).nanos(); masterService.currentTimeOverride += TimeValue.timeValueSeconds(32).nanos();
throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task"); throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task");
} }
@ -644,10 +620,10 @@ public class ClusterServiceTests extends ESTestCase {
latch.countDown(); latch.countDown();
} }
}); });
clusterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() { masterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
@Override @Override
public ClusterState execute(ClusterState currentState) throws Exception { public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(33).nanos(); masterService.currentTimeOverride += TimeValue.timeValueSeconds(33).nanos();
return ClusterState.builder(currentState).incrementVersion().build(); return ClusterState.builder(currentState).incrementVersion().build();
} }
@ -661,10 +637,10 @@ public class ClusterServiceTests extends ESTestCase {
fail(); fail();
} }
}); });
clusterService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() { masterService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
@Override @Override
public ClusterState execute(ClusterState currentState) throws Exception { public ClusterState execute(ClusterState currentState) throws Exception {
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(34).nanos(); masterService.currentTimeOverride += TimeValue.timeValueSeconds(34).nanos();
return currentState; return currentState;
} }
@ -680,7 +656,7 @@ public class ClusterServiceTests extends ESTestCase {
}); });
// Additional update task to make sure all previous logging made it to the loggerName // Additional update task to make sure all previous logging made it to the loggerName
// We don't check logging for this on since there is no guarantee that it will occur before our check // We don't check logging for this on since there is no guarantee that it will occur before our check
clusterService.submitStateUpdateTask("test5", new ClusterStateUpdateTask() { masterService.submitStateUpdateTask("test5", new ClusterStateUpdateTask() {
@Override @Override
public ClusterState execute(ClusterState currentState) { public ClusterState execute(ClusterState currentState) {
return currentState; return currentState;
@ -704,211 +680,12 @@ public class ClusterServiceTests extends ESTestCase {
mockAppender.assertAllExpectationsMatched(); mockAppender.assertAllExpectationsMatched();
} }
public void testDisconnectFromNewlyAddedNodesIfClusterStatePublishingFails() throws InterruptedException { static class TimedMasterService extends MasterService {
TimedClusterService timedClusterService = new TimedClusterService(Settings.builder().put("cluster.name",
"ClusterServiceTests").build(), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
threadPool, () -> new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(),
emptySet(), Version.CURRENT));
Set<DiscoveryNode> currentNodes = new HashSet<>();
timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
@Override
public void connectToNodes(DiscoveryNodes discoveryNodes) {
discoveryNodes.forEach(currentNodes::add);
}
@Override
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
Set<DiscoveryNode> nodeSet = new HashSet<>();
nodesToKeep.iterator().forEachRemaining(nodeSet::add);
currentNodes.removeIf(node -> nodeSet.contains(node) == false);
}
});
AtomicBoolean failToCommit = new AtomicBoolean();
timedClusterService.setClusterStatePublisher((event, ackListener) -> {
if (failToCommit.get()) {
throw new Discovery.FailedToCommitClusterStateException("just to test this");
}
});
timedClusterService.setDiscoverySettings(new DiscoverySettings(Settings.EMPTY,
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)));
timedClusterService.start();
ClusterState state = timedClusterService.state();
final DiscoveryNodes nodes = state.nodes();
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(nodes)
.masterNodeId(nodes.getLocalNodeId());
state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK)
.nodes(nodesBuilder).build();
setState(timedClusterService, state);
assertThat(currentNodes, equalTo(Sets.newHashSet(timedClusterService.state().getNodes())));
final CountDownLatch latch = new CountDownLatch(1);
// try to add node when cluster state publishing fails
failToCommit.set(true);
timedClusterService.submitStateUpdateTask("test", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
DiscoveryNode newNode = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(),
emptySet(), Version.CURRENT);
return ClusterState.builder(currentState).nodes(DiscoveryNodes.builder(currentState.nodes()).add(newNode)).build();
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
latch.countDown();
}
});
latch.await();
assertThat(currentNodes, equalTo(Sets.newHashSet(timedClusterService.state().getNodes())));
timedClusterService.close();
}
public void testLocalNodeMasterListenerCallbacks() throws Exception {
TimedClusterService timedClusterService = createTimedClusterService(false);
AtomicBoolean isMaster = new AtomicBoolean();
timedClusterService.addLocalNodeMasterListener(new LocalNodeMasterListener() {
@Override
public void onMaster() {
isMaster.set(true);
}
@Override
public void offMaster() {
isMaster.set(false);
}
@Override
public String executorName() {
return ThreadPool.Names.SAME;
}
});
ClusterState state = timedClusterService.state();
DiscoveryNodes nodes = state.nodes();
DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(nodes).masterNodeId(nodes.getLocalNodeId());
state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).nodes(nodesBuilder).build();
setState(timedClusterService, state);
assertThat(isMaster.get(), is(true));
nodes = state.nodes();
nodesBuilder = DiscoveryNodes.builder(nodes).masterNodeId(null);
state = ClusterState.builder(state).blocks(ClusterBlocks.builder().addGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_WRITES))
.nodes(nodesBuilder).build();
setState(timedClusterService, state);
assertThat(isMaster.get(), is(false));
nodesBuilder = DiscoveryNodes.builder(nodes).masterNodeId(nodes.getLocalNodeId());
state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).nodes(nodesBuilder).build();
setState(timedClusterService, state);
assertThat(isMaster.get(), is(true));
timedClusterService.close();
}
public void testClusterStateApplierCantSampleClusterState() throws InterruptedException {
AtomicReference<Throwable> error = new AtomicReference<>();
AtomicBoolean applierCalled = new AtomicBoolean();
clusterService.addStateApplier(event -> {
try {
applierCalled.set(true);
clusterService.state();
error.set(new AssertionError("successfully sampled state"));
} catch (AssertionError e) {
if (e.getMessage().contains("should not be called by a cluster state applier") == false) {
error.set(e);
}
}
});
CountDownLatch latch = new CountDownLatch(1);
clusterService.submitStateUpdateTask("test", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
return ClusterState.builder(currentState).build();
}
@Override
public void onFailure(String source, Exception e) {
error.compareAndSet(null, e);
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
});
latch.await();
assertNull(error.get());
assertTrue(applierCalled.get());
}
public void testClusterStateApplierCanCreateAnObserver() throws InterruptedException {
AtomicReference<Throwable> error = new AtomicReference<>();
AtomicBoolean applierCalled = new AtomicBoolean();
clusterService.addStateApplier(event -> {
try {
applierCalled.set(true);
ClusterStateObserver observer = new ClusterStateObserver(event.state(),
clusterService, null, logger, threadPool.getThreadContext());
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
}
@Override
public void onClusterServiceClose() {
}
@Override
public void onTimeout(TimeValue timeout) {
}
});
} catch (AssertionError e) {
error.set(e);
}
});
CountDownLatch latch = new CountDownLatch(1);
clusterService.submitStateUpdateTask("test", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
return ClusterState.builder(currentState).build();
}
@Override
public void onFailure(String source, Exception e) {
error.compareAndSet(null, e);
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
});
latch.await();
assertNull(error.get());
assertTrue(applierCalled.get());
}
static class TimedClusterService extends ClusterService {
public volatile Long currentTimeOverride = null; public volatile Long currentTimeOverride = null;
TimedClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool, TimedMasterService(Settings settings, ThreadPool threadPool) {
Supplier<DiscoveryNode> localNodeSupplier) { super(settings, threadPool);
super(settings, clusterSettings, threadPool, localNodeSupplier);
} }
@Override @Override
@ -919,4 +696,11 @@ public class ClusterServiceTests extends ESTestCase {
return super.currentTimeInNanos(); return super.currentTimeInNanos();
} }
} }
/**
* Returns the cluster state that the master service uses (and that is provided by the discovery layer)
*/
public static ClusterState discoveryState(MasterService masterService) {
return masterService.state();
}
} }

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
@ -282,7 +283,7 @@ public class ClusterSettingsIT extends ESIntegTestCase {
assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1L)); assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1L));
} }
private DiscoverySettings getDiscoverySettings() {return internalCluster().getInstance(Discovery.class).getDiscoverySettings();} private DiscoverySettings getDiscoverySettings() {return ((ZenDiscovery) internalCluster().getInstance(Discovery.class)).getDiscoverySettings();}
public void testClusterUpdateSettingsWithBlocks() { public void testClusterUpdateSettingsWithBlocks() {
String key1 = "cluster.routing.allocation.enable"; String key1 = "cluster.routing.allocation.enable";

View File

@ -18,47 +18,43 @@
*/ */
package org.elasticsearch.discovery; package org.elasticsearch.discovery;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;
import java.util.function.Supplier;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.zen.UnicastHostsProvider; import org.elasticsearch.discovery.zen.UnicastHostsProvider;
import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.discovery.zen.ZenPing;
import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.plugins.DiscoveryPlugin;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.NoopDiscovery; import org.elasticsearch.test.NoopDiscovery;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.mockito.Mock;
import org.mockito.Mockito; import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Supplier;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class DiscoveryModuleTests extends ESTestCase { public class DiscoveryModuleTests extends ESTestCase {
private TransportService transportService; private TransportService transportService;
private NamedWriteableRegistry namedWriteableRegistry; private NamedWriteableRegistry namedWriteableRegistry;
private ClusterService clusterService; private MasterService masterService;
private ClusterApplier clusterApplier;
private ThreadPool threadPool; private ThreadPool threadPool;
private ClusterSettings clusterSettings;
public interface DummyHostsProviderPlugin extends DiscoveryPlugin { public interface DummyHostsProviderPlugin extends DiscoveryPlugin {
Map<String, Supplier<UnicastHostsProvider>> impl(); Map<String, Supplier<UnicastHostsProvider>> impl();
@ -74,7 +70,8 @@ public class DiscoveryModuleTests extends ESTestCase {
@Override @Override
default Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, default Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) { MasterService masterService, ClusterApplier clusterApplier,
ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
return impl(); return impl();
} }
} }
@ -82,11 +79,11 @@ public class DiscoveryModuleTests extends ESTestCase {
@Before @Before
public void setupDummyServices() { public void setupDummyServices() {
transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, null, null); transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, null, null);
clusterService = mock(ClusterService.class); masterService = mock(MasterService.class);
namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList()); namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); clusterApplier = mock(ClusterApplier.class);
when(clusterService.getClusterSettings()).thenReturn(clusterSettings);
threadPool = mock(ThreadPool.class); threadPool = mock(ThreadPool.class);
clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
} }
@After @After
@ -95,7 +92,8 @@ public class DiscoveryModuleTests extends ESTestCase {
} }
private DiscoveryModule newModule(Settings settings, List<DiscoveryPlugin> plugins) { private DiscoveryModule newModule(Settings settings, List<DiscoveryPlugin> plugins) {
return new DiscoveryModule(settings, threadPool, transportService, namedWriteableRegistry, null, clusterService, plugins); return new DiscoveryModule(settings, threadPool, transportService, namedWriteableRegistry, null, masterService,
clusterApplier, clusterSettings, plugins);
} }
public void testDefaults() { public void testDefaults() {

View File

@ -447,6 +447,8 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
ensureGreen("test"); ensureGreen("test");
// verify all cluster states are the same // verify all cluster states are the same
// use assert busy to wait for cluster states to be applied (as publish_timeout has low value)
assertBusy(() -> {
ClusterState state = null; ClusterState state = null;
for (String node : nodes) { for (String node : nodes) {
ClusterState nodeState = getNodeClusterState(node); ClusterState nodeState = getNodeClusterState(node);
@ -460,9 +462,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
assertEquals("unequal node count", state.nodes().getSize(), nodeState.nodes().getSize()); assertEquals("unequal node count", state.nodes().getSize(), nodeState.nodes().getSize());
assertEquals("different masters ", state.nodes().getMasterNodeId(), nodeState.nodes().getMasterNodeId()); assertEquals("different masters ", state.nodes().getMasterNodeId(), nodeState.nodes().getMasterNodeId());
assertEquals("different meta data version", state.metaData().version(), nodeState.metaData().version()); assertEquals("different meta data version", state.metaData().version(), nodeState.metaData().version());
if (!state.routingTable().toString().equals(nodeState.routingTable().toString())) { assertEquals("different routing", state.routingTable().toString(), nodeState.routingTable().toString());
fail("different routing");
}
} catch (AssertionError t) { } catch (AssertionError t) {
fail("failed comparing cluster state: " + t.getMessage() + "\n" + fail("failed comparing cluster state: " + t.getMessage() + "\n" +
"--- cluster state of node [" + nodes.get(0) + "]: ---\n" + state + "--- cluster state of node [" + nodes.get(0) + "]: ---\n" + state +
@ -470,6 +470,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
} }
} }
});
} }
/** /**

View File

@ -25,7 +25,6 @@ import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@ -58,17 +57,14 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.singleton; import static java.util.Collections.singleton;
import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
import static org.elasticsearch.test.ClusterServiceUtils.setState;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo;
public class ZenFaultDetectionTests extends ESTestCase { public class ZenFaultDetectionTests extends ESTestCase {
protected ThreadPool threadPool; protected ThreadPool threadPool;
protected ClusterService clusterServiceA;
protected ClusterService clusterServiceB;
private CircuitBreakerService circuitBreakerService; private CircuitBreakerService circuitBreakerService;
protected static final Version version0 = Version.fromId(/*0*/99); protected static final Version version0 = Version.fromId(/*0*/99);
@ -97,8 +93,6 @@ public class ZenFaultDetectionTests extends ESTestCase {
settingsB = Settings.builder().put("node.name", "TS_B").put(settings).build(); settingsB = Settings.builder().put("node.name", "TS_B").put(settings).build();
serviceB = build(settingsB, version1); serviceB = build(settingsB, version1);
nodeB = serviceB.getLocalDiscoNode(); nodeB = serviceB.getLocalDiscoNode();
clusterServiceA = createClusterService(settingsA, threadPool, nodeA);
clusterServiceB = createClusterService(settingsB, threadPool, nodeB);
// wait till all nodes are properly connected and the event has been sent, so tests in this class // wait till all nodes are properly connected and the event has been sent, so tests in this class
// will not get this callback called on the connections done in this setup // will not get this callback called on the connections done in this setup
@ -133,8 +127,6 @@ public class ZenFaultDetectionTests extends ESTestCase {
super.tearDown(); super.tearDown();
serviceA.close(); serviceA.close();
serviceB.close(); serviceB.close();
clusterServiceA.close();
clusterServiceB.close();
terminate(threadPool); terminate(threadPool);
} }
@ -241,9 +233,9 @@ public class ZenFaultDetectionTests extends ESTestCase {
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m").put("cluster.name", clusterName.value()); .put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m").put("cluster.name", clusterName.value());
final ClusterState state = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build(); final ClusterState state = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build();
setState(clusterServiceA, state); AtomicReference<ClusterState> clusterStateSupplier = new AtomicReference<>(state);
MasterFaultDetection masterFD = new MasterFaultDetection(settings.build(), threadPool, serviceA, MasterFaultDetection masterFD = new MasterFaultDetection(settings.build(), threadPool, serviceA,
clusterServiceA); clusterStateSupplier::get, null, clusterName);
masterFD.restart(nodeB, "test"); masterFD.restart(nodeB, "test");
final String[] failureReason = new String[1]; final String[] failureReason = new String[1];
@ -278,7 +270,7 @@ public class ZenFaultDetectionTests extends ESTestCase {
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "1s") .put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "1s")
.put("cluster.name", clusterName.value()).build(); .put("cluster.name", clusterName.value()).build();
final ClusterState stateNodeA = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build(); final ClusterState stateNodeA = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build();
setState(clusterServiceA, stateNodeA); AtomicReference<ClusterState> clusterStateSupplierA = new AtomicReference<>(stateNodeA);
int minExpectedPings = 2; int minExpectedPings = 2;
@ -289,14 +281,14 @@ public class ZenFaultDetectionTests extends ESTestCase {
serviceB.addTracer(pingProbeB); serviceB.addTracer(pingProbeB);
MasterFaultDetection masterFDNodeA = new MasterFaultDetection(Settings.builder().put(settingsA).put(settings).build(), MasterFaultDetection masterFDNodeA = new MasterFaultDetection(Settings.builder().put(settingsA).put(settings).build(),
threadPool, serviceA, clusterServiceA); threadPool, serviceA, clusterStateSupplierA::get, null, clusterName);
masterFDNodeA.restart(nodeB, "test"); masterFDNodeA.restart(nodeB, "test");
final ClusterState stateNodeB = ClusterState.builder(clusterName).nodes(buildNodesForB(true)).build(); final ClusterState stateNodeB = ClusterState.builder(clusterName).nodes(buildNodesForB(true)).build();
setState(clusterServiceB, stateNodeB); AtomicReference<ClusterState> clusterStateSupplierB = new AtomicReference<>(stateNodeB);
MasterFaultDetection masterFDNodeB = new MasterFaultDetection(Settings.builder().put(settingsB).put(settings).build(), MasterFaultDetection masterFDNodeB = new MasterFaultDetection(Settings.builder().put(settingsB).put(settings).build(),
threadPool, serviceB, clusterServiceB); threadPool, serviceB, clusterStateSupplierB::get, null, clusterName);
masterFDNodeB.restart(nodeB, "test"); masterFDNodeB.restart(nodeB, "test");
// let's do a few pings // let's do a few pings

View File

@ -21,14 +21,10 @@ package org.elasticsearch.discovery.single;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.TestThreadPool;
@ -36,7 +32,6 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable; import java.io.Closeable;
import java.util.Stack; import java.util.Stack;
import java.util.concurrent.CountDownLatch;
import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
@ -57,37 +52,10 @@ public class SingleNodeDiscoveryTests extends ESTestCase {
final ClusterService clusterService = createClusterService(threadPool, node); final ClusterService clusterService = createClusterService(threadPool, node);
stack.push(clusterService); stack.push(clusterService);
final SingleNodeDiscovery discovery = final SingleNodeDiscovery discovery =
new SingleNodeDiscovery(Settings.EMPTY, clusterService); new SingleNodeDiscovery(Settings.EMPTY, transportService,
clusterService.getClusterApplierService());
discovery.startInitialJoin(); discovery.startInitialJoin();
final DiscoveryNodes nodes = discovery.getInitialClusterState().nodes();
// we are racing against the initial join which is asynchronous so we use an observer
final ClusterState state = clusterService.state();
final ThreadContext threadContext = threadPool.getThreadContext();
final ClusterStateObserver observer =
new ClusterStateObserver(state, clusterService, null, logger, threadContext);
if (state.nodes().getMasterNodeId() == null) {
final CountDownLatch latch = new CountDownLatch(1);
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
latch.countDown();
}
@Override
public void onClusterServiceClose() {
latch.countDown();
}
@Override
public void onTimeout(TimeValue timeout) {
assert false;
}
}, s -> s.nodes().getMasterNodeId() != null);
latch.await();
}
final DiscoveryNodes nodes = clusterService.state().nodes();
assertThat(nodes.getSize(), equalTo(1)); assertThat(nodes.getSize(), equalTo(1));
assertThat(nodes.getMasterNode().getId(), equalTo(node.getId())); assertThat(nodes.getMasterNode().getId(), equalTo(node.getId()));
} finally { } finally {

View File

@ -22,8 +22,10 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier; import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.NotMasterException;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
@ -35,14 +37,14 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.cluster.service.MasterServiceTests;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.BaseFuture; import org.elasticsearch.common.util.concurrent.BaseFuture;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ClusterServiceUtils;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.junit.annotations.TestLogging;
@ -81,8 +83,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED;
import static org.elasticsearch.cluster.routing.RoutingTableTests.updateActiveAllocations; import static org.elasticsearch.cluster.routing.RoutingTableTests.updateActiveAllocations;
import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; import static org.elasticsearch.cluster.service.MasterServiceTests.discoveryState;
import static org.elasticsearch.test.ClusterServiceUtils.setState;
import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
@ -93,12 +94,12 @@ public class NodeJoinControllerTests extends ESTestCase {
private static ThreadPool threadPool; private static ThreadPool threadPool;
private ClusterService clusterService; private MasterService masterService;
private NodeJoinController nodeJoinController; private NodeJoinController nodeJoinController;
@BeforeClass @BeforeClass
public static void beforeClass() { public static void beforeClass() {
threadPool = new TestThreadPool("ShardReplicationTests"); threadPool = new TestThreadPool("NodeJoinControllerTests");
} }
@AfterClass @AfterClass
@ -110,25 +111,39 @@ public class NodeJoinControllerTests extends ESTestCase {
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
super.setUp(); super.setUp();
clusterService = createClusterService(threadPool);
final DiscoveryNodes initialNodes = clusterService.state().nodes();
final DiscoveryNode localNode = initialNodes.getLocalNode();
// make sure we have a master
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(
DiscoveryNodes.builder(initialNodes).masterNodeId(localNode.getId())));
nodeJoinController = new NodeJoinController(clusterService, createAllocationService(Settings.EMPTY),
new ElectMasterService(Settings.EMPTY), Settings.EMPTY);
} }
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
super.tearDown(); super.tearDown();
clusterService.close(); masterService.close();
}
private static ClusterState initialState(boolean withMaster) {
DiscoveryNode localNode = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())),Version.CURRENT);
ClusterState initialClusterState = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName()))
.nodes(DiscoveryNodes.builder()
.add(localNode)
.localNodeId(localNode.getId())
.masterNodeId(withMaster ? localNode.getId() : null))
.blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build();
return initialClusterState;
}
private void setupMasterServiceAndNodeJoinController(ClusterState initialState) {
if (masterService != null || nodeJoinController != null) {
throw new IllegalStateException("method setupMasterServiceAndNodeJoinController can only be called once");
}
masterService = ClusterServiceUtils.createMasterService(threadPool, initialState);
nodeJoinController = new NodeJoinController(masterService, createAllocationService(Settings.EMPTY),
new ElectMasterService(Settings.EMPTY), Settings.EMPTY);
} }
public void testSimpleJoinAccumulation() throws InterruptedException, ExecutionException { public void testSimpleJoinAccumulation() throws InterruptedException, ExecutionException {
setupMasterServiceAndNodeJoinController(initialState(true));
List<DiscoveryNode> nodes = new ArrayList<>(); List<DiscoveryNode> nodes = new ArrayList<>();
nodes.add(clusterService.localNode()); nodes.add(discoveryState(masterService).nodes().getLocalNode());
int nodeId = 0; int nodeId = 0;
for (int i = randomInt(5); i > 0; i--) { for (int i = randomInt(5); i > 0; i--) {
@ -162,9 +177,7 @@ public class NodeJoinControllerTests extends ESTestCase {
} }
public void testFailingJoinsWhenNotMaster() throws ExecutionException, InterruptedException { public void testFailingJoinsWhenNotMaster() throws ExecutionException, InterruptedException {
// remove current master flag setupMasterServiceAndNodeJoinController(initialState(false));
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null);
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
int nodeId = 0; int nodeId = 0;
try { try {
joinNode(newNode(nodeId++)); joinNode(newNode(nodeId++));
@ -194,8 +207,7 @@ public class NodeJoinControllerTests extends ESTestCase {
} }
public void testSimpleMasterElectionWithoutRequiredJoins() throws InterruptedException, ExecutionException { public void testSimpleMasterElectionWithoutRequiredJoins() throws InterruptedException, ExecutionException {
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null); setupMasterServiceAndNodeJoinController(initialState(false));
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
int nodeId = 0; int nodeId = 0;
final int requiredJoins = 0; final int requiredJoins = 0;
logger.debug("--> using requiredJoins [{}]", requiredJoins); logger.debug("--> using requiredJoins [{}]", requiredJoins);
@ -244,8 +256,7 @@ public class NodeJoinControllerTests extends ESTestCase {
} }
public void testSimpleMasterElection() throws InterruptedException, ExecutionException { public void testSimpleMasterElection() throws InterruptedException, ExecutionException {
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null); setupMasterServiceAndNodeJoinController(initialState(false));
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
int nodeId = 0; int nodeId = 0;
final int requiredJoins = 1 + randomInt(5); final int requiredJoins = 1 + randomInt(5);
logger.debug("--> using requiredJoins [{}]", requiredJoins); logger.debug("--> using requiredJoins [{}]", requiredJoins);
@ -356,10 +367,8 @@ public class NodeJoinControllerTests extends ESTestCase {
} }
public void testMasterElectionTimeout() throws InterruptedException { public void testMasterElectionTimeout() throws InterruptedException {
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null); setupMasterServiceAndNodeJoinController(initialState(false));
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
int nodeId = 0; int nodeId = 0;
final int requiredJoins = 1 + randomInt(5); final int requiredJoins = 1 + randomInt(5);
logger.debug("--> using requiredJoins [{}]", requiredJoins); logger.debug("--> using requiredJoins [{}]", requiredJoins);
@ -422,22 +431,23 @@ public class NodeJoinControllerTests extends ESTestCase {
} }
public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, ExecutionException { public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, ExecutionException {
ClusterState state = clusterService.state(); ClusterState state = initialState(true);
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes());
final DiscoveryNode other_node = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), final DiscoveryNode other_node = new DiscoveryNode("other_node", buildNewFakeTransportAddress(),
emptyMap(), emptySet(), Version.CURRENT); emptyMap(), emptySet(), Version.CURRENT);
nodesBuilder.add(other_node); nodesBuilder.add(other_node);
setState(clusterService, ClusterState.builder(state).nodes(nodesBuilder)); setupMasterServiceAndNodeJoinController(ClusterState.builder(state).nodes(nodesBuilder).build());
state = clusterService.state(); state = discoveryState(masterService);
joinNode(other_node); joinNode(other_node);
assertTrue("failed to publish a new state upon existing join", clusterService.state() != state); assertTrue("failed to publish a new state upon existing join", discoveryState(masterService) != state);
} }
public void testNormalConcurrentJoins() throws InterruptedException { public void testNormalConcurrentJoins() throws InterruptedException {
setupMasterServiceAndNodeJoinController(initialState(true));
Thread[] threads = new Thread[3 + randomInt(5)]; Thread[] threads = new Thread[3 + randomInt(5)];
ArrayList<DiscoveryNode> nodes = new ArrayList<>(); ArrayList<DiscoveryNode> nodes = new ArrayList<>();
nodes.add(clusterService.localNode()); nodes.add(discoveryState(masterService).nodes().getLocalNode());
final CyclicBarrier barrier = new CyclicBarrier(threads.length); final CyclicBarrier barrier = new CyclicBarrier(threads.length);
final List<Throwable> backgroundExceptions = new CopyOnWriteArrayList<>(); final List<Throwable> backgroundExceptions = new CopyOnWriteArrayList<>();
for (int i = 0; i < threads.length; i++) { for (int i = 0; i < threads.length; i++) {
@ -472,15 +482,14 @@ public class NodeJoinControllerTests extends ESTestCase {
} }
public void testElectionWithConcurrentJoins() throws InterruptedException, BrokenBarrierException { public void testElectionWithConcurrentJoins() throws InterruptedException, BrokenBarrierException {
DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null); setupMasterServiceAndNodeJoinController(initialState(false));
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodesBuilder));
nodeJoinController.startElectionContext(); nodeJoinController.startElectionContext();
Thread[] threads = new Thread[3 + randomInt(5)]; Thread[] threads = new Thread[3 + randomInt(5)];
final int requiredJoins = randomInt(threads.length); final int requiredJoins = randomInt(threads.length);
ArrayList<DiscoveryNode> nodes = new ArrayList<>(); ArrayList<DiscoveryNode> nodes = new ArrayList<>();
nodes.add(clusterService.localNode()); nodes.add(discoveryState(masterService).nodes().getLocalNode());
final CyclicBarrier barrier = new CyclicBarrier(threads.length + 1); final CyclicBarrier barrier = new CyclicBarrier(threads.length + 1);
final List<Throwable> backgroundExceptions = new CopyOnWriteArrayList<>(); final List<Throwable> backgroundExceptions = new CopyOnWriteArrayList<>();
for (int i = 0; i < threads.length; i++) { for (int i = 0; i < threads.length; i++) {
@ -539,7 +548,7 @@ public class NodeJoinControllerTests extends ESTestCase {
public void testRejectingJoinWithSameAddressButDifferentId() throws InterruptedException, ExecutionException { public void testRejectingJoinWithSameAddressButDifferentId() throws InterruptedException, ExecutionException {
addNodes(randomInt(5)); addNodes(randomInt(5));
ClusterState state = clusterService.state(); ClusterState state = discoveryState(masterService);
final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList()));
final DiscoveryNode other_node = new DiscoveryNode("other_node", existing.getAddress(), emptyMap(), emptySet(), Version.CURRENT); final DiscoveryNode other_node = new DiscoveryNode("other_node", existing.getAddress(), emptyMap(), emptySet(), Version.CURRENT);
@ -549,7 +558,7 @@ public class NodeJoinControllerTests extends ESTestCase {
public void testRejectingJoinWithSameIdButDifferentNode() throws InterruptedException, ExecutionException { public void testRejectingJoinWithSameIdButDifferentNode() throws InterruptedException, ExecutionException {
addNodes(randomInt(5)); addNodes(randomInt(5));
ClusterState state = clusterService.state(); ClusterState state = discoveryState(masterService);
final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList()));
final DiscoveryNode other_node = new DiscoveryNode( final DiscoveryNode other_node = new DiscoveryNode(
randomBoolean() ? existing.getName() : "other_name", randomBoolean() ? existing.getName() : "other_name",
@ -565,7 +574,7 @@ public class NodeJoinControllerTests extends ESTestCase {
public void testRejectingRestartedNodeJoinsBeforeProcessingNodeLeft() throws InterruptedException, ExecutionException { public void testRejectingRestartedNodeJoinsBeforeProcessingNodeLeft() throws InterruptedException, ExecutionException {
addNodes(randomInt(5)); addNodes(randomInt(5));
ClusterState state = clusterService.state(); ClusterState state = discoveryState(masterService);
final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList()));
joinNode(existing); // OK joinNode(existing); // OK
@ -581,15 +590,16 @@ public class NodeJoinControllerTests extends ESTestCase {
* nodes that conflict with the joins it got and needs to become a master * nodes that conflict with the joins it got and needs to become a master
*/ */
public void testElectionBasedOnConflictingNodes() throws InterruptedException, ExecutionException { public void testElectionBasedOnConflictingNodes() throws InterruptedException, ExecutionException {
final DiscoveryNode masterNode = clusterService.localNode(); ClusterState initialState = initialState(true);
final DiscoveryNode masterNode = initialState.nodes().getLocalNode();
final DiscoveryNode otherNode = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(), final DiscoveryNode otherNode = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(),
EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT); EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT);
// simulate master going down with stale nodes in it's cluster state (for example when min master nodes is set to 2) // simulate master going down with stale nodes in it's cluster state (for example when min master nodes is set to 2)
// also add some shards to that node // also add some shards to that node
DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(clusterService.state().nodes()); DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(initialState.nodes());
discoBuilder.masterNodeId(null); discoBuilder.masterNodeId(null);
discoBuilder.add(otherNode); discoBuilder.add(otherNode);
ClusterState.Builder stateBuilder = ClusterState.builder(clusterService.state()).nodes(discoBuilder); ClusterState.Builder stateBuilder = ClusterState.builder(initialState).nodes(discoBuilder);
if (randomBoolean()) { if (randomBoolean()) {
IndexMetaData indexMetaData = IndexMetaData.builder("test").settings(Settings.builder() IndexMetaData indexMetaData = IndexMetaData.builder("test").settings(Settings.builder()
.put(SETTING_VERSION_CREATED, Version.CURRENT) .put(SETTING_VERSION_CREATED, Version.CURRENT)
@ -623,7 +633,7 @@ public class NodeJoinControllerTests extends ESTestCase {
.routingTable(RoutingTable.builder().add(indexRoutingTable).build()); .routingTable(RoutingTable.builder().add(indexRoutingTable).build());
} }
setState(clusterService, stateBuilder.build()); setupMasterServiceAndNodeJoinController(stateBuilder.build());
// conflict on node id or address // conflict on node id or address
final DiscoveryNode conflictingNode = randomBoolean() ? final DiscoveryNode conflictingNode = randomBoolean() ?
@ -652,7 +662,7 @@ public class NodeJoinControllerTests extends ESTestCase {
joinFuture.get(); // throw any exception joinFuture.get(); // throw any exception
final ClusterState finalState = clusterService.state(); final ClusterState finalState = discoveryState(masterService);
final DiscoveryNodes finalNodes = finalState.nodes(); final DiscoveryNodes finalNodes = finalState.nodes();
assertTrue(finalNodes.isLocalNodeElectedMaster()); assertTrue(finalNodes.isLocalNodeElectedMaster());
assertThat(finalNodes.getLocalNode(), equalTo(masterNode)); assertThat(finalNodes.getLocalNode(), equalTo(masterNode));
@ -666,18 +676,18 @@ public class NodeJoinControllerTests extends ESTestCase {
private void addNodes(int count) { private void addNodes(int count) {
ClusterState state = clusterService.state(); ClusterState state = initialState(true);
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes());
for (int i = 0;i< count;i++) { for (int i = 0;i< count;i++) {
final DiscoveryNode node = new DiscoveryNode("node_" + state.nodes().getSize() + i, buildNewFakeTransportAddress(), final DiscoveryNode node = new DiscoveryNode("node_" + state.nodes().getSize() + i, buildNewFakeTransportAddress(),
emptyMap(), new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), Version.CURRENT); emptyMap(), new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), Version.CURRENT);
nodesBuilder.add(node); nodesBuilder.add(node);
} }
setState(clusterService, ClusterState.builder(state).nodes(nodesBuilder)); setupMasterServiceAndNodeJoinController(ClusterState.builder(state).nodes(nodesBuilder).build());
} }
protected void assertNodesInCurrentState(List<DiscoveryNode> expectedNodes) { protected void assertNodesInCurrentState(List<DiscoveryNode> expectedNodes) {
final ClusterState state = clusterService.state(); final ClusterState state = discoveryState(masterService);
logger.info("assert for [{}] in:\n{}", expectedNodes, state); logger.info("assert for [{}] in:\n{}", expectedNodes, state);
DiscoveryNodes discoveryNodes = state.nodes(); DiscoveryNodes discoveryNodes = state.nodes();
for (DiscoveryNode node : expectedNodes) { for (DiscoveryNode node : expectedNodes) {

View File

@ -260,6 +260,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
"}"; "}";
internalCluster().startNode(); internalCluster().startNode();
ensureGreen(); // ensures that all events are processed (in particular state recovery fully completed)
logger.info("--> request node discovery stats"); logger.info("--> request node discovery stats");
NodesStatsResponse statsResponse = client().admin().cluster().prepareNodesStats().clear().setDiscovery(true).get(); NodesStatsResponse statsResponse = client().admin().cluster().prepareNodesStats().clear().setDiscovery(true).get();

View File

@ -37,12 +37,14 @@ import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.zen.PublishClusterStateActionTests.AssertingAckListener; import org.elasticsearch.discovery.zen.PublishClusterStateActionTests.AssertingAckListener;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ClusterServiceUtils;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
@ -65,6 +67,7 @@ import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static java.util.Collections.emptyMap; import static java.util.Collections.emptyMap;
@ -74,9 +77,9 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED;
import static org.elasticsearch.cluster.routing.RoutingTableTests.updateActiveAllocations; import static org.elasticsearch.cluster.routing.RoutingTableTests.updateActiveAllocations;
import static org.elasticsearch.cluster.service.MasterServiceTests.discoveryState;
import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING;
import static org.elasticsearch.discovery.zen.ZenDiscovery.shouldIgnoreOrRejectNewClusterState; import static org.elasticsearch.discovery.zen.ZenDiscovery.shouldIgnoreOrRejectNewClusterState;
import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
import static org.elasticsearch.test.ClusterServiceUtils.setState; import static org.elasticsearch.test.ClusterServiceUtils.setState;
import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.arrayWithSize;
import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.containsString;
@ -182,26 +185,31 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
DiscoveryNode masterNode = masterTransport.getLocalNode(); DiscoveryNode masterNode = masterTransport.getLocalNode();
toClose.addFirst(masterTransport); toClose.addFirst(masterTransport);
ClusterState state = ClusterStateCreationUtils.state(masterNode, masterNode, masterNode); ClusterState state = ClusterStateCreationUtils.state(masterNode, masterNode, masterNode);
// build the zen discovery and cluster service // build the zen discovery and discovery service
ClusterService masterClusterService = createClusterService(threadPool, masterNode); MasterService masterMasterService = ClusterServiceUtils.createMasterService(threadPool, masterNode);
toClose.addFirst(masterClusterService); toClose.addFirst(masterMasterService);
// TODO: clustername shouldn't be stored twice in cluster service, but for now, work around it // TODO: clustername shouldn't be stored twice in cluster service, but for now, work around it
state = ClusterState.builder(masterClusterService.getClusterName()).nodes(state.nodes()).build(); state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build();
setState(masterClusterService, state); Settings settingsWithClusterName = Settings.builder().put(settings).put(
ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterClusterService, threadPool); ClusterName.CLUSTER_NAME_SETTING.getKey(), discoveryState(masterMasterService).getClusterName().value()).build();
ZenDiscovery masterZen = buildZenDiscovery(
settingsWithClusterName,
masterTransport, masterMasterService, threadPool);
masterZen.setState(state);
toClose.addFirst(masterZen); toClose.addFirst(masterZen);
masterTransport.acceptIncomingRequests(); masterTransport.acceptIncomingRequests();
final MockTransportService otherTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null); final MockTransportService otherTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null);
otherTransport.start(); otherTransport.start();
toClose.addFirst(otherTransport); toClose.addFirst(otherTransport);
DiscoveryNode otherNode = otherTransport.getLocalNode(); DiscoveryNode otherNode = otherTransport.getLocalNode();
final ClusterState otherState = ClusterState.builder(masterClusterService.getClusterName()) final ClusterState otherState = ClusterState.builder(discoveryState(masterMasterService).getClusterName())
.nodes(DiscoveryNodes.builder().add(otherNode).localNodeId(otherNode.getId())).build(); .nodes(DiscoveryNodes.builder().add(otherNode).localNodeId(otherNode.getId())).build();
ClusterService otherClusterService = createClusterService(threadPool, masterNode); MasterService otherMasterService = ClusterServiceUtils.createMasterService(threadPool, otherNode);
toClose.addFirst(otherClusterService); toClose.addFirst(otherMasterService);
setState(otherClusterService, otherState); ZenDiscovery otherZen = buildZenDiscovery(settingsWithClusterName, otherTransport, otherMasterService, threadPool);
ZenDiscovery otherZen = buildZenDiscovery(settings, otherTransport, otherClusterService, threadPool); otherZen.setState(otherState);
toClose.addFirst(otherZen); toClose.addFirst(otherZen);
otherTransport.acceptIncomingRequests(); otherTransport.acceptIncomingRequests();
@ -210,7 +218,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
// a new cluster state with a new discovery node (we will test if the cluster state // a new cluster state with a new discovery node (we will test if the cluster state
// was updated by the presence of this node in NodesFaultDetection) // was updated by the presence of this node in NodesFaultDetection)
ClusterState newState = ClusterState.builder(masterClusterService.state()).incrementVersion().nodes( ClusterState newState = ClusterState.builder(discoveryState(masterMasterService)).incrementVersion().nodes(
DiscoveryNodes.builder(state.nodes()).add(otherNode).masterNodeId(masterNode.getId()) DiscoveryNodes.builder(state.nodes()).add(otherNode).masterNodeId(masterNode.getId())
).build(); ).build();
@ -220,7 +228,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
AssertingAckListener listener = new AssertingAckListener(newState.nodes().getSize() - 1); AssertingAckListener listener = new AssertingAckListener(newState.nodes().getSize() - 1);
expectedFDNodes = masterZen.getFaultDetectionNodes(); expectedFDNodes = masterZen.getFaultDetectionNodes();
masterZen.publish(clusterChangedEvent, listener); masterZen.publish(clusterChangedEvent, listener);
listener.await(1, TimeUnit.HOURS); listener.await(10, TimeUnit.SECONDS);
// publish was a success, update expected FD nodes based on new cluster state // publish was a success, update expected FD nodes based on new cluster state
expectedFDNodes = fdNodesForState(newState, masterNode); expectedFDNodes = fdNodesForState(newState, masterNode);
} catch (Discovery.FailedToCommitClusterStateException e) { } catch (Discovery.FailedToCommitClusterStateException e) {
@ -249,12 +257,12 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
DiscoveryNode masterNode = masterTransport.getLocalNode(); DiscoveryNode masterNode = masterTransport.getLocalNode();
toClose.addFirst(masterTransport); toClose.addFirst(masterTransport);
ClusterState state = ClusterStateCreationUtils.state(masterNode, null, masterNode); ClusterState state = ClusterStateCreationUtils.state(masterNode, null, masterNode);
// build the zen discovery and cluster service // build the zen discovery and master service for the master node
ClusterService masterClusterService = createClusterService(threadPool, masterNode); MasterService masterMasterService = ClusterServiceUtils.createMasterService(threadPool, masterNode);
toClose.addFirst(masterClusterService); toClose.addFirst(masterMasterService);
state = ClusterState.builder(masterClusterService.getClusterName()).nodes(state.nodes()).build(); state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build();
setState(masterClusterService, state); ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterMasterService, threadPool);
ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterClusterService, threadPool); masterZen.setState(state);
toClose.addFirst(masterZen); toClose.addFirst(masterZen);
masterTransport.acceptIncomingRequests(); masterTransport.acceptIncomingRequests();
@ -263,8 +271,8 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
// a new cluster state with a new discovery node (we will test if the cluster state // a new cluster state with a new discovery node (we will test if the cluster state
// was updated by the presence of this node in NodesFaultDetection) // was updated by the presence of this node in NodesFaultDetection)
ClusterState newState = ClusterState.builder(masterClusterService.state()).incrementVersion().nodes( ClusterState newState = ClusterState.builder(discoveryState(masterMasterService)).incrementVersion().nodes(
DiscoveryNodes.builder(masterClusterService.state().nodes()).masterNodeId(masterNode.getId()) DiscoveryNodes.builder(discoveryState(masterMasterService).nodes()).masterNodeId(masterNode.getId())
).build(); ).build();
@ -277,9 +285,8 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
// publish was a success, check that queue as cleared // publish was a success, check that queue as cleared
assertThat(masterZen.pendingClusterStates(), emptyArray()); assertThat(masterZen.pendingClusterStates(), emptyArray());
} catch (Discovery.FailedToCommitClusterStateException e) { } catch (Discovery.FailedToCommitClusterStateException e) {
// not successful, so the pending queue should stay // not successful, so the pending queue should be cleaned
assertThat(masterZen.pendingClusterStates(), arrayWithSize(1)); assertThat(Arrays.toString(masterZen.pendingClusterStates()), masterZen.pendingClusterStates(), arrayWithSize(0));
assertThat(masterZen.pendingClusterStates()[0].getClusterName().value(), equalTo("foreign"));
} }
} finally { } finally {
IOUtils.close(toClose); IOUtils.close(toClose);
@ -287,9 +294,12 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
} }
} }
private ZenDiscovery buildZenDiscovery(Settings settings, TransportService service, ClusterService clusterService, ThreadPool threadPool) { private ZenDiscovery buildZenDiscovery(Settings settings, TransportService service, MasterService masterService,
ThreadPool threadPool) {
ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, new NamedWriteableRegistry(ClusterModule.getNamedWriteables()), ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, new NamedWriteableRegistry(ClusterModule.getNamedWriteables()),
clusterService, Collections::emptyList); masterService, (source, clusterStateSupplier, listener) -> listener.clusterStateProcessed(source, clusterStateSupplier.get(), clusterStateSupplier.get()),
clusterSettings, Collections::emptyList);
zenDiscovery.start(); zenDiscovery.start();
return zenDiscovery; return zenDiscovery;
} }

View File

@ -19,10 +19,7 @@
package org.elasticsearch.gateway; package org.elasticsearch.gateway;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
@ -37,7 +34,7 @@ public class GatewayServiceTests extends ESTestCase {
private GatewayService createService(Settings.Builder settings) { private GatewayService createService(Settings.Builder settings) {
ClusterService clusterService = new ClusterService(Settings.builder().put("cluster.name", "GatewayServiceTests").build(), ClusterService clusterService = new ClusterService(Settings.builder().put("cluster.name", "GatewayServiceTests").build(),
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
null, () -> new DiscoveryNode(UUIDs.randomBase64UUID(), buildNewFakeTransportAddress(), Version.CURRENT)); null);
return new GatewayService(settings.build(), return new GatewayService(settings.build(),
null, clusterService, null, null, null, new NoopDiscovery(), null); null, clusterService, null, null, null, new NoopDiscovery(), null);
} }

View File

@ -32,6 +32,7 @@ import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.ESIntegTestCase.Scope;
import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.transport.TransportService;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -204,8 +205,8 @@ public class IndexLifecycleActionIT extends ESIntegTestCase {
} }
private String getLocalNodeId(String name) { private String getLocalNodeId(String name) {
Discovery discovery = internalCluster().getInstance(Discovery.class, name); TransportService transportService = internalCluster().getInstance(TransportService.class, name);
String nodeId = discovery.localNode().getId(); String nodeId = transportService.getLocalNode().getId();
assertThat(nodeId, not(nullValue())); assertThat(nodeId, not(nullValue()));
return nodeId; return nodeId;
} }

View File

@ -20,9 +20,12 @@
package org.elasticsearch.indices.store; package org.elasticsearch.indices.store;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.LocalClusterUpdateTask; import org.elasticsearch.cluster.LocalClusterUpdateTask;
import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -35,6 +38,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.service.ClusterApplierService;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -434,26 +438,35 @@ public class IndicesStoreIntegrationIT extends ESIntegTestCase {
// disable relocations when we do this, to make sure the shards are not relocated from node2 // disable relocations when we do this, to make sure the shards are not relocated from node2
// due to rebalancing, and delete its content // due to rebalancing, and delete its content
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE)).get(); client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE)).get();
internalCluster().getInstance(ClusterService.class, nonMasterNode).submitStateUpdateTask("test", new LocalClusterUpdateTask(Priority.IMMEDIATE) {
@Override ClusterApplierService clusterApplierService = internalCluster().getInstance(ClusterService.class, nonMasterNode).getClusterApplierService();
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception { ClusterState currentState = clusterApplierService.state();
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index); IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index);
for (int i = 0; i < numShards; i++) { for (int j = 0; j < numShards; j++) {
indexRoutingTableBuilder.addIndexShard( indexRoutingTableBuilder.addIndexShard(
new IndexShardRoutingTable.Builder(new ShardId(index, i)) new IndexShardRoutingTable.Builder(new ShardId(index, j))
.addShard(TestShardRouting.newShardRouting("test", i, masterId, true, ShardRoutingState.STARTED)) .addShard(TestShardRouting.newShardRouting("test", j, masterId, true, ShardRoutingState.STARTED))
.build() .build()
); );
} }
return newState(ClusterState.builder(currentState) ClusterState newState = ClusterState.builder(currentState)
.incrementVersion()
.routingTable(RoutingTable.builder().add(indexRoutingTableBuilder).build()) .routingTable(RoutingTable.builder().add(indexRoutingTableBuilder).build())
.build()); .build();
CountDownLatch latch = new CountDownLatch(1);
clusterApplierService.onNewClusterState("test", () -> newState, new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
} }
@Override @Override
public void onFailure(String source, Exception e) { public void onFailure(String source, Exception e) {
latch.countDown();
fail("Excepted proper response " + ExceptionsHelper.detailedMessage(e));
} }
}); });
latch.await();
waitNoPendingTasksOnAll(); waitNoPendingTasksOnAll();
logger.info("Checking if shards aren't removed"); logger.info("Checking if shards aren't removed");
for (int shard : node2Shards) { for (int shard : node2Shards) {

View File

@ -59,6 +59,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
assertThat(geoCentroid.getName(), equalTo(aggName)); assertThat(geoCentroid.getName(), equalTo(aggName));
GeoPoint centroid = geoCentroid.centroid(); GeoPoint centroid = geoCentroid.centroid();
assertThat(centroid, equalTo(null)); assertThat(centroid, equalTo(null));
assertEquals(0, geoCentroid.count());
} }
public void testUnmapped() throws Exception { public void testUnmapped() throws Exception {
@ -72,6 +73,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
assertThat(geoCentroid.getName(), equalTo(aggName)); assertThat(geoCentroid.getName(), equalTo(aggName));
GeoPoint centroid = geoCentroid.centroid(); GeoPoint centroid = geoCentroid.centroid();
assertThat(centroid, equalTo(null)); assertThat(centroid, equalTo(null));
assertEquals(0, geoCentroid.count());
} }
public void testPartiallyUnmapped() throws Exception { public void testPartiallyUnmapped() throws Exception {
@ -86,6 +88,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
GeoPoint centroid = geoCentroid.centroid(); GeoPoint centroid = geoCentroid.centroid();
assertThat(centroid.lat(), closeTo(singleCentroid.lat(), GEOHASH_TOLERANCE)); assertThat(centroid.lat(), closeTo(singleCentroid.lat(), GEOHASH_TOLERANCE));
assertThat(centroid.lon(), closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE)); assertThat(centroid.lon(), closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE));
assertEquals(numDocs, geoCentroid.count());
} }
public void testSingleValuedField() throws Exception { public void testSingleValuedField() throws Exception {
@ -101,6 +104,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
GeoPoint centroid = geoCentroid.centroid(); GeoPoint centroid = geoCentroid.centroid();
assertThat(centroid.lat(), closeTo(singleCentroid.lat(), GEOHASH_TOLERANCE)); assertThat(centroid.lat(), closeTo(singleCentroid.lat(), GEOHASH_TOLERANCE));
assertThat(centroid.lon(), closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE)); assertThat(centroid.lon(), closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE));
assertEquals(numDocs, geoCentroid.count());
} }
public void testSingleValueFieldGetProperty() throws Exception { public void testSingleValueFieldGetProperty() throws Exception {
@ -130,6 +134,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE)); closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE));
assertThat((double) ((InternalAggregation)global).getProperty(aggName + ".lat"), closeTo(singleCentroid.lat(), GEOHASH_TOLERANCE)); assertThat((double) ((InternalAggregation)global).getProperty(aggName + ".lat"), closeTo(singleCentroid.lat(), GEOHASH_TOLERANCE));
assertThat((double) ((InternalAggregation)global).getProperty(aggName + ".lon"), closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE)); assertThat((double) ((InternalAggregation)global).getProperty(aggName + ".lon"), closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE));
assertEquals(numDocs, (long) ((InternalAggregation) global).getProperty(aggName + ".count"));
} }
public void testMultiValuedField() throws Exception { public void testMultiValuedField() throws Exception {
@ -145,6 +150,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
GeoPoint centroid = geoCentroid.centroid(); GeoPoint centroid = geoCentroid.centroid();
assertThat(centroid.lat(), closeTo(multiCentroid.lat(), GEOHASH_TOLERANCE)); assertThat(centroid.lat(), closeTo(multiCentroid.lat(), GEOHASH_TOLERANCE));
assertThat(centroid.lon(), closeTo(multiCentroid.lon(), GEOHASH_TOLERANCE)); assertThat(centroid.lon(), closeTo(multiCentroid.lon(), GEOHASH_TOLERANCE));
assertEquals(2 * numDocs, geoCentroid.count());
} }
public void testSingleValueFieldAsSubAggToGeohashGrid() throws Exception { public void testSingleValueFieldAsSubAggToGeohashGrid() throws Exception {

View File

@ -42,8 +42,11 @@ public class InternalGeoCentroidTests extends InternalAggregationTestCase<Intern
centroid.resetLon(GeoEncodingUtils.decodeLongitude(encodedLon)); centroid.resetLon(GeoEncodingUtils.decodeLongitude(encodedLon));
int encodedLat = GeoEncodingUtils.encodeLatitude(centroid.lat()); int encodedLat = GeoEncodingUtils.encodeLatitude(centroid.lat());
centroid.resetLat(GeoEncodingUtils.decodeLatitude(encodedLat)); centroid.resetLat(GeoEncodingUtils.decodeLatitude(encodedLat));
long count = randomIntBetween(0, 1000);
return new InternalGeoCentroid("_name", centroid, 1, Collections.emptyList(), Collections.emptyMap()); if (count == 0) {
centroid = null;
}
return new InternalGeoCentroid("_name", centroid, count, Collections.emptyList(), Collections.emptyMap());
} }
@Override @Override
@ -53,14 +56,18 @@ public class InternalGeoCentroidTests extends InternalAggregationTestCase<Intern
@Override @Override
protected void assertReduced(InternalGeoCentroid reduced, List<InternalGeoCentroid> inputs) { protected void assertReduced(InternalGeoCentroid reduced, List<InternalGeoCentroid> inputs) {
GeoPoint expected = new GeoPoint(0, 0); double lonSum = 0;
int i = 0; double latSum = 0;
int totalCount = 0;
for (InternalGeoCentroid input : inputs) { for (InternalGeoCentroid input : inputs) {
expected.reset(expected.lat() + (input.centroid().lat() - expected.lat()) / (i+1), if (input.count() > 0) {
expected.lon() + (input.centroid().lon() - expected.lon()) / (i+1)); lonSum += (input.count() * input.centroid().getLon());
i++; latSum += (input.count() * input.centroid().getLat());
} }
assertEquals(expected.getLat(), reduced.centroid().getLat(), 1E-5D); totalCount += input.count();
assertEquals(expected.getLon(), reduced.centroid().getLon(), 1E-5D); }
assertEquals(latSum/totalCount, reduced.centroid().getLat(), 1E-5D);
assertEquals(lonSum/totalCount, reduced.centroid().getLon(), 1E-5D);
assertEquals(totalCount, reduced.count());
} }
} }

View File

@ -18,19 +18,9 @@
*/ */
package org.elasticsearch.snapshots; package org.elasticsearch.snapshots;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.cluster.service.PendingClusterTask;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
@ -47,13 +37,10 @@ import java.nio.file.attribute.BasicFileAttributes;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Predicate;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase { public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
@ -180,121 +167,4 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
public static void unblockNode(final String repository, final String node) { public static void unblockNode(final String repository, final String node) {
((MockRepository)internalCluster().getInstance(RepositoriesService.class, node).repository(repository)).unblock(); ((MockRepository)internalCluster().getInstance(RepositoriesService.class, node).repository(repository)).unblock();
} }
protected void assertBusyPendingTasks(final String taskPrefix, final int expectedCount) throws Exception {
assertBusy(new Runnable() {
@Override
public void run() {
PendingClusterTasksResponse tasks = client().admin().cluster().preparePendingClusterTasks().get();
int count = 0;
for(PendingClusterTask task : tasks) {
if (task.getSource().toString().startsWith(taskPrefix)) {
count++;
}
}
assertThat(count, greaterThanOrEqualTo(expectedCount));
}
}, 1, TimeUnit.MINUTES);
}
/**
* Cluster state task that blocks waits for the blockOn task to show up and then blocks execution not letting
* any cluster state update task to be performed unless they have priority higher then passThroughPriority.
*
* This class is useful to testing of cluster state update task batching for lower priority tasks.
*/
protected class BlockingClusterStateListener implements ClusterStateListener {
private final Predicate<ClusterChangedEvent> blockOn;
private final Predicate<ClusterChangedEvent> countOn;
private final ClusterService clusterService;
private final CountDownLatch latch;
private final Priority passThroughPriority;
private int count;
private boolean timedOut;
private final TimeValue timeout;
private long stopWaitingAt = -1;
public BlockingClusterStateListener(ClusterService clusterService, String blockOn, String countOn, Priority passThroughPriority) {
// Waiting for the 70 seconds here to make sure that the last check at 65 sec mark in assertBusyPendingTasks has a chance
// to finish before we timeout on the cluster state block. Otherwise the last check in assertBusyPendingTasks kicks in
// after the cluster state block clean up takes place and it's assert doesn't reflect the actual failure
this(clusterService, blockOn, countOn, passThroughPriority, TimeValue.timeValueSeconds(70));
}
public BlockingClusterStateListener(ClusterService clusterService, final String blockOn, final String countOn, Priority passThroughPriority, TimeValue timeout) {
this.clusterService = clusterService;
this.blockOn = clusterChangedEvent -> clusterChangedEvent.source().startsWith(blockOn);
this.countOn = clusterChangedEvent -> clusterChangedEvent.source().startsWith(countOn);
this.latch = new CountDownLatch(1);
this.passThroughPriority = passThroughPriority;
this.timeout = timeout;
}
public void unblock() {
latch.countDown();
}
@Override
public void clusterChanged(ClusterChangedEvent event) {
if (blockOn.test(event)) {
logger.info("blocking cluster state tasks on [{}]", event.source());
assert stopWaitingAt < 0; // Make sure we are the first time here
stopWaitingAt = System.currentTimeMillis() + timeout.getMillis();
addBlock();
}
if (countOn.test(event)) {
count++;
}
}
private void addBlock() {
// We should block after this task - add blocking cluster state update task
clusterService.submitStateUpdateTask("test_block", new ClusterStateUpdateTask(passThroughPriority) {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
while(System.currentTimeMillis() < stopWaitingAt) {
for (PendingClusterTask task : clusterService.pendingTasks()) {
if (task.getSource().string().equals("test_block") == false && passThroughPriority.sameOrAfter(task.getPriority())) {
// There are other higher priority tasks in the queue and let them pass through and then set the block again
logger.info("passing through cluster state task {}", task.getSource());
addBlock();
return currentState;
}
}
try {
logger.info("waiting....");
if (latch.await(Math.min(100, timeout.millis()), TimeUnit.MILLISECONDS)){
// Done waiting - unblock
logger.info("unblocked");
return currentState;
}
logger.info("done waiting....");
} catch (InterruptedException ex) {
logger.info("interrupted....");
Thread.currentThread().interrupt();
return currentState;
}
}
timedOut = true;
return currentState;
}
@Override
public void onFailure(String source, Exception e) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to execute [{}]", source), e);
}
});
}
public int count() {
return count;
}
public boolean timedOut() {
return timedOut;
}
}
} }

View File

@ -724,14 +724,9 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
} }
public void testMasterShutdownDuringSnapshot() throws Exception { public void testMasterShutdownDuringSnapshot() throws Exception {
Settings masterSettings = Settings.builder().put(Node.NODE_DATA_SETTING.getKey(), false).build();
Settings dataSettings = Settings.builder().put(Node.NODE_MASTER_SETTING.getKey(), false).build();
logger.info("--> starting two master nodes and two data nodes"); logger.info("--> starting two master nodes and two data nodes");
internalCluster().startNode(masterSettings); internalCluster().startMasterOnlyNodes(2);
internalCluster().startNode(masterSettings); internalCluster().startDataOnlyNodes(2);
internalCluster().startNode(dataSettings);
internalCluster().startNode(dataSettings);
final Client client = client(); final Client client = client();
@ -758,35 +753,17 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
final int numberOfShards = getNumShards("test-idx").numPrimaries; final int numberOfShards = getNumShards("test-idx").numPrimaries;
logger.info("number of shards: {}", numberOfShards); logger.info("number of shards: {}", numberOfShards);
final ClusterService clusterService = internalCluster().clusterService(internalCluster().getMasterName());
BlockingClusterStateListener snapshotListener = new BlockingClusterStateListener(clusterService, "update_snapshot [", "update snapshot state", Priority.HIGH);
try {
clusterService.addListener(snapshotListener);
logger.info("--> snapshot");
dataNodeClient().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get(); dataNodeClient().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get();
// Await until some updates are in pending state.
assertBusyPendingTasks("update snapshot state", 1);
logger.info("--> stopping master node"); logger.info("--> stopping master node");
internalCluster().stopCurrentMasterNode(); internalCluster().stopCurrentMasterNode();
logger.info("--> unblocking snapshot execution");
snapshotListener.unblock();
} finally {
clusterService.removeListener(snapshotListener);
}
logger.info("--> wait until the snapshot is done"); logger.info("--> wait until the snapshot is done");
assertBusy(new Runnable() { assertBusy(() -> {
@Override
public void run() {
GetSnapshotsResponse snapshotsStatusResponse = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get(); GetSnapshotsResponse snapshotsStatusResponse = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get();
SnapshotInfo snapshotInfo = snapshotsStatusResponse.getSnapshots().get(0); SnapshotInfo snapshotInfo = snapshotsStatusResponse.getSnapshots().get(0);
assertTrue(snapshotInfo.state().completed()); assertTrue(snapshotInfo.state().completed());
}
}, 1, TimeUnit.MINUTES); }, 1, TimeUnit.MINUTES);
logger.info("--> verify that snapshot was succesful"); logger.info("--> verify that snapshot was succesful");

View File

@ -2286,59 +2286,6 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
return awaitBusy(() -> client().admin().cluster().prepareHealth(index).execute().actionGet().getRelocatingShards() > 0, timeout.millis(), TimeUnit.MILLISECONDS); return awaitBusy(() -> client().admin().cluster().prepareHealth(index).execute().actionGet().getRelocatingShards() > 0, timeout.millis(), TimeUnit.MILLISECONDS);
} }
public void testBatchingShardUpdateTask() throws Exception {
final Client client = client();
logger.info("--> creating repository");
assertAcked(client.admin().cluster().preparePutRepository("test-repo")
.setType("fs").setSettings(Settings.builder()
.put("location", randomRepoPath())
.put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
assertAcked(prepareCreate("test-idx", 0, Settings.builder().put("number_of_shards", between(1, 10))
.put("number_of_replicas", 0)));
ensureGreen();
logger.info("--> indexing some data");
final int numdocs = randomIntBetween(10, 100);
IndexRequestBuilder[] builders = new IndexRequestBuilder[numdocs];
for (int i = 0; i < builders.length; i++) {
builders[i] = client().prepareIndex("test-idx", "type1", Integer.toString(i)).setSource("field1", "bar " + i);
}
indexRandom(true, builders);
flushAndRefresh();
final int numberOfShards = getNumShards("test-idx").numPrimaries;
logger.info("number of shards: {}", numberOfShards);
final ClusterService clusterService = internalCluster().clusterService(internalCluster().getMasterName());
BlockingClusterStateListener snapshotListener = new BlockingClusterStateListener(clusterService, "update_snapshot [", "update snapshot state", Priority.HIGH);
try {
clusterService.addListener(snapshotListener);
logger.info("--> snapshot");
ListenableActionFuture<CreateSnapshotResponse> snapshotFuture = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").execute();
// Await until shard updates are in pending state.
assertBusyPendingTasks("update snapshot state", numberOfShards);
snapshotListener.unblock();
// Check that the snapshot was successful
CreateSnapshotResponse createSnapshotResponse = snapshotFuture.actionGet();
assertEquals(SnapshotState.SUCCESS, createSnapshotResponse.getSnapshotInfo().state());
assertEquals(numberOfShards, createSnapshotResponse.getSnapshotInfo().totalShards());
assertEquals(numberOfShards, createSnapshotResponse.getSnapshotInfo().successfulShards());
} finally {
clusterService.removeListener(snapshotListener);
}
// Check that we didn't timeout
assertFalse(snapshotListener.timedOut());
// Check that cluster state update task was called only once
assertEquals(1, snapshotListener.count());
}
public void testSnapshotName() throws Exception { public void testSnapshotName() throws Exception {
final Client client = client(); final Client client = client();

View File

@ -19,27 +19,15 @@
package org.elasticsearch.test; package org.elasticsearch.test;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.component.LifecycleListener; import org.elasticsearch.common.component.LifecycleListener;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.DiscoveryStats; import org.elasticsearch.discovery.DiscoveryStats;
public class NoopDiscovery implements Discovery { public class NoopDiscovery implements Discovery {
@Override
public DiscoveryNode localNode() {
return null;
}
@Override
public String nodeDescription() {
return null;
}
@Override @Override
public void setAllocationService(AllocationService allocationService) { public void setAllocationService(AllocationService allocationService) {
@ -51,12 +39,17 @@ public class NoopDiscovery implements Discovery {
} }
@Override @Override
public DiscoveryStats stats() { public ClusterState getInitialClusterState() {
return null; return null;
} }
@Override @Override
public DiscoverySettings getDiscoverySettings() { public ClusterState clusterState() {
return null;
}
@Override
public DiscoveryStats stats() {
return null; return null;
} }

View File

@ -492,60 +492,6 @@ public class UpdateIT extends ESIntegTestCase {
assertThat(updateResponse.getGetResult().sourceAsMap().get("extra").toString(), equalTo("foo")); assertThat(updateResponse.getGetResult().sourceAsMap().get("extra").toString(), equalTo("foo"));
} }
public void testVersionedUpdate() throws Exception {
assertAcked(prepareCreate("test").addAlias(new Alias("alias")));
ensureGreen();
index("test", "type", "1", "text", "value"); // version is now 1
assertThrows(client().prepareUpdate(indexOrAlias(), "type", "1")
.setScript(new Script(ScriptType.INLINE, "put_values", "", Collections.singletonMap("text", "v2"))).setVersion(2)
.execute(),
VersionConflictEngineException.class);
client().prepareUpdate(indexOrAlias(), "type", "1")
.setScript(new Script(ScriptType.INLINE, "put_values", "", Collections.singletonMap("text", "v2"))).setVersion(1).get();
assertThat(client().prepareGet("test", "type", "1").get().getVersion(), equalTo(2L));
// and again with a higher version..
client().prepareUpdate(indexOrAlias(), "type", "1")
.setScript(new Script(ScriptType.INLINE, "put_values", "", Collections.singletonMap("text", "v3"))).setVersion(2).get();
assertThat(client().prepareGet("test", "type", "1").get().getVersion(), equalTo(3L));
// after delete
client().prepareDelete("test", "type", "1").get();
assertThrows(client().prepareUpdate("test", "type", "1")
.setScript(new Script(ScriptType.INLINE, "put_values", "", Collections.singletonMap("text", "v2"))).setVersion(3)
.execute(),
DocumentMissingException.class);
// external versioning
client().prepareIndex("test", "type", "2").setSource("text", "value").setVersion(10).setVersionType(VersionType.EXTERNAL).get();
assertThrows(client().prepareUpdate(indexOrAlias(), "type", "2")
.setScript(new Script(ScriptType.INLINE, "put_values", "", Collections.singletonMap("text", "v2"))).setVersion(2)
.setVersionType(VersionType.EXTERNAL).execute(),
ActionRequestValidationException.class);
GetResponse get = get("test", "type", "2");
assertThat(get.getVersion(), equalTo(10L));
assertThat((String) get.getSource().get("text"), equalTo("value"));
// upserts - the combination with versions is a bit weird. Test are here to ensure we do not change our behavior unintentionally
// With internal versions, tt means "if object is there with version X, update it or explode. If it is not there, index.
client().prepareUpdate(indexOrAlias(), "type", "3")
.setScript(new Script(ScriptType.INLINE, "put_values", "", Collections.singletonMap("text", "v2")))
.setVersion(10).setUpsert("{ \"text\": \"v0\" }", XContentType.JSON).get();
get = get("test", "type", "3");
assertThat(get.getVersion(), equalTo(1L));
assertThat((String) get.getSource().get("text"), equalTo("v0"));
// retry on conflict is rejected:
assertThrows(client().prepareUpdate(indexOrAlias(), "type", "1").setVersion(10).setRetryOnConflict(5), ActionRequestValidationException.class);
}
public void testIndexAutoCreation() throws Exception { public void testIndexAutoCreation() throws Exception {
UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1") UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1")
.setUpsert(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject()) .setUpsert(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject())

View File

@ -62,7 +62,8 @@ The response for the above aggregation:
"location": { "location": {
"lat": 51.00982963107526, "lat": 51.00982963107526,
"lon": 3.9662130922079086 "lon": 3.9662130922079086
} },
"count": 6
} }
} }
} }
@ -114,7 +115,8 @@ The response for the above aggregation:
"location": { "location": {
"lat": 52.371655656024814, "lat": 52.371655656024814,
"lon": 4.909563297405839 "lon": 4.909563297405839
} },
"count": 3
} }
}, },
{ {
@ -124,7 +126,8 @@ The response for the above aggregation:
"location": { "location": {
"lat": 48.86055548675358, "lat": 48.86055548675358,
"lon": 2.3316944623366 "lon": 2.3316944623366
} },
"count": 2
} }
}, },
{ {
@ -134,7 +137,8 @@ The response for the above aggregation:
"location": { "location": {
"lat": 51.22289997059852, "lat": 51.22289997059852,
"lon": 4.40519998781383 "lon": 4.40519998781383
} },
"count": 1
} }
} }
] ]

View File

@ -54,5 +54,6 @@ alias4 test1 - 2 1,2
The output shows that `alias2` has configured a filter, and specific routing The output shows that `alias2` has configured a filter, and specific routing
configurations in `alias3` and `alias4`. configurations in `alias3` and `alias4`.
If you only want to get information about a single alias, you can specify If you only want to get information about specific aliases, you can specify
the alias in the URL, for example `/_cat/aliases/alias1`. the aliases in comma-delimited format as a URL parameter, e.g.,
/_cat/aliases/aliases/alias1,alias2.

View File

@ -16,7 +16,8 @@ GET /_cat/health?v
epoch timestamp cluster status node.total node.data shards pri relo init unassign pending_tasks max_task_wait_time active_shards_percent epoch timestamp cluster status node.total node.data shards pri relo init unassign pending_tasks max_task_wait_time active_shards_percent
1475871424 16:17:04 elasticsearch green 1 1 5 5 0 0 0 0 - 100.0% 1475871424 16:17:04 elasticsearch green 1 1 5 5 0 0 0 0 - 100.0%
-------------------------------------------------- --------------------------------------------------
// TESTRESPONSE[s/1475871424 16:17:04/\\d+ \\d+:\\d+:\\d+/ s/elasticsearch/[^ ]+/ s/0 -/\\d+ -/ _cat] // TESTRESPONSE[s/1475871424 16:17:04/\\d+ \\d+:\\d+:\\d+/]
// TESTRESPONSE[s/elasticsearch/[^ ]+/ s/0 -/\\d+ (-|\\d+(\\.\\d+)?[ms]+)/ _cat]
It has one option `ts` to disable the timestamping: It has one option `ts` to disable the timestamping:
@ -34,7 +35,7 @@ which looks like:
cluster status node.total node.data shards pri relo init unassign pending_tasks max_task_wait_time active_shards_percent cluster status node.total node.data shards pri relo init unassign pending_tasks max_task_wait_time active_shards_percent
elasticsearch green 1 1 5 5 0 0 0 0 - 100.0% elasticsearch green 1 1 5 5 0 0 0 0 - 100.0%
-------------------------------------------------- --------------------------------------------------
// TESTRESPONSE[s/elasticsearch/[^ ]+/ s/0 -/\\d+ -/ _cat] // TESTRESPONSE[s/elasticsearch/[^ ]+/ s/0 -/\\d+ (-|\\d+(\\.\\d+)?[ms]+)/ _cat]
A common use of this command is to verify the health is consistent A common use of this command is to verify the health is consistent
across nodes: across nodes:

View File

@ -5,3 +5,7 @@
Document modification operations may no longer specify the `version_type` of Document modification operations may no longer specify the `version_type` of
`force` to override any previous version checks. `force` to override any previous version checks.
==== <<upserts>> no longer support versions
Adding a `version` to an upsert request is no longer supported.

View File

@ -148,6 +148,55 @@ An important default is that the `_source` returned in hits inside `inner_hits`
So in the above example only the comment part is returned per nested hit and not the entire source of the top level So in the above example only the comment part is returned per nested hit and not the entire source of the top level
document that contained the comment. document that contained the comment.
[[nested-inner-hits-source]]
==== Nested inner hits and _source
Nested document don't have a `_source` field, because the entire source of document is stored with the root document under
its `_source` field. To include the source of just the nested document, the source of the root document is parsed and just
the relevant bit for the nested document is included as source in the inner hit. Doing this for each matching nested document
has an impact on the time it takes to execute the entire search request, especially when `size` and the inner hits' `size`
are set higher than the default. To avoid the relative expensive source extraction for nested inner hits, one can disable
including the source and solely rely on stored fields.
Enabled stored field for fields under the nested object field in your mapping:
[source,js]
--------------------------------------------------
{
"properties": {
"comment": {
"type": "comments",
"properties" : {
"message" : {
"type" : "text",
"store" : true
}
}
}
}
}
--------------------------------------------------
Disable including source and include specific stored fields in the inner hits definition:
[source,js]
--------------------------------------------------
{
"query" : {
"nested" : {
"path" : "comments",
"query" : {
"match" : {"comments.message" : "[actual query]"}
},
"inner_hits" : {
"_source" : false,
"stored_fields" : ["comments.text"]
}
}
}
}
--------------------------------------------------
[[hierarchical-nested-inner-hits]] [[hierarchical-nested-inner-hits]]
==== Hierarchical levels of nested object fields and inner hits. ==== Hierarchical levels of nested object fields and inner hits.

View File

@ -19,22 +19,16 @@
package org.elasticsearch.plugin.discovery.azure.classic; package org.elasticsearch.plugin.discovery.azure.classic;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cloud.azure.classic.management.AzureComputeService; import org.elasticsearch.cloud.azure.classic.management.AzureComputeService;
import org.elasticsearch.cloud.azure.classic.management.AzureComputeServiceImpl; import org.elasticsearch.cloud.azure.classic.management.AzureComputeServiceImpl;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.common.Strings; import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
@ -42,12 +36,17 @@ import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.azure.classic.AzureUnicastHostsProvider; import org.elasticsearch.discovery.azure.classic.AzureUnicastHostsProvider;
import org.elasticsearch.discovery.zen.UnicastHostsProvider; import org.elasticsearch.discovery.zen.UnicastHostsProvider;
import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.discovery.zen.ZenPing;
import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.plugins.DiscoveryPlugin;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
public class AzureDiscoveryPlugin extends Plugin implements DiscoveryPlugin { public class AzureDiscoveryPlugin extends Plugin implements DiscoveryPlugin {
public static final String AZURE = "azure"; public static final String AZURE = "azure";
@ -76,10 +75,12 @@ public class AzureDiscoveryPlugin extends Plugin implements DiscoveryPlugin {
@Override @Override
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) { MasterService masterService, ClusterApplier clusterApplier,
ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
// this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider // this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider
return Collections.singletonMap(AZURE, () -> return Collections.singletonMap(AZURE, () ->
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider)); new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
clusterSettings, hostsProvider));
} }
@Override @Override

View File

@ -20,6 +20,30 @@
package org.elasticsearch.discovery.ec2; package org.elasticsearch.discovery.ec2;
import com.amazonaws.util.json.Jackson; import com.amazonaws.util.json.Jackson;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.discovery.zen.UnicastHostsProvider;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.DiscoveryPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
@ -37,28 +61,6 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.Supplier; import java.util.function.Supplier;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.zen.UnicastHostsProvider;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.DiscoveryPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
public class Ec2DiscoveryPlugin extends Plugin implements DiscoveryPlugin, Closeable { public class Ec2DiscoveryPlugin extends Plugin implements DiscoveryPlugin, Closeable {
private static Logger logger = Loggers.getLogger(Ec2DiscoveryPlugin.class); private static Logger logger = Loggers.getLogger(Ec2DiscoveryPlugin.class);
@ -95,10 +97,12 @@ public class Ec2DiscoveryPlugin extends Plugin implements DiscoveryPlugin, Close
@Override @Override
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) { MasterService masterService, ClusterApplier clusterApplier,
ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
// this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider // this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider
return Collections.singletonMap(EC2, () -> return Collections.singletonMap(EC2, () ->
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider)); new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
clusterSettings, hostsProvider));
} }
@Override @Override

View File

@ -24,28 +24,25 @@ import com.google.api.client.util.ClassInfo;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.SetOnce;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.cloud.gce.GceInstancesService; import org.elasticsearch.cloud.gce.GceInstancesService;
import org.elasticsearch.cloud.gce.GceInstancesServiceImpl; import org.elasticsearch.cloud.gce.GceInstancesServiceImpl;
import org.elasticsearch.cloud.gce.GceMetadataService; import org.elasticsearch.cloud.gce.GceMetadataService;
import org.elasticsearch.cloud.gce.GceModule;
import org.elasticsearch.cloud.gce.network.GceNameResolver; import org.elasticsearch.cloud.gce.network.GceNameResolver;
import org.elasticsearch.cloud.gce.util.Access; import org.elasticsearch.cloud.gce.util.Access;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.discovery.gce.GceUnicastHostsProvider; import org.elasticsearch.discovery.gce.GceUnicastHostsProvider;
import org.elasticsearch.discovery.zen.UnicastHostsProvider; import org.elasticsearch.discovery.zen.UnicastHostsProvider;
import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.discovery.zen.ZenPing;
import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.plugins.DiscoveryPlugin;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -53,11 +50,7 @@ import org.elasticsearch.transport.TransportService;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -92,10 +85,12 @@ public class GceDiscoveryPlugin extends Plugin implements DiscoveryPlugin, Close
@Override @Override
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) { MasterService masterService, ClusterApplier clusterApplier,
ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
// this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider // this is for backcompat with pre 5.1, where users would set discovery.type to use ec2 hosts provider
return Collections.singletonMap(GCE, () -> return Collections.singletonMap(GCE, () ->
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider)); new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
clusterSettings, hostsProvider));
} }
@Override @Override

View File

@ -126,6 +126,52 @@
- match: - match:
$body: / (^|\n)test_2 .+ \n/ $body: / (^|\n)test_2 .+ \n/
---
"Multiple alias names":
- skip:
version: " - 5.99.99"
reason: multiple aliases are supported only from 6.0.0 on
- do:
indices.create:
index: test
- do:
indices.create:
index: test2
- do:
indices.create:
index: test3
- do:
indices.put_alias:
index: test
name: foo
- do:
indices.put_alias:
index: test2
name: bar
- do:
indices.put_alias:
index: test3
name: baz
- do:
cat.aliases:
name: foo,bar
v: true
h: [alias, index]
s: [index]
- match:
$body: |
/^ alias \s+ index \n
foo \s+ test \n
bar \s+ test2
$/
--- ---
"Column headers": "Column headers":

View File

@ -18,82 +18,90 @@
*/ */
package org.elasticsearch.test; package org.elasticsearch.test;
import org.apache.logging.log4j.core.util.Throwables;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.LocalClusterUpdateTask; import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.cluster.service.ClusterApplierService;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.Discovery.AckListener;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer;
import static junit.framework.TestCase.fail; import static junit.framework.TestCase.fail;
public class ClusterServiceUtils { public class ClusterServiceUtils {
public static ClusterService createClusterService(ThreadPool threadPool) { public static MasterService createMasterService(ThreadPool threadPool, ClusterState initialClusterState) {
DiscoveryNode discoveryNode = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(), MasterService masterService = new MasterService(Settings.EMPTY, threadPool);
EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT); AtomicReference<ClusterState> clusterStateRef = new AtomicReference<>(initialClusterState);
return createClusterService(threadPool, discoveryNode); masterService.setClusterStatePublisher((event, ackListener) -> clusterStateRef.set(event.state()));
masterService.setClusterStateSupplier(clusterStateRef::get);
masterService.start();
return masterService;
} }
public static ClusterService createClusterService(ThreadPool threadPool, DiscoveryNode localNode) { public static MasterService createMasterService(ThreadPool threadPool, DiscoveryNode localNode) {
return createClusterService(Settings.EMPTY, threadPool, localNode); ClusterState initialClusterState = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName()))
.nodes(DiscoveryNodes.builder()
.add(localNode)
.localNodeId(localNode.getId())
.masterNodeId(localNode.getId()))
.blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build();
return createMasterService(threadPool, initialClusterState);
} }
public static ClusterService createClusterService(Settings settings, ThreadPool threadPool, DiscoveryNode localNode) { public static void setState(ClusterApplierService executor, ClusterState clusterState) {
ClusterService clusterService = new ClusterService(
Settings.builder().put("cluster.name", "ClusterServiceTests").put(settings).build(),
new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
threadPool, () -> localNode);
clusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
@Override
public void connectToNodes(DiscoveryNodes discoveryNodes) {
// skip
}
@Override
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
// skip
}
});
clusterService.setClusterStatePublisher((event, ackListener) -> {
});
clusterService.setDiscoverySettings(new DiscoverySettings(Settings.EMPTY,
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)));
clusterService.start();
final DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes());
nodes.masterNodeId(clusterService.localNode().getId());
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
return clusterService;
}
public static ClusterService createClusterService(ClusterState initialState, ThreadPool threadPool) {
ClusterService clusterService = createClusterService(threadPool);
setState(clusterService, initialState);
return clusterService;
}
public static void setState(ClusterService clusterService, ClusterState.Builder clusterStateBuilder) {
setState(clusterService, clusterStateBuilder.build());
}
public static void setState(ClusterService clusterService, ClusterState clusterState) {
CountDownLatch latch = new CountDownLatch(1); CountDownLatch latch = new CountDownLatch(1);
clusterService.submitStateUpdateTask("test setting state", new LocalClusterUpdateTask() { AtomicReference<Exception> exception = new AtomicReference<>();
executor.onNewClusterState("test setting state",
() -> ClusterState.builder(clusterState).version(clusterState.version() + 1).build(), new ClusterStateTaskListener() {
@Override @Override
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception { public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
exception.set(e);
latch.countDown();
}
});
try {
latch.await();
if (exception.get() != null) {
Throwables.rethrow(exception.get());
}
} catch (InterruptedException e) {
throw new ElasticsearchException("unexpected exception", e);
}
}
public static void setState(MasterService executor, ClusterState clusterState) {
CountDownLatch latch = new CountDownLatch(1);
executor.submitStateUpdateTask("test setting state", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
// make sure we increment versions as listener may depend on it for change // make sure we increment versions as listener may depend on it for change
return newState(ClusterState.builder(clusterState).version(currentState.version() + 1).build()); return ClusterState.builder(clusterState).build();
} }
@Override @Override
@ -112,4 +120,88 @@ public class ClusterServiceUtils {
throw new ElasticsearchException("unexpected interruption", e); throw new ElasticsearchException("unexpected interruption", e);
} }
} }
public static ClusterService createClusterService(ThreadPool threadPool) {
DiscoveryNode discoveryNode = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT);
return createClusterService(threadPool, discoveryNode);
}
public static ClusterService createClusterService(ThreadPool threadPool, DiscoveryNode localNode) {
return createClusterService(threadPool, localNode, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
}
public static ClusterService createClusterService(ThreadPool threadPool, DiscoveryNode localNode, ClusterSettings clusterSettings) {
ClusterService clusterService = new ClusterService(Settings.builder().put("cluster.name", "ClusterServiceTests").build(),
clusterSettings, threadPool);
clusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
@Override
public void connectToNodes(DiscoveryNodes discoveryNodes) {
// skip
}
@Override
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
// skip
}
});
ClusterState initialClusterState = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName()))
.nodes(DiscoveryNodes.builder()
.add(localNode)
.localNodeId(localNode.getId())
.masterNodeId(localNode.getId()))
.blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build();
clusterService.getClusterApplierService().setInitialState(initialClusterState);
clusterService.getMasterService().setClusterStatePublisher(
createClusterStatePublisher(clusterService.getClusterApplierService()));
clusterService.getMasterService().setClusterStateSupplier(clusterService.getClusterApplierService()::state);
clusterService.start();
return clusterService;
}
public static BiConsumer<ClusterChangedEvent, AckListener> createClusterStatePublisher(ClusterApplier clusterApplier) {
return (event, ackListener) -> {
CountDownLatch latch = new CountDownLatch(1);
AtomicReference<Exception> ex = new AtomicReference<>();
clusterApplier.onNewClusterState("mock_publish_to_self[" + event.source() + "]", () -> event.state(),
new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
latch.countDown();
}
@Override
public void onFailure(String source, Exception e) {
ex.set(e);
latch.countDown();
}
}
);
try {
latch.await();
} catch (InterruptedException e) {
Throwables.rethrow(e);
}
if (ex.get() != null) {
Throwables.rethrow(ex.get());
}
};
}
public static ClusterService createClusterService(ClusterState initialState, ThreadPool threadPool) {
ClusterService clusterService = createClusterService(threadPool);
setState(clusterService, initialState);
return clusterService;
}
public static void setState(ClusterService clusterService, ClusterState.Builder clusterStateBuilder) {
setState(clusterService, clusterStateBuilder.build());
}
/**
* Sets the state on the cluster applier service
*/
public static void setState(ClusterService clusterService, ClusterState clusterState) {
setState(clusterService.getClusterApplierService(), clusterState);
}
} }

View File

@ -570,7 +570,8 @@ public abstract class ESIntegTestCase extends ESTestCase {
final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery; final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery;
assertBusy(() -> { assertBusy(() -> {
final ClusterState[] states = zenDiscovery.pendingClusterStates(); final ClusterState[] states = zenDiscovery.pendingClusterStates();
assertThat(zenDiscovery.localNode().getName() + " still having pending states:\n" + assertThat(zenDiscovery.clusterState().nodes().getLocalNode().getName() +
" still having pending states:\n" +
Stream.of(states).map(ClusterState::toString).collect(Collectors.joining("\n")), Stream.of(states).map(ClusterState::toString).collect(Collectors.joining("\n")),
states, emptyArray()); states, emptyArray());
}); });

View File

@ -24,8 +24,10 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.Supplier; import java.util.function.Supplier;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterApplier;
import org.elasticsearch.cluster.service.MasterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
@ -56,9 +58,11 @@ public class TestZenDiscovery extends ZenDiscovery {
@Override @Override
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, NamedWriteableRegistry namedWriteableRegistry,
ClusterService clusterService, UnicastHostsProvider hostsProvider) { MasterService masterService, ClusterApplier clusterApplier,
ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
return Collections.singletonMap("test-zen", return Collections.singletonMap("test-zen",
() -> new TestZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider)); () -> new TestZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService,
clusterApplier, clusterSettings, hostsProvider));
} }
@Override @Override
@ -73,9 +77,10 @@ public class TestZenDiscovery extends ZenDiscovery {
} }
private TestZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService, private TestZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry, ClusterService clusterService, NamedWriteableRegistry namedWriteableRegistry, MasterService masterService,
UnicastHostsProvider hostsProvider) { ClusterApplier clusterApplier, ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
super(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider); super(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier, clusterSettings,
hostsProvider);
} }
@Override @Override

View File

@ -18,8 +18,7 @@
*/ */
package org.elasticsearch.test.disruption; package org.elasticsearch.test.disruption;
import org.elasticsearch.cluster.ClusterState; import org.apache.logging.log4j.core.util.Throwables;
import org.elasticsearch.cluster.LocalClusterUpdateTask;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
@ -58,23 +57,19 @@ public class BlockClusterStateProcessing extends SingleNodeDisruption {
boolean success = disruptionLatch.compareAndSet(null, new CountDownLatch(1)); boolean success = disruptionLatch.compareAndSet(null, new CountDownLatch(1));
assert success : "startDisrupting called without waiting on stopDisrupting to complete"; assert success : "startDisrupting called without waiting on stopDisrupting to complete";
final CountDownLatch started = new CountDownLatch(1); final CountDownLatch started = new CountDownLatch(1);
clusterService.submitStateUpdateTask("service_disruption_block", new LocalClusterUpdateTask(Priority.IMMEDIATE) { clusterService.getClusterApplierService().runOnApplierThread("service_disruption_block",
currentState -> {
@Override
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
started.countDown(); started.countDown();
CountDownLatch latch = disruptionLatch.get(); CountDownLatch latch = disruptionLatch.get();
if (latch != null) { if (latch != null) {
try {
latch.await(); latch.await();
} catch (InterruptedException e) {
Throwables.rethrow(e);
} }
return unchanged();
} }
}, (source, e) -> logger.error("unexpected error during disruption", e),
@Override Priority.IMMEDIATE);
public void onFailure(String source, Exception e) {
logger.error("unexpected error during disruption", e);
}
});
try { try {
started.await(); started.await();
} catch (InterruptedException e) { } catch (InterruptedException e) {

View File

@ -18,8 +18,7 @@
*/ */
package org.elasticsearch.test.disruption; package org.elasticsearch.test.disruption;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.cluster.LocalClusterUpdateTask;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
@ -102,10 +101,9 @@ public class SlowClusterStateProcessing extends SingleNodeDisruption {
return false; return false;
} }
final AtomicBoolean stopped = new AtomicBoolean(false); final AtomicBoolean stopped = new AtomicBoolean(false);
clusterService.submitStateUpdateTask("service_disruption_delay", new LocalClusterUpdateTask(Priority.IMMEDIATE) { clusterService.getClusterApplierService().runOnApplierThread("service_disruption_delay",
currentState -> {
@Override try {
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
long count = duration.millis() / 200; long count = duration.millis() / 200;
// wait while checking for a stopped // wait while checking for a stopped
for (; count > 0 && !stopped.get(); count--) { for (; count > 0 && !stopped.get(); count--) {
@ -115,14 +113,11 @@ public class SlowClusterStateProcessing extends SingleNodeDisruption {
Thread.sleep(duration.millis() % 200); Thread.sleep(duration.millis() % 200);
} }
countDownLatch.countDown(); countDownLatch.countDown();
return unchanged(); } catch (InterruptedException e) {
ExceptionsHelper.reThrowIfNotNull(e);
} }
}, (source, e) -> countDownLatch.countDown(),
@Override Priority.IMMEDIATE);
public void onFailure(String source, Exception e) {
countDownLatch.countDown();
}
});
try { try {
countDownLatch.await(); countDownLatch.await();
} catch (InterruptedException e) { } catch (InterruptedException e) {