Merge branch 'master' into feature/client_aggs_parsing
This commit is contained in:
commit
db07a34718
|
@ -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[/\\]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[/\\]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[/\\]FaultDetection.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]zen[/\\]MasterFaultDetection.java" checks="LineLength" />
|
||||
|
|
|
@ -194,14 +194,14 @@ public class TransportClusterHealthAction extends TransportMasterNodeReadAction<
|
|||
}
|
||||
|
||||
private boolean validateRequest(final ClusterHealthRequest request, ClusterState clusterState, final int waitFor) {
|
||||
ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.numberOfPendingTasks(),
|
||||
gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMaxTaskWaitTime());
|
||||
ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.getMasterService().numberOfPendingTasks(),
|
||||
gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMasterService().getMaxTaskWaitTime());
|
||||
return prepareResponse(request, response, clusterState, waitFor);
|
||||
}
|
||||
|
||||
private ClusterHealthResponse getResponse(final ClusterHealthRequest request, ClusterState clusterState, final int waitFor, boolean timedOut) {
|
||||
ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.numberOfPendingTasks(),
|
||||
gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMaxTaskWaitTime());
|
||||
ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.getMasterService().numberOfPendingTasks(),
|
||||
gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMasterService().getMaxTaskWaitTime());
|
||||
boolean valid = prepareResponse(request, response, clusterState, waitFor);
|
||||
assert valid || timedOut;
|
||||
// we check for a timeout here since this method might be called from the wait_for_events
|
||||
|
|
|
@ -65,7 +65,7 @@ public class TransportPendingClusterTasksAction extends TransportMasterNodeReadA
|
|||
@Override
|
||||
protected void masterOperation(PendingClusterTasksRequest request, ClusterState state, ActionListener<PendingClusterTasksResponse> listener) {
|
||||
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");
|
||||
listener.onResponse(new PendingClusterTasksResponse(pendingTasks));
|
||||
}
|
||||
|
|
|
@ -98,6 +98,12 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
|
|||
@Override
|
||||
public ActionRequestValidationException 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) {
|
||||
validationException = addValidationError("type is missing", validationException);
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.elasticsearch.cluster;
|
|||
import com.carrotsearch.hppc.cursors.IntObjectCursor;
|
||||
import com.carrotsearch.hppc.cursors.ObjectCursor;
|
||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.block.ClusterBlock;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.cluster;
|
|||
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.cluster.service.ClusterApplierService;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
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 ClusterService clusterService;
|
||||
private final ClusterApplierService clusterApplierService;
|
||||
private final ThreadContext contextHolder;
|
||||
volatile TimeValue timeOutValue;
|
||||
|
||||
|
@ -74,7 +75,12 @@ public class ClusterStateObserver {
|
|||
*/
|
||||
public ClusterStateObserver(ClusterState initialState, ClusterService clusterService, @Nullable TimeValue timeout, Logger logger,
|
||||
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.timeOutValue = timeout;
|
||||
if (timeOutValue != null) {
|
||||
|
@ -89,7 +95,7 @@ public class ClusterStateObserver {
|
|||
if (observingContext.get() != null) {
|
||||
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));
|
||||
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));
|
||||
// update to latest, in case people want to retry
|
||||
timedOut = true;
|
||||
lastObservedState.set(new StoredState(clusterService.state()));
|
||||
lastObservedState.set(new StoredState(clusterApplierService.state()));
|
||||
listener.onTimeout(timeOutValue);
|
||||
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,
|
||||
// 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)) {
|
||||
// good enough, let's go.
|
||||
logger.trace("observer: sampled state accepted by predicate ({})", newState);
|
||||
|
@ -163,7 +169,7 @@ public class ClusterStateObserver {
|
|||
if (!observingContext.compareAndSet(null, context)) {
|
||||
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();
|
||||
if (context.statePredicate.test(state)) {
|
||||
if (observingContext.compareAndSet(context, null)) {
|
||||
clusterService.removeTimeoutListener(this);
|
||||
clusterApplierService.removeTimeoutListener(this);
|
||||
logger.trace("observer: accepting cluster state change ({})", state);
|
||||
lastObservedState.set(new StoredState(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
|
||||
return;
|
||||
}
|
||||
ClusterState newState = clusterService.state();
|
||||
ClusterState newState = clusterApplierService.state();
|
||||
if (lastObservedState.get().isOlderOrDifferentMaster(newState) && context.statePredicate.test(newState)) {
|
||||
// double check we're still listening
|
||||
if (observingContext.compareAndSet(context, null)) {
|
||||
logger.trace("observer: post adding listener: accepting current cluster state ({})", newState);
|
||||
clusterService.removeTimeoutListener(this);
|
||||
clusterApplierService.removeTimeoutListener(this);
|
||||
lastObservedState.set(new StoredState(newState));
|
||||
context.listener.onNewClusterState(newState);
|
||||
} else {
|
||||
|
@ -220,7 +226,7 @@ public class ClusterStateObserver {
|
|||
|
||||
if (context != null) {
|
||||
logger.trace("observer: cluster service closed. notifying listener.");
|
||||
clusterService.removeTimeoutListener(this);
|
||||
clusterApplierService.removeTimeoutListener(this);
|
||||
context.listener.onClusterServiceClose();
|
||||
}
|
||||
}
|
||||
|
@ -229,11 +235,11 @@ public class ClusterStateObserver {
|
|||
public void onTimeout(TimeValue timeout) {
|
||||
ObservingContext context = observingContext.getAndSet(null);
|
||||
if (context != null) {
|
||||
clusterService.removeTimeoutListener(this);
|
||||
clusterApplierService.removeTimeoutListener(this);
|
||||
long timeSinceStartMS = TimeValue.nsecToMSec(System.nanoTime() - startTimeNS);
|
||||
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
|
||||
lastObservedState.set(new StoredState(clusterService.state()));
|
||||
lastObservedState.set(new StoredState(clusterApplierService.state()));
|
||||
timedOut = true;
|
||||
context.listener.onTimeout(timeOutValue);
|
||||
}
|
||||
|
|
|
@ -71,21 +71,18 @@ public interface ClusterStateTaskExecutor<T> {
|
|||
* @param <T> the type of the cluster state update task
|
||||
*/
|
||||
class ClusterTasksResult<T> {
|
||||
public final boolean noMaster;
|
||||
@Nullable
|
||||
public final ClusterState resultingState;
|
||||
public final Map<T, TaskResult> executionResults;
|
||||
|
||||
/**
|
||||
* 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 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.executionResults = executionResults;
|
||||
this.noMaster = noMaster;
|
||||
}
|
||||
|
||||
public static <T> Builder<T> builder() {
|
||||
|
@ -124,11 +121,11 @@ public interface ClusterStateTaskExecutor<T> {
|
|||
}
|
||||
|
||||
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) {
|
||||
return new ClusterTasksResult<>(result.noMaster, result.resultingState == null ? previousState : result.resultingState,
|
||||
return new ClusterTasksResult<>(result.resultingState == null ? previousState : result.resultingState,
|
||||
executionResults);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
*/
|
||||
package org.elasticsearch.cluster;
|
||||
|
||||
import org.elasticsearch.cluster.service.MasterService;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public interface ClusterStateTaskListener {
|
||||
|
@ -28,7 +30,8 @@ public interface ClusterStateTaskListener {
|
|||
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) {
|
||||
onFailure(source, new NotMasterException("no longer master. source: [" + source + "]"));
|
||||
|
|
|
@ -50,25 +50,11 @@ public abstract class LocalClusterUpdateTask implements ClusterStateTaskConfig,
|
|||
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
|
||||
*/
|
||||
public static ClusterTasksResult<LocalClusterUpdateTask> unchanged() {
|
||||
return new ClusterTasksResult(false, null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* locally apply cluster state received from a master
|
||||
*/
|
||||
public static ClusterTasksResult<LocalClusterUpdateTask> newState(ClusterState clusterState) {
|
||||
return new ClusterTasksResult(false, clusterState, null);
|
||||
return new ClusterTasksResult<>(null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
private void scheduleIfNeeded(long currentNanoTime, ClusterState state) {
|
||||
assertClusterStateThread();
|
||||
private synchronized void scheduleIfNeeded(long currentNanoTime, ClusterState state) {
|
||||
assertClusterOrMasterStateThread();
|
||||
long nextDelayNanos = UnassignedInfo.findNextDelayedAllocation(currentNanoTime, state);
|
||||
if (nextDelayNanos < 0) {
|
||||
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 void assertClusterStateThread() {
|
||||
ClusterService.assertClusterStateThread();
|
||||
protected void assertClusterOrMasterStateThread() {
|
||||
assert ClusterService.assertClusterOrMasterStateThread();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -41,7 +41,7 @@ import java.util.concurrent.ThreadLocalRandom;
|
|||
* setting a reproducible seed. When running the Elasticsearch server
|
||||
* process, non-reproducible sources of randomness are provided (unless
|
||||
* 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 {
|
||||
private static final Method currentMethod;
|
||||
|
|
|
@ -19,7 +19,8 @@
|
|||
|
||||
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.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transports;
|
||||
|
@ -63,7 +64,8 @@ public abstract class BaseFuture<V> implements Future<V> {
|
|||
assert timeout <= 0 ||
|
||||
(Transports.assertNotTransportThread(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));
|
||||
}
|
||||
|
||||
|
@ -87,7 +89,8 @@ public abstract class BaseFuture<V> implements Future<V> {
|
|||
public V get() throws InterruptedException, ExecutionException {
|
||||
assert Transports.assertNotTransportThread(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();
|
||||
}
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.discovery;
|
|||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
|
@ -36,10 +37,6 @@ import java.io.IOException;
|
|||
*/
|
||||
public interface Discovery extends LifecycleComponent {
|
||||
|
||||
DiscoveryNode localNode();
|
||||
|
||||
String nodeDescription();
|
||||
|
||||
/**
|
||||
* Another hack to solve dep injection problem..., note, this will be called before
|
||||
* 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
|
||||
* 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
|
||||
* they updated their own cluster state or not.
|
||||
|
@ -58,6 +55,18 @@ public interface Discovery extends LifecycleComponent {
|
|||
*/
|
||||
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 {
|
||||
void onNodeAck(DiscoveryNode node, @Nullable Exception e);
|
||||
void onTimeout();
|
||||
|
@ -83,8 +92,6 @@ public interface Discovery extends LifecycleComponent {
|
|||
*/
|
||||
DiscoveryStats stats();
|
||||
|
||||
DiscoverySettings getDiscoverySettings();
|
||||
|
||||
/**
|
||||
* Triggers the first join cycle
|
||||
*/
|
||||
|
|
|
@ -19,10 +19,12 @@
|
|||
|
||||
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.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.Setting.Property;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -55,8 +57,8 @@ public class DiscoveryModule {
|
|||
private final Discovery discovery;
|
||||
|
||||
public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportService transportService,
|
||||
NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService, ClusterService clusterService,
|
||||
List<DiscoveryPlugin> plugins) {
|
||||
NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService, MasterService masterService,
|
||||
ClusterApplier clusterApplier, ClusterSettings clusterSettings, List<DiscoveryPlugin> plugins) {
|
||||
final UnicastHostsProvider hostsProvider;
|
||||
|
||||
Map<String, Supplier<UnicastHostsProvider>> hostProviders = new HashMap<>();
|
||||
|
@ -80,12 +82,13 @@ public class DiscoveryModule {
|
|||
|
||||
Map<String, Supplier<Discovery>> discoveryTypes = new HashMap<>();
|
||||
discoveryTypes.put("zen",
|
||||
() -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
|
||||
discoveryTypes.put("none", () -> new NoneDiscovery(settings, clusterService, clusterService.getClusterSettings()));
|
||||
discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, clusterService));
|
||||
() -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
|
||||
clusterSettings, hostsProvider));
|
||||
discoveryTypes.put("tribe", () -> new TribeDiscovery(settings, transportService, clusterApplier));
|
||||
discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, transportService, clusterApplier));
|
||||
for (DiscoveryPlugin plugin : plugins) {
|
||||
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) {
|
||||
throw new IllegalArgumentException("Cannot register discovery type [" + entry.getKey() + "] twice");
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -19,53 +19,44 @@
|
|||
|
||||
package org.elasticsearch.discovery.single;
|
||||
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskConfig;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskListener;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.cluster.service.ClusterApplier;
|
||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.DiscoveryStats;
|
||||
import org.elasticsearch.discovery.zen.PendingClusterStateStats;
|
||||
import org.elasticsearch.discovery.zen.PendingClusterStatesQueue;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
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.
|
||||
*/
|
||||
public class SingleNodeDiscovery extends AbstractLifecycleComponent implements Discovery {
|
||||
|
||||
private final ClusterService clusterService;
|
||||
private final DiscoverySettings discoverySettings;
|
||||
protected final TransportService transportService;
|
||||
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));
|
||||
this.clusterService = Objects.requireNonNull(clusterService);
|
||||
final ClusterSettings clusterSettings =
|
||||
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();
|
||||
this.transportService = Objects.requireNonNull(transportService);
|
||||
this.clusterApplier = clusterApplier;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -74,8 +65,57 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
|
|||
}
|
||||
|
||||
@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
|
||||
|
@ -84,41 +124,11 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
|
|||
}
|
||||
|
||||
@Override
|
||||
public DiscoverySettings getDiscoverySettings() {
|
||||
return discoverySettings;
|
||||
}
|
||||
|
||||
@Override
|
||||
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);
|
||||
public synchronized void startInitialJoin() {
|
||||
// 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();
|
||||
clusterApplier.onNewClusterState("single-node-start-initial-join", this::clusterState, (source, e) -> {});
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -127,8 +137,9 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doStart() {
|
||||
|
||||
protected synchronized void doStart() {
|
||||
initialState = getInitialClusterState();
|
||||
clusterState = initialState;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
|||
import org.elasticsearch.cluster.NotMasterException;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
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.io.stream.StreamInput;
|
||||
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 volatile MasterPinger masterPinger;
|
||||
|
@ -78,9 +79,11 @@ public class MasterFaultDetection extends FaultDetection {
|
|||
private final AtomicBoolean notifiedMasterFailure = new AtomicBoolean();
|
||||
|
||||
public MasterFaultDetection(Settings settings, ThreadPool threadPool, TransportService transportService,
|
||||
ClusterService clusterService) {
|
||||
super(settings, threadPool, transportService, clusterService.getClusterName());
|
||||
this.clusterService = clusterService;
|
||||
java.util.function.Supplier<ClusterState> clusterStateSupplier, MasterService masterService,
|
||||
ClusterName clusterName) {
|
||||
super(settings, threadPool, transportService, clusterName);
|
||||
this.clusterStateSupplier = clusterStateSupplier;
|
||||
this.masterService = masterService;
|
||||
|
||||
logger.debug("[master] uses ping_interval [{}], ping_timeout [{}], ping_retries [{}]", pingInterval, pingRetryTimeout,
|
||||
pingRetryCount);
|
||||
|
@ -215,7 +218,8 @@ public class MasterFaultDetection extends FaultDetection {
|
|||
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)
|
||||
.withTimeout(pingRetryTimeout).build();
|
||||
transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options,
|
||||
|
@ -323,7 +327,7 @@ public class MasterFaultDetection extends FaultDetection {
|
|||
|
||||
@Override
|
||||
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
|
||||
// this can happen if the master got "kill -9" and then another node started using the same port
|
||||
if (!request.masterNode.equals(nodes.getLocalNode())) {
|
||||
|
@ -348,7 +352,7 @@ public class MasterFaultDetection extends FaultDetection {
|
|||
|
||||
if (!nodes.isLocalNodeElectedMaster() || !nodes.nodeExists(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
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
|
|
|
@ -24,20 +24,21 @@ import org.apache.logging.log4j.util.Supplier;
|
|||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskConfig;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskListener;
|
||||
import org.elasticsearch.cluster.NotMasterException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.cluster.service.MasterService;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -55,7 +56,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
*/
|
||||
public class NodeJoinController extends AbstractComponent {
|
||||
|
||||
private final ClusterService clusterService;
|
||||
private final MasterService masterService;
|
||||
private final AllocationService allocationService;
|
||||
private final ElectMasterService electMaster;
|
||||
private final JoinTaskExecutor joinTaskExecutor = new JoinTaskExecutor();
|
||||
|
@ -65,10 +66,10 @@ public class NodeJoinController extends AbstractComponent {
|
|||
private ElectionContext electionContext = null;
|
||||
|
||||
|
||||
public NodeJoinController(ClusterService clusterService, AllocationService allocationService, ElectMasterService electMaster,
|
||||
public NodeJoinController(MasterService masterService, AllocationService allocationService, ElectMasterService electMaster,
|
||||
Settings settings) {
|
||||
super(settings);
|
||||
this.clusterService = clusterService;
|
||||
this.masterService = masterService;
|
||||
this.allocationService = allocationService;
|
||||
this.electMaster = electMaster;
|
||||
}
|
||||
|
@ -176,7 +177,7 @@ public class NodeJoinController extends AbstractComponent {
|
|||
electionContext.addIncomingJoin(node, callback);
|
||||
checkPendingJoinsAndElectIfNeeded();
|
||||
} else {
|
||||
clusterService.submitStateUpdateTask("zen-disco-node-join",
|
||||
masterService.submitStateUpdateTask("zen-disco-node-join",
|
||||
node, ClusterStateTaskConfig.build(Priority.URGENT),
|
||||
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(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) {
|
||||
|
@ -287,7 +288,7 @@ public class NodeJoinController extends AbstractComponent {
|
|||
Map<DiscoveryNode, ClusterStateTaskListener> tasks = getPendingAsTasks();
|
||||
final String source = "zen-disco-election-stop [" + reason + "]";
|
||||
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() {
|
||||
|
@ -307,7 +308,7 @@ public class NodeJoinController extends AbstractComponent {
|
|||
}
|
||||
|
||||
private void onElectedAsMaster(ClusterState state) {
|
||||
ClusterService.assertClusterStateThread();
|
||||
assert MasterService.assertMasterUpdateThread();
|
||||
assert state.nodes().isLocalNodeElectedMaster() : "onElectedAsMaster called but local node is not master";
|
||||
ElectionCallback callback = getCallback(); // get under lock
|
||||
if (callback != null) {
|
||||
|
@ -316,7 +317,7 @@ public class NodeJoinController extends AbstractComponent {
|
|||
}
|
||||
|
||||
private void onFailure(Throwable t) {
|
||||
ClusterService.assertClusterStateThread();
|
||||
assert MasterService.assertMasterUpdateThread();
|
||||
ElectionCallback callback = getCallback(); // get under lock
|
||||
if (callback != null) {
|
||||
callback.onFailure(t);
|
||||
|
@ -469,6 +470,7 @@ public class NodeJoinController extends AbstractComponent {
|
|||
DiscoveryNodes currentNodes = currentState.nodes();
|
||||
DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(currentNodes);
|
||||
nodesBuilder.masterNodeId(currentState.nodes().getLocalNodeId());
|
||||
|
||||
for (final DiscoveryNode joiningNode : joiningNodes) {
|
||||
final DiscoveryNode nodeWithSameId = nodesBuilder.get(joiningNode.getId());
|
||||
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
|
||||
// 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,
|
||||
"removed dead nodes on election"));
|
||||
}
|
||||
|
|
|
@ -31,28 +31,28 @@ import org.elasticsearch.cluster.ClusterState;
|
|||
import org.elasticsearch.cluster.ClusterStateTaskConfig;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskListener;
|
||||
import org.elasticsearch.cluster.LocalClusterUpdateTask;
|
||||
import org.elasticsearch.cluster.NotMasterException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.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.component.AbstractLifecycleComponent;
|
||||
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.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.logging.LoggerMessageFormat;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Setting.Property;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.DiscoveryStats;
|
||||
|
@ -68,8 +68,10 @@ import org.elasticsearch.transport.TransportService;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -78,6 +80,7 @@ import java.util.function.Consumer;
|
|||
import java.util.stream.Collectors;
|
||||
|
||||
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 {
|
||||
|
||||
|
@ -105,8 +108,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
public static final String DISCOVERY_REJOIN_ACTION_NAME = "internal:discovery/zen/rejoin";
|
||||
|
||||
private final TransportService transportService;
|
||||
private final NamedWriteableRegistry namedWriteableRegistry;
|
||||
private final ClusterService clusterService;
|
||||
private final MasterService masterService;
|
||||
private AllocationService allocationService;
|
||||
private final ClusterName clusterName;
|
||||
private final DiscoverySettings discoverySettings;
|
||||
|
@ -142,15 +144,19 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
private volatile NodeJoinController nodeJoinController;
|
||||
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,
|
||||
NamedWriteableRegistry namedWriteableRegistry,
|
||||
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
|
||||
NamedWriteableRegistry namedWriteableRegistry, MasterService masterService, ClusterApplier clusterApplier,
|
||||
ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
|
||||
super(settings);
|
||||
this.clusterService = clusterService;
|
||||
this.clusterName = clusterService.getClusterName();
|
||||
this.masterService = masterService;
|
||||
this.clusterApplier = clusterApplier;
|
||||
this.transportService = transportService;
|
||||
this.namedWriteableRegistry = namedWriteableRegistry;
|
||||
this.discoverySettings = new DiscoverySettings(settings, clusterService.getClusterSettings());
|
||||
this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
|
||||
this.zenPing = newZenPing(settings, threadPool, transportService, hostsProvider);
|
||||
this.electMaster = new ElectMasterService(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.sendLeaveRequest = SEND_LEAVE_REQUEST_SETTING.get(settings);
|
||||
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.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 [{}]",
|
||||
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) -> {
|
||||
final ClusterState clusterState = clusterService.state();
|
||||
final ClusterState clusterState = this.clusterState();
|
||||
int masterNodes = clusterState.nodes().getMasterNodes().size();
|
||||
// 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
|
||||
|
@ -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.nodesFD = new NodesFaultDetection(settings, threadPool, transportService, clusterService.getClusterName());
|
||||
this.nodesFD = new NodesFaultDetection(settings, threadPool, transportService, clusterName);
|
||||
this.nodesFD.addListener(new NodeFaultDetectionListener());
|
||||
|
||||
this.publishClusterState =
|
||||
|
@ -198,10 +206,10 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
settings,
|
||||
transportService,
|
||||
namedWriteableRegistry,
|
||||
clusterService::state,
|
||||
this::clusterState,
|
||||
new NewPendingClusterStateListener(),
|
||||
discoverySettings,
|
||||
clusterService.getClusterName());
|
||||
clusterName);
|
||||
this.membership = new MembershipAction(settings, transportService, new MembershipListener());
|
||||
this.joinThreadControl = new JoinThreadControl();
|
||||
|
||||
|
@ -222,30 +230,26 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
|
||||
@Override
|
||||
protected void doStart() {
|
||||
nodesFD.setLocalNode(clusterService.localNode());
|
||||
joinThreadControl.start();
|
||||
DiscoveryNode localNode = transportService.getLocalNode();
|
||||
assert localNode != null;
|
||||
synchronized (stateMutex) {
|
||||
initialState = getInitialClusterState();
|
||||
state.set(initialState);
|
||||
nodesFD.setLocalNode(localNode);
|
||||
joinThreadControl.start();
|
||||
}
|
||||
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);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startInitialJoin() {
|
||||
// start the join thread from a cluster state update. See {@link JoinThreadControl} for details.
|
||||
clusterService.submitStateUpdateTask("initial_join", new LocalClusterUpdateTask() {
|
||||
|
||||
@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();
|
||||
return unchanged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, @org.elasticsearch.common.Nullable Exception e) {
|
||||
logger.warn("failed to start initial join process", e);
|
||||
}
|
||||
});
|
||||
synchronized (stateMutex) {
|
||||
// do the join on a different thread, the caller of this method waits for 30s anyhow till it is discovered
|
||||
joinThreadControl.startNewThreadIfNotRunning();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -286,58 +290,122 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
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 */
|
||||
@Override
|
||||
public DiscoveryNodes nodes() {
|
||||
return clusterService.state().nodes();
|
||||
return clusterState().nodes();
|
||||
}
|
||||
|
||||
@Override
|
||||
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 */
|
||||
|
||||
|
||||
@Override
|
||||
public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) {
|
||||
if (!clusterChangedEvent.state().getNodes().isLocalNodeElectedMaster()) {
|
||||
throw new IllegalStateException("Shouldn't publish state when not master");
|
||||
ClusterState newState = clusterChangedEvent.state();
|
||||
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 {
|
||||
publishClusterState.publish(clusterChangedEvent, electMaster.minimumMasterNodes(), ackListener);
|
||||
} catch (FailedToCommitClusterStateException t) {
|
||||
// cluster service logs a WARN message
|
||||
logger.debug("failed to publish cluster state version [{}] (not enough nodes acknowledged, min master nodes [{}])", clusterChangedEvent.state().version(), electMaster.minimumMasterNodes());
|
||||
submitRejoin("zen-disco-failed-to-publish");
|
||||
logger.debug("failed to publish cluster state version [{}] (not enough nodes acknowledged, min master nodes [{}])",
|
||||
newState.version(), electMaster.minimumMasterNodes());
|
||||
|
||||
synchronized (stateMutex) {
|
||||
publishClusterState.pendingStatesQueue().failAllStatesAndClear(
|
||||
new ElasticsearchException("failed to publish cluster state"));
|
||||
|
||||
rejoin("zen-disco-failed-to-publish");
|
||||
}
|
||||
throw t;
|
||||
}
|
||||
|
||||
// update the set of nodes to ping after the new cluster state has been published
|
||||
nodesFD.updateNodesAndPing(clusterChangedEvent.state());
|
||||
final DiscoveryNode localNode = newState.getNodes().getLocalNode();
|
||||
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
|
||||
// note that we also clean the queue on master failure (see handleMasterGone) but a delayed cluster state publish
|
||||
// from a stale master can still make it in the queue during the election (but not be committed)
|
||||
publishClusterState.pendingStatesQueue().failAllStatesAndClear(new ElasticsearchException("elected as master"));
|
||||
@Override
|
||||
public void onNewClusterStateFailed(Exception e) {
|
||||
processedOrFailed.set(true);
|
||||
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.
|
||||
* NB: for testing purposes
|
||||
*/
|
||||
public Set<DiscoveryNode> getFaultDetectionNodes() {
|
||||
Set<DiscoveryNode> getFaultDetectionNodes() {
|
||||
return nodesFD.getNodes();
|
||||
}
|
||||
|
||||
|
@ -347,7 +415,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
return new DiscoveryStats(queueStats);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DiscoverySettings getDiscoverySettings() {
|
||||
return discoverySettings;
|
||||
}
|
||||
|
@ -391,22 +458,24 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
return;
|
||||
}
|
||||
|
||||
if (clusterService.localNode().equals(masterNode)) {
|
||||
if (transportService.getLocalNode().equals(masterNode)) {
|
||||
final int requiredJoins = Math.max(0, electMaster.minimumMasterNodes() - 1); // we count as one
|
||||
logger.debug("elected as master, waiting for incoming joins ([{}] needed)", requiredJoins);
|
||||
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, masterElectionWaitForJoinsTimeout,
|
||||
new NodeJoinController.ElectionCallback() {
|
||||
@Override
|
||||
public void onElectedAsMaster(ClusterState state) {
|
||||
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
|
||||
synchronized (stateMutex) {
|
||||
joinThreadControl.markThreadAsDone(currentThread);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
logger.trace("failed while waiting for nodes to join, rejoining", t);
|
||||
joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
|
||||
synchronized (stateMutex) {
|
||||
joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -418,41 +487,25 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
// send join request
|
||||
final boolean success = joinElectedMaster(masterNode);
|
||||
|
||||
// finalize join through the cluster state update thread
|
||||
final DiscoveryNode finalMasterNode = masterNode;
|
||||
clusterService.submitStateUpdateTask("finalize_join (" + masterNode + ")", new LocalClusterUpdateTask() {
|
||||
@Override
|
||||
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) {
|
||||
synchronized (stateMutex) {
|
||||
if (success) {
|
||||
DiscoveryNode currentMasterNode = this.clusterState().getNodes().getMasterNode();
|
||||
if (currentMasterNode == null) {
|
||||
// Post 1.3.0, the master should publish a new cluster state before acking our join request. we now should have
|
||||
// a valid master.
|
||||
logger.debug("no master node is set, despite of join request completing. retrying pings.");
|
||||
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);
|
||||
return unchanged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, @Nullable Exception e) {
|
||||
logger.error("unexpected error while trying to finalize cluster join", e);
|
||||
} else {
|
||||
// failed to join. Try again...
|
||||
joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -473,7 +526,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
while (true) {
|
||||
try {
|
||||
logger.trace("joining master {}", masterNode);
|
||||
membership.sendJoinRequestBlocking(masterNode, clusterService.localNode(), joinTimeout);
|
||||
membership.sendJoinRequestBlocking(masterNode, transportService.getLocalNode(), joinTimeout);
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
final Throwable unwrap = ExceptionsHelper.unwrapCause(e);
|
||||
|
@ -503,18 +556,16 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
}
|
||||
|
||||
private void submitRejoin(String source) {
|
||||
clusterService.submitStateUpdateTask(source, new LocalClusterUpdateTask(Priority.IMMEDIATE) {
|
||||
@Override
|
||||
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) {
|
||||
return rejoin(currentState, source);
|
||||
}
|
||||
synchronized (stateMutex) {
|
||||
rejoin(source);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
|
||||
}
|
||||
|
||||
});
|
||||
// visible for testing
|
||||
void setState(ClusterState clusterState) {
|
||||
synchronized (stateMutex) {
|
||||
state.set(clusterState);
|
||||
}
|
||||
}
|
||||
|
||||
// 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) {
|
||||
clusterService.submitStateUpdateTask(
|
||||
masterService.submitStateUpdateTask(
|
||||
source + "(" + node + "), reason(" + reason + ")",
|
||||
new NodeRemovalClusterStateTaskExecutor.Task(node, reason),
|
||||
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.
|
||||
return;
|
||||
}
|
||||
clusterService.submitStateUpdateTask("zen-disco-min-master-nodes-changed", new LocalClusterUpdateTask(Priority.IMMEDIATE) {
|
||||
@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
|
||||
if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) {
|
||||
return rejoin(currentState, "not enough master nodes on change of minimum_master_nodes from [" + prevMinimumMasterNode + "] to [" + minimumMasterNodes + "]");
|
||||
}
|
||||
return unchanged();
|
||||
synchronized (stateMutex) {
|
||||
// check if we have enough master nodes, if not, we need to move into joining the cluster again
|
||||
if (!electMaster.hasEnoughMasterNodes(state.get().nodes())) {
|
||||
rejoin("not enough master nodes on change of minimum_master_nodes from [" + prevMinimumMasterNode + "] to [" + minimumMasterNodes + "]");
|
||||
}
|
||||
|
||||
|
||||
@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) {
|
||||
|
@ -694,116 +725,135 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
|
||||
logger.info((Supplier<?>) () -> new ParameterizedMessage("master_left [{}], reason [{}]", masterNode, reason), cause);
|
||||
|
||||
clusterService.submitStateUpdateTask("master_failed (" + masterNode + ")", new LocalClusterUpdateTask(Priority.IMMEDIATE) {
|
||||
|
||||
@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();
|
||||
}
|
||||
|
||||
synchronized (stateMutex) {
|
||||
if (localNodeMaster() == false && masterNode.equals(state.get().nodes().getMasterNode())) {
|
||||
// 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));
|
||||
|
||||
return rejoin(currentState, "master left (reason = " + reason + ")");
|
||||
rejoin("master left (reason = " + reason + ")");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
|
||||
}
|
||||
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
void processNextPendingClusterState(String reason) {
|
||||
clusterService.submitStateUpdateTask("zen-disco-receive(from master [" + reason + "])", new LocalClusterUpdateTask(Priority.URGENT) {
|
||||
ClusterState newClusterState = null;
|
||||
// return true if state has been sent to applier
|
||||
boolean processNextCommittedClusterState(String reason) {
|
||||
assert Thread.holdsLock(stateMutex);
|
||||
|
||||
@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
|
||||
if (newClusterState == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// all pending states have been processed
|
||||
if (newClusterState == null) {
|
||||
return unchanged();
|
||||
}
|
||||
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.nodes().getMasterNode() != null : "received a cluster state without a master";
|
||||
assert !newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block";
|
||||
if (currentState.nodes().isLocalNodeElectedMaster() && newClusterState.nodes().isLocalNodeElectedMaster() == false) {
|
||||
handleAnotherMaster(currentState, newClusterState.nodes().getMasterNode(), newClusterState.version(), "via a new cluster state");
|
||||
return false;
|
||||
}
|
||||
|
||||
if (currentState.nodes().isLocalNodeElectedMaster()) {
|
||||
return handleAnotherMaster(currentState, newClusterState.nodes().getMasterNode(), newClusterState.version(), "via a new cluster state");
|
||||
}
|
||||
try {
|
||||
if (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) {
|
||||
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 (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) {
|
||||
return unchanged();
|
||||
}
|
||||
if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) {
|
||||
// its a fresh update from the master as we transition from a start of not having a master to having one
|
||||
logger.debug("got first state from fresh master [{}]", newClusterState.nodes().getMasterNodeId());
|
||||
return newState(newClusterState);
|
||||
}
|
||||
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())) {
|
||||
// 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());
|
||||
adaptedNewClusterState = newClusterState;
|
||||
} else if (newClusterState.nodes().isLocalNodeElectedMaster() == false) {
|
||||
// some optimizations to make sure we keep old objects where possible
|
||||
ClusterState.Builder builder = ClusterState.builder(newClusterState);
|
||||
|
||||
// some optimizations to make sure we keep old objects where possible
|
||||
ClusterState.Builder builder = ClusterState.builder(newClusterState);
|
||||
|
||||
// if the routing table did not change, use the original one
|
||||
if (newClusterState.routingTable().version() == currentState.routingTable().version()) {
|
||||
builder.routingTable(currentState.routingTable());
|
||||
}
|
||||
// same for metadata
|
||||
if (newClusterState.metaData().version() == currentState.metaData().version()) {
|
||||
builder.metaData(currentState.metaData());
|
||||
} else {
|
||||
// if its not the same version, only copy over new indices or ones that changed the version
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder(newClusterState.metaData()).removeAllIndices();
|
||||
for (IndexMetaData indexMetaData : newClusterState.metaData()) {
|
||||
IndexMetaData currentIndexMetaData = currentState.metaData().index(indexMetaData.getIndex());
|
||||
if (currentIndexMetaData != null && currentIndexMetaData.isSameUUID(indexMetaData.getIndexUUID()) &&
|
||||
currentIndexMetaData.getVersion() == indexMetaData.getVersion()) {
|
||||
// safe to reuse
|
||||
metaDataBuilder.put(currentIndexMetaData, false);
|
||||
} else {
|
||||
metaDataBuilder.put(indexMetaData, false);
|
||||
}
|
||||
// if the routing table did not change, use the original one
|
||||
if (newClusterState.routingTable().version() == currentState.routingTable().version()) {
|
||||
builder.routingTable(currentState.routingTable());
|
||||
}
|
||||
// same for metadata
|
||||
if (newClusterState.metaData().version() == currentState.metaData().version()) {
|
||||
builder.metaData(currentState.metaData());
|
||||
} else {
|
||||
// if its not the same version, only copy over new indices or ones that changed the version
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder(newClusterState.metaData()).removeAllIndices();
|
||||
for (IndexMetaData indexMetaData : newClusterState.metaData()) {
|
||||
IndexMetaData currentIndexMetaData = currentState.metaData().index(indexMetaData.getIndex());
|
||||
if (currentIndexMetaData != null && currentIndexMetaData.isSameUUID(indexMetaData.getIndexUUID()) &&
|
||||
currentIndexMetaData.getVersion() == indexMetaData.getVersion()) {
|
||||
// safe to reuse
|
||||
metaDataBuilder.put(currentIndexMetaData, false);
|
||||
} else {
|
||||
metaDataBuilder.put(indexMetaData, false);
|
||||
}
|
||||
builder.metaData(metaDataBuilder);
|
||||
}
|
||||
|
||||
return newState(builder.build());
|
||||
builder.metaData(metaDataBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
|
||||
if (newClusterState != null) {
|
||||
adaptedNewClusterState = builder.build();
|
||||
} else {
|
||||
adaptedNewClusterState = newClusterState;
|
||||
}
|
||||
|
||||
if (currentState == adaptedNewClusterState) {
|
||||
return false;
|
||||
}
|
||||
|
||||
state.set(adaptedNewClusterState);
|
||||
|
||||
clusterApplier.onNewClusterState("apply cluster state (from master [" + reason + "])",
|
||||
this::clusterState,
|
||||
new ClusterStateTaskListener() {
|
||||
@Override
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
try {
|
||||
publishClusterState.pendingStatesQueue().markAsProcessed(newClusterState);
|
||||
} catch (Exception 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 [{}]", source), inner);
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected exception while failing [{}]", reason), inner);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
@Override
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
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);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
onFailure(source, e);
|
||||
}
|
||||
}
|
||||
});
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -888,13 +938,13 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
logger.trace("full ping responses:{}", sb);
|
||||
}
|
||||
|
||||
final DiscoveryNode localNode = clusterService.localNode();
|
||||
final DiscoveryNode localNode = transportService.getLocalNode();
|
||||
|
||||
// add our selves
|
||||
assert fullPingResponses.stream().map(ZenPing.PingResponse::node)
|
||||
.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
|
||||
final List<ZenPing.PingResponse> pingResponses = filterPingResponses(fullPingResponses, masterElectionIgnoreNonMasters, logger);
|
||||
|
@ -956,10 +1006,9 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
return pingResponses;
|
||||
}
|
||||
|
||||
protected ClusterStateTaskExecutor.ClusterTasksResult<LocalClusterUpdateTask> rejoin(ClusterState clusterState, String reason) {
|
||||
|
||||
// *** called from within an cluster state update task *** //
|
||||
assert Thread.currentThread().getName().contains(ClusterService.UPDATE_THREAD_NAME);
|
||||
protected void rejoin(String reason) {
|
||||
assert Thread.holdsLock(stateMutex);
|
||||
ClusterState clusterState = state.get();
|
||||
|
||||
logger.warn("{}, current nodes: {}", reason, clusterState.nodes());
|
||||
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
|
||||
// before a decision is made.
|
||||
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() {
|
||||
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 Thread.currentThread().getName().contains(ClusterService.UPDATE_THREAD_NAME) : "not called from the cluster state update thread";
|
||||
assert Thread.holdsLock(stateMutex);
|
||||
|
||||
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 {
|
||||
// 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);
|
||||
// spawn to a background thread to not do blocking operations on the cluster state thread
|
||||
threadPool.generic().execute(new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), e);
|
||||
}
|
||||
try {
|
||||
// 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
|
||||
// in the past (after a master failure, for example)
|
||||
transportService.connectToNode(otherMaster);
|
||||
transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, new RejoinClusterRequest(localClusterState.nodes().getLocalNodeId()), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
|
||||
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
// 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
|
||||
// in the past (after a master failure, for example)
|
||||
transportService.connectToNode(otherMaster);
|
||||
transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, new RejoinClusterRequest(localNode().getId()), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
|
||||
|
||||
@Override
|
||||
public void handleException(TransportException exp) {
|
||||
logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), exp);
|
||||
}
|
||||
});
|
||||
}
|
||||
});
|
||||
return LocalClusterUpdateTask.unchanged();
|
||||
@Override
|
||||
public void handleException(TransportException 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1033,14 +1092,16 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
|
||||
@Override
|
||||
public void onNewClusterState(String reason) {
|
||||
processNextPendingClusterState(reason);
|
||||
synchronized (stateMutex) {
|
||||
processNextCommittedClusterState(reason);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private class MembershipListener implements MembershipAction.MembershipListener {
|
||||
@Override
|
||||
public void onJoin(DiscoveryNode node, MembershipAction.JoinCallback callback) {
|
||||
handleJoinRequest(node, clusterService.state(), callback);
|
||||
handleJoinRequest(node, ZenDiscovery.this.clusterState(), callback);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1072,23 +1133,13 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
return;
|
||||
}
|
||||
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) {
|
||||
|
||||
@Override
|
||||
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
|
||||
if (currentState.nodes().isLocalNodeElectedMaster()) {
|
||||
pingsWhileMaster.set(0);
|
||||
return handleAnotherMaster(currentState, pingRequest.masterNode(), pingRequest.clusterStateVersion(), "node fd ping");
|
||||
} else {
|
||||
return unchanged();
|
||||
}
|
||||
synchronized (stateMutex) {
|
||||
ClusterState currentState = state.get();
|
||||
if (currentState.nodes().isLocalNodeElectedMaster()) {
|
||||
pingsWhileMaster.set(0);
|
||||
handleAnotherMaster(currentState, pingRequest.masterNode(), pingRequest.clusterStateVersion(), "node fd ping");
|
||||
}
|
||||
|
||||
@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> {
|
||||
@Override
|
||||
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 {
|
||||
channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
||||
} catch (Exception 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 + "]");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
|
||||
}
|
||||
});
|
||||
try {
|
||||
channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
||||
} catch (Exception e) {
|
||||
logger.warn("failed to send response on rejoin cluster request handling", e);
|
||||
}
|
||||
synchronized (stateMutex) {
|
||||
rejoin("received a request to rejoin the cluster from [" + request.fromNodeId + "]");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1169,15 +1211,15 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
}
|
||||
|
||||
/** cleans any running joining thread and calls {@link #rejoin} */
|
||||
public ClusterStateTaskExecutor.ClusterTasksResult<LocalClusterUpdateTask> stopRunningThreadAndRejoin(ClusterState clusterState, String reason) {
|
||||
ClusterService.assertClusterStateThread();
|
||||
public void stopRunningThreadAndRejoin(String reason) {
|
||||
assert Thread.holdsLock(stateMutex);
|
||||
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 */
|
||||
public void startNewThreadIfNotRunning() {
|
||||
ClusterService.assertClusterStateThread();
|
||||
assert Thread.holdsLock(stateMutex);
|
||||
if (joinThreadActive()) {
|
||||
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.
|
||||
*/
|
||||
public void markThreadAsDoneAndStartNew(Thread joinThread) {
|
||||
ClusterService.assertClusterStateThread();
|
||||
assert Thread.holdsLock(stateMutex);
|
||||
if (!markThreadAsDone(joinThread)) {
|
||||
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 */
|
||||
public boolean markThreadAsDone(Thread joinThread) {
|
||||
ClusterService.assertClusterStateThread();
|
||||
assert Thread.holdsLock(stateMutex);
|
||||
return currentJoinThread.compareAndSet(joinThread, null);
|
||||
}
|
||||
|
||||
|
|
|
@ -122,9 +122,6 @@ public class GatewayService extends AbstractLifecycleComponent implements Cluste
|
|||
// TODO: change me once the minimum_master_nodes is changed too
|
||||
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
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.elasticsearch.cluster.ClusterName;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateListener;
|
||||
import org.elasticsearch.cluster.ClusterStateObserver;
|
||||
import org.elasticsearch.cluster.LocalClusterUpdateTask;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
|
@ -280,26 +279,20 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe
|
|||
return;
|
||||
}
|
||||
|
||||
clusterService.submitStateUpdateTask("indices_store ([" + shardId + "] active fully on other nodes)", new LocalClusterUpdateTask() {
|
||||
@Override
|
||||
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
|
||||
clusterService.getClusterApplierService().runOnApplierThread("indices_store ([" + shardId + "] active fully on other nodes)",
|
||||
currentState -> {
|
||||
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);
|
||||
return unchanged();
|
||||
return;
|
||||
}
|
||||
try {
|
||||
indicesService.deleteShardStore("no longer used", shardId, currentState);
|
||||
} catch (Exception ex) {
|
||||
logger.debug((Supplier<?>) () -> new ParameterizedMessage("{} failed to delete unallocated shard, ignoring", shardId), ex);
|
||||
}
|
||||
return unchanged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("{} unexpected error during deletion of unallocated shard", shardId), e);
|
||||
}
|
||||
});
|
||||
},
|
||||
(source, e) -> logger.error((Supplier<?>) () -> new ParameterizedMessage("{} unexpected error during deletion of unallocated shard", shardId), e)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -260,7 +260,7 @@ public class Node implements Closeable {
|
|||
nodeEnvironment = new NodeEnvironment(tmpSettings, environment);
|
||||
resourcesToClose.add(nodeEnvironment);
|
||||
} 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);
|
||||
Logger logger = Loggers.getLogger(Node.class, tmpSettings);
|
||||
|
@ -336,8 +336,7 @@ public class Node implements Closeable {
|
|||
resourcesToClose.add(resourceWatcherService);
|
||||
final NetworkService networkService = new NetworkService(settings,
|
||||
getCustomNameResolvers(pluginsService.filterPlugins(DiscoveryPlugin.class)));
|
||||
final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool,
|
||||
localNodeFactory::getNode);
|
||||
final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool);
|
||||
clusterService.addListener(scriptModule.getScriptService());
|
||||
resourcesToClose.add(clusterService);
|
||||
final IngestService ingestService = new IngestService(settings, threadPool, this.environment,
|
||||
|
@ -428,12 +427,13 @@ public class Node implements Closeable {
|
|||
};
|
||||
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(),
|
||||
transportService, indicesService, pluginsService, circuitBreakerService, scriptModule.getScriptService(),
|
||||
httpServerTransport, ingestService, clusterService, settingsModule.getSettingsFilter());
|
||||
|
||||
modules.add(b -> {
|
||||
b.bind(NodeService.class).toInstance(nodeService);
|
||||
b.bind(NamedXContentRegistry.class).toInstance(xContentRegistry);
|
||||
|
@ -662,9 +662,7 @@ public class Node implements Closeable {
|
|||
injector.getInstance(ResourceWatcherService.class).start();
|
||||
injector.getInstance(GatewayService.class).start();
|
||||
Discovery discovery = injector.getInstance(Discovery.class);
|
||||
clusterService.setDiscoverySettings(discovery.getDiscoverySettings());
|
||||
clusterService.addInitialStateBlock(discovery.getDiscoverySettings().getNoMasterBlock());
|
||||
clusterService.setClusterStatePublisher(discovery::publish);
|
||||
clusterService.getMasterService().setClusterStatePublisher(discovery::publish);
|
||||
|
||||
// start before the cluster service since it adds/removes initial Cluster state blocks
|
||||
final TribeService tribeService = injector.getInstance(TribeService.class);
|
||||
|
@ -674,18 +672,20 @@ public class Node implements Closeable {
|
|||
TransportService transportService = injector.getInstance(TransportService.class);
|
||||
transportService.getTaskManager().setTaskResultsService(injector.getInstance(TaskResultsService.class));
|
||||
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()
|
||||
.flatMap(p -> p.getBootstrapChecks().stream()).collect(Collectors.toList()));
|
||||
|
||||
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();
|
||||
assert localNodeFactory.getNode() != null;
|
||||
assert transportService.getLocalNode().equals(localNodeFactory.getNode())
|
||||
: "transportService has a different local node than the factory provided";
|
||||
discovery.start();
|
||||
assert clusterService.localNode().equals(localNodeFactory.getNode())
|
||||
: "clusterService has a different local node than the factory provided";
|
||||
// start after cluster service so the local disco is known
|
||||
discovery.start();
|
||||
transportService.acceptIncomingRequests();
|
||||
discovery.startInitialJoin();
|
||||
// tribe nodes don't have a master so we shouldn't register an observer s
|
||||
|
|
|
@ -82,7 +82,7 @@ public class NodeService extends AbstractComponent implements Closeable {
|
|||
|
||||
public NodeInfo info(boolean settings, boolean os, boolean process, boolean jvm, boolean threadPool,
|
||||
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,
|
||||
os ? monitorService.osService().info() : null,
|
||||
process ? monitorService.processService().info() : null,
|
||||
|
@ -101,7 +101,7 @@ public class NodeService extends AbstractComponent implements Closeable {
|
|||
boolean script, boolean discoveryStats, boolean ingest) {
|
||||
// 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)
|
||||
return new NodeStats(discovery.localNode(), System.currentTimeMillis(),
|
||||
return new NodeStats(transportService.getLocalNode(), System.currentTimeMillis(),
|
||||
indices.anySet() ? indicesService.stats(true, indices) : null,
|
||||
os ? monitorService.osService().stats() : null,
|
||||
process ? monitorService.processService().stats() : null,
|
||||
|
|
|
@ -23,13 +23,14 @@ import java.util.Collections;
|
|||
import java.util.Map;
|
||||
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.network.NetworkService;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.zen.UnicastHostsProvider;
|
||||
import org.elasticsearch.discovery.zen.ZenPing;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
|
@ -57,12 +58,17 @@ public interface DiscoveryPlugin {
|
|||
*
|
||||
* @param threadPool Use to schedule ping actions
|
||||
* @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
|
||||
*/
|
||||
default Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
|
||||
NamedWriteableRegistry namedWriteableRegistry,
|
||||
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
|
||||
MasterService masterService,
|
||||
ClusterApplier clusterApplier,
|
||||
ClusterSettings clusterSettings,
|
||||
UnicastHostsProvider hostsProvider) {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
|
|
|
@ -148,7 +148,8 @@ public class RepositoriesService extends AbstractComponent implements ClusterSta
|
|||
|
||||
@Override
|
||||
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
|
||||
public boolean mustAck(DiscoveryNode discoveryNode) {
|
||||
// Since operation occurs only on masters, it's enough that only master-eligible nodes acked
|
||||
return discoveryNode.isMasterNode();
|
||||
// repository was created on both master and data nodes
|
||||
return discoveryNode.isMasterNode() || discoveryNode.isDataNode();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -46,7 +46,7 @@ public class RestAliasAction extends AbstractCatAction {
|
|||
@Override
|
||||
protected RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) {
|
||||
final GetAliasesRequest getAliasesRequest = request.hasParam("alias") ?
|
||||
new GetAliasesRequest(request.param("alias")) :
|
||||
new GetAliasesRequest(Strings.commaDelimitedListToStringArray(request.param("alias"))) :
|
||||
new GetAliasesRequest();
|
||||
getAliasesRequest.local(request.paramAsBoolean("local", getAliasesRequest.local()));
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.search.aggregations.metrics.geocentroid;
|
||||
|
||||
import org.apache.lucene.geo.GeoEncodingUtils;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.geo.GeoPoint;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -36,8 +37,8 @@ import java.util.Objects;
|
|||
* Serialization and merge logic for {@link GeoCentroidAggregator}.
|
||||
*/
|
||||
public class InternalGeoCentroid extends InternalAggregation implements GeoCentroid {
|
||||
protected final GeoPoint centroid;
|
||||
protected final long count;
|
||||
private final GeoPoint centroid;
|
||||
private final long count;
|
||||
|
||||
public static long encodeLatLon(double lat, double 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();
|
||||
case "lon":
|
||||
return centroid.lon();
|
||||
case "count":
|
||||
return count;
|
||||
default:
|
||||
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 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
|
||||
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
return renderXContent(builder, params, centroid, count);
|
||||
}
|
||||
|
||||
static XContentBuilder renderXContent(XContentBuilder builder, Params params, GeoPoint centroid, long count) throws IOException {
|
||||
if (centroid != null) {
|
||||
builder.startObject(Fields.CENTROID).field("lat", centroid.lat()).field("lon", centroid.lon()).endObject();
|
||||
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;
|
||||
}
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.elasticsearch.common.Priority;
|
|||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||
import org.elasticsearch.common.component.Lifecycle;
|
||||
import org.elasticsearch.common.hash.MurmurHash3;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
|
||||
|
@ -73,6 +74,7 @@ import java.util.EnumSet;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.function.Function;
|
||||
|
@ -129,7 +131,7 @@ public class TribeService extends AbstractLifecycleComponent {
|
|||
if (!NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.exists(settings)) {
|
||||
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
|
||||
sb.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
|
||||
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.blockIndicesRead = BLOCKS_READ_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);
|
||||
}
|
||||
|
||||
|
@ -290,12 +282,7 @@ public class TribeService extends AbstractLifecycleComponent {
|
|||
|
||||
@Override
|
||||
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() {
|
||||
|
@ -516,7 +503,10 @@ public class TribeService extends AbstractLifecycleComponent {
|
|||
final List<Node> tribeClientNodes = TribeService.this.nodes;
|
||||
Map<String, MetaData.Custom> mergedCustomMetaDataMap = mergeChangedCustomMetaData(changedCustomMetaDataTypeSet,
|
||||
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(clusterMetaData -> ((MetaData.Custom) clusterMetaData.custom(customMetaDataType)))
|
||||
.filter(custom1 -> custom1 != null && custom1 instanceof MergableCustomMetaData)
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.elasticsearch.test.ESTestCase;
|
|||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -255,4 +256,39 @@ public class BulkRequestTests extends ESTestCase {
|
|||
assertEquals(1, request.sourceAsMap().size());
|
||||
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"));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -59,9 +59,11 @@ import static java.util.Collections.emptyMap;
|
|||
import static java.util.Collections.singletonList;
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
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.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
|
||||
import static org.hamcrest.Matchers.arrayContaining;
|
||||
import static org.hamcrest.Matchers.contains;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
@ -485,4 +487,19 @@ public class UpdateRequestTests extends ESTestCase {
|
|||
BytesReference finalBytes = toXContent(parsedUpdateRequest, xContentType, humanReadable);
|
||||
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"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
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.allocation.RoutingAllocation;
|
||||
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.ThrottlingAllocationDecider;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.inject.ModuleTestCase;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.IndexScopedSettings;
|
||||
|
@ -61,8 +58,7 @@ import java.util.function.Supplier;
|
|||
|
||||
public class ClusterModuleTests extends ModuleTestCase {
|
||||
private ClusterService clusterService = new ClusterService(Settings.EMPTY,
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null, () ->
|
||||
new DiscoveryNode(UUIDs.randomBase64UUID(), buildNewFakeTransportAddress(), Version.CURRENT));
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null);
|
||||
static class FakeAllocationDecider extends AllocationDecider {
|
||||
protected FakeAllocationDecider(Settings settings) {
|
||||
super(settings);
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.elasticsearch.action.support.IndicesOptions;
|
|||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.LocalClusterUpdateTask;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
|
@ -130,18 +129,11 @@ public class ClusterStateHealthTests extends ESTestCase {
|
|||
});
|
||||
|
||||
logger.info("--> submit task to restore master");
|
||||
clusterService.submitStateUpdateTask("restore master", new LocalClusterUpdateTask() {
|
||||
@Override
|
||||
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
|
||||
return newState(ClusterState.builder(currentState).nodes(
|
||||
DiscoveryNodes.builder(currentState.nodes()).masterNodeId(currentState.nodes().getLocalNodeId())).build());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
logger.warn("unexpected failure", e);
|
||||
}
|
||||
});
|
||||
ClusterState currentState = clusterService.getClusterApplierService().state();
|
||||
clusterService.getClusterApplierService().onNewClusterState("restore master",
|
||||
() -> ClusterState.builder(currentState)
|
||||
.nodes(DiscoveryNodes.builder(currentState.nodes()).masterNodeId(currentState.nodes().getLocalNodeId())).build(),
|
||||
(source, e) -> {});
|
||||
|
||||
logger.info("--> waiting for listener to be called and cluster state being blocked");
|
||||
listenerCalled.await();
|
||||
|
|
|
@ -465,12 +465,12 @@ public class DelayedAllocationServiceTests extends ESAllocationTestCase {
|
|||
private volatile long nanoTimeOverride = -1L;
|
||||
|
||||
TestDelayAllocationService(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
AllocationService allocationService) {
|
||||
AllocationService allocationService) {
|
||||
super(settings, threadPool, clusterService, allocationService);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void assertClusterStateThread() {
|
||||
protected void assertClusterOrMasterStateThread() {
|
||||
// do not check this in the unit tests
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -382,7 +382,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
|
|||
|
||||
// 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"));
|
||||
List<PendingClusterTask> pendingClusterTasks = clusterService.pendingTasks();
|
||||
List<PendingClusterTask> pendingClusterTasks = clusterService.getMasterService().pendingTasks();
|
||||
assertThat(pendingClusterTasks.size(), greaterThanOrEqualTo(10));
|
||||
assertThat(pendingClusterTasks.get(0).getSource().string(), equalTo("1"));
|
||||
assertThat(pendingClusterTasks.get(0).isExecuting(), equalTo(true));
|
||||
|
@ -404,7 +404,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
|
|||
invoked2.await();
|
||||
|
||||
// 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();
|
||||
|
||||
final CountDownLatch block2 = new CountDownLatch(1);
|
||||
|
@ -444,7 +444,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
|
|||
}
|
||||
Thread.sleep(100);
|
||||
|
||||
pendingClusterTasks = clusterService.pendingTasks();
|
||||
pendingClusterTasks = clusterService.getMasterService().pendingTasks();
|
||||
assertThat(pendingClusterTasks.size(), greaterThanOrEqualTo(5));
|
||||
controlSources = new HashSet<>(Arrays.asList("1", "2", "3", "4", "5"));
|
||||
for (PendingClusterTask task : pendingClusterTasks) {
|
||||
|
|
|
@ -20,32 +20,25 @@ package org.elasticsearch.cluster.service;
|
|||
|
||||
import org.apache.logging.log4j.Level;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.util.Supplier;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateObserver;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskConfig;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskListener;
|
||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||
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.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
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.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.MockLogAppender;
|
||||
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.emptySet;
|
||||
import static org.elasticsearch.test.ClusterServiceUtils.setState;
|
||||
import static org.hamcrest.Matchers.anyOf;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
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;
|
||||
TimedClusterService clusterService;
|
||||
private static ThreadPool threadPool;
|
||||
private TimedMasterService masterService;
|
||||
|
||||
@BeforeClass
|
||||
public static void createThreadPool() {
|
||||
threadPool = new TestThreadPool(ClusterServiceTests.class.getName());
|
||||
threadPool = new TestThreadPool(MasterServiceTests.class.getName());
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -103,48 +94,35 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
clusterService = createTimedClusterService(true);
|
||||
masterService = createTimedMasterService(true);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
clusterService.close();
|
||||
masterService.close();
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
TimedClusterService createTimedClusterService(boolean makeMaster) throws InterruptedException {
|
||||
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));
|
||||
timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
|
||||
@Override
|
||||
public void connectToNodes(DiscoveryNodes discoveryNodes) {
|
||||
// skip
|
||||
}
|
||||
|
||||
@Override
|
||||
public void disconnectFromNodesExcept(DiscoveryNodes nodesToKeep) {
|
||||
// skip
|
||||
}
|
||||
});
|
||||
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;
|
||||
private TimedMasterService createTimedMasterService(boolean makeMaster) throws InterruptedException {
|
||||
DiscoveryNode localNode = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(),
|
||||
emptySet(), Version.CURRENT);
|
||||
TimedMasterService timedMasterService = new TimedMasterService(Settings.builder().put("cluster.name",
|
||||
MasterServiceTests.class.getSimpleName()).build(), threadPool);
|
||||
ClusterState initialClusterState = ClusterState.builder(new ClusterName(MasterServiceTests.class.getSimpleName()))
|
||||
.nodes(DiscoveryNodes.builder()
|
||||
.add(localNode)
|
||||
.localNodeId(localNode.getId())
|
||||
.masterNodeId(makeMaster ? localNode.getId() : null))
|
||||
.blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build();
|
||||
AtomicReference<ClusterState> clusterStateRef = new AtomicReference<>(initialClusterState);
|
||||
timedMasterService.setClusterStatePublisher((event, ackListener) -> clusterStateRef.set(event.state()));
|
||||
timedMasterService.setClusterStateSupplier(clusterStateRef::get);
|
||||
timedMasterService.start();
|
||||
return timedMasterService;
|
||||
}
|
||||
|
||||
public void testMasterAwareExecution() throws Exception {
|
||||
ClusterService nonMaster = createTimedClusterService(false);
|
||||
TimedMasterService nonMaster = createTimedMasterService(false);
|
||||
|
||||
final boolean[] taskFailed = {false};
|
||||
final CountDownLatch latch1 = new CountDownLatch(1);
|
||||
|
@ -196,7 +174,7 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
AtomicBoolean published = new AtomicBoolean();
|
||||
|
||||
clusterService.submitStateUpdateTask(
|
||||
masterService.submitStateUpdateTask(
|
||||
"testClusterStateTaskListenerThrowingExceptionIsOkay",
|
||||
new Object(),
|
||||
ClusterStateTaskConfig.build(Priority.NORMAL),
|
||||
|
@ -229,49 +207,109 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
assertTrue(published.get());
|
||||
}
|
||||
|
||||
public void testBlockingCallInClusterStateTaskListenerFails() throws InterruptedException {
|
||||
assumeTrue("assertions must be enabled for this test to work", BaseFuture.class.desiredAssertionStatus());
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
final AtomicReference<AssertionError> assertionRef = new AtomicReference<>();
|
||||
@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",
|
||||
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(
|
||||
"testBlockingCallInClusterStateTaskListenerFails",
|
||||
new Object(),
|
||||
ClusterStateTaskConfig.build(Priority.NORMAL),
|
||||
new ClusterStateTaskExecutor<Object>() {
|
||||
Logger clusterLogger = Loggers.getLogger(masterService.getClass().getPackage().getName());
|
||||
Loggers.addAppender(clusterLogger, mockAppender);
|
||||
try {
|
||||
final CountDownLatch latch = new CountDownLatch(4);
|
||||
masterService.currentTimeOverride = System.nanoTime();
|
||||
masterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterTasksResult<Object> execute(ClusterState currentState, List<Object> tasks) throws Exception {
|
||||
ClusterState newClusterState = ClusterState.builder(currentState).build();
|
||||
return ClusterTasksResult.builder().successes(tasks).build(newClusterState);
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
masterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
|
||||
return currentState;
|
||||
}
|
||||
},
|
||||
new ClusterStateTaskListener() {
|
||||
|
||||
@Override
|
||||
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
|
||||
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");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
latch.await();
|
||||
assertNotNull(assertionRef.get());
|
||||
assertThat(assertionRef.get().getMessage(), containsString("not be the cluster state update thread. Reason: [Blocking operation]"));
|
||||
@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();
|
||||
} finally {
|
||||
Loggers.removeAppender(clusterLogger, mockAppender);
|
||||
mockAppender.stop();
|
||||
}
|
||||
mockAppender.assertAllExpectationsMatched();
|
||||
}
|
||||
|
||||
public void testClusterStateBatchedUpdates() throws BrokenBarrierException, InterruptedException {
|
||||
|
@ -414,7 +452,7 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
submittedTasksPerThread.computeIfAbsent(threadName, key -> new AtomicInteger()).addAndGet(tasks.size());
|
||||
final TaskExecutor executor = assignment.v1();
|
||||
if (tasks.size() == 1) {
|
||||
clusterService.submitStateUpdateTask(
|
||||
masterService.submitStateUpdateTask(
|
||||
threadName,
|
||||
tasks.stream().findFirst().get(),
|
||||
ClusterStateTaskConfig.build(randomFrom(Priority.values())),
|
||||
|
@ -423,7 +461,7 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
} else {
|
||||
Map<Task, ClusterStateTaskListener> taskListeners = new HashMap<>();
|
||||
tasks.stream().forEach(t -> taskListeners.put(t, listener));
|
||||
clusterService.submitStateUpdateTasks(
|
||||
masterService.submitStateUpdateTasks(
|
||||
threadName,
|
||||
taskListeners, ClusterStateTaskConfig.build(randomFrom(Priority.values())),
|
||||
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 testClusterStateUpdateLogging() throws Exception {
|
||||
MockLogAppender mockAppender = new MockLogAppender();
|
||||
mockAppender.start();
|
||||
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");
|
||||
Loggers.addAppender(clusterLogger, mockAppender);
|
||||
try {
|
||||
final CountDownLatch latch = new CountDownLatch(4);
|
||||
clusterService.currentTimeOverride = System.nanoTime();
|
||||
clusterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
|
||||
return currentState;
|
||||
}
|
||||
public void testBlockingCallInClusterStateTaskListenerFails() throws InterruptedException {
|
||||
assumeTrue("assertions must be enabled for this test to work", BaseFuture.class.desiredAssertionStatus());
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
final AtomicReference<AssertionError> assertionRef = new AtomicReference<>();
|
||||
|
||||
masterService.submitStateUpdateTask(
|
||||
"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) {
|
||||
latch.countDown();
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
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();
|
||||
} finally {
|
||||
Loggers.removeAppender(clusterLogger, mockAppender);
|
||||
mockAppender.stop();
|
||||
}
|
||||
mockAppender.assertAllExpectationsMatched();
|
||||
latch.await();
|
||||
assertNotNull(assertionRef.get());
|
||||
assertThat(assertionRef.get().getMessage(),
|
||||
containsString("Reason: [Blocking operation]"));
|
||||
}
|
||||
|
||||
@TestLogging("org.elasticsearch.cluster.service:WARN") // To ensure that we log cluster state events on WARN level
|
||||
|
@ -577,40 +553,40 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
MockLogAppender mockAppender = new MockLogAppender();
|
||||
mockAppender.start();
|
||||
mockAppender.addExpectation(
|
||||
new MockLogAppender.UnseenEventExpectation(
|
||||
"test1 shouldn't see because setting is too low",
|
||||
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService",
|
||||
Level.WARN,
|
||||
"*cluster state update task [test1] took [*] above the warn threshold of *"));
|
||||
new MockLogAppender.UnseenEventExpectation(
|
||||
"test1 shouldn't see because setting is too low",
|
||||
masterService.getClass().getName(),
|
||||
Level.WARN,
|
||||
"*cluster state update task [test1] took [*] above the warn threshold of *"));
|
||||
mockAppender.addExpectation(
|
||||
new MockLogAppender.SeenEventExpectation(
|
||||
"test2",
|
||||
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService",
|
||||
Level.WARN,
|
||||
"*cluster state update task [test2] took [32s] above the warn threshold of *"));
|
||||
new MockLogAppender.SeenEventExpectation(
|
||||
"test2",
|
||||
masterService.getClass().getName(),
|
||||
Level.WARN,
|
||||
"*cluster state update task [test2] took [32s] above the warn threshold of *"));
|
||||
mockAppender.addExpectation(
|
||||
new MockLogAppender.SeenEventExpectation(
|
||||
"test3",
|
||||
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService",
|
||||
Level.WARN,
|
||||
"*cluster state update task [test3] took [33s] above the warn threshold of *"));
|
||||
new MockLogAppender.SeenEventExpectation(
|
||||
"test3",
|
||||
masterService.getClass().getName(),
|
||||
Level.WARN,
|
||||
"*cluster state update task [test3] took [33s] above the warn threshold of *"));
|
||||
mockAppender.addExpectation(
|
||||
new MockLogAppender.SeenEventExpectation(
|
||||
"test4",
|
||||
"org.elasticsearch.cluster.service.ClusterServiceTests$TimedClusterService",
|
||||
Level.WARN,
|
||||
"*cluster state update task [test4] took [34s] above the warn threshold of *"));
|
||||
new MockLogAppender.SeenEventExpectation(
|
||||
"test4",
|
||||
masterService.getClass().getName(),
|
||||
Level.WARN,
|
||||
"*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);
|
||||
try {
|
||||
final CountDownLatch latch = new CountDownLatch(5);
|
||||
final CountDownLatch processedFirstTask = new CountDownLatch(1);
|
||||
clusterService.currentTimeOverride = System.nanoTime();
|
||||
clusterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
|
||||
masterService.currentTimeOverride = System.nanoTime();
|
||||
masterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
|
||||
masterService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos();
|
||||
return currentState;
|
||||
}
|
||||
|
||||
|
@ -627,10 +603,10 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
});
|
||||
|
||||
processedFirstTask.await();
|
||||
clusterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
|
||||
masterService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
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");
|
||||
}
|
||||
|
||||
|
@ -644,10 +620,10 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
latch.countDown();
|
||||
}
|
||||
});
|
||||
clusterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
|
||||
masterService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
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();
|
||||
}
|
||||
|
||||
|
@ -661,10 +637,10 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
fail();
|
||||
}
|
||||
});
|
||||
clusterService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
|
||||
masterService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
clusterService.currentTimeOverride += TimeValue.timeValueSeconds(34).nanos();
|
||||
masterService.currentTimeOverride += TimeValue.timeValueSeconds(34).nanos();
|
||||
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
|
||||
// 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
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
return currentState;
|
||||
|
@ -704,211 +680,12 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
mockAppender.assertAllExpectationsMatched();
|
||||
}
|
||||
|
||||
public void testDisconnectFromNewlyAddedNodesIfClusterStatePublishingFails() throws InterruptedException {
|
||||
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 {
|
||||
static class TimedMasterService extends MasterService {
|
||||
|
||||
public volatile Long currentTimeOverride = null;
|
||||
|
||||
TimedClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool,
|
||||
Supplier<DiscoveryNode> localNodeSupplier) {
|
||||
super(settings, clusterSettings, threadPool, localNodeSupplier);
|
||||
TimedMasterService(Settings settings, ThreadPool threadPool) {
|
||||
super(settings, threadPool);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -919,4 +696,11 @@ public class ClusterServiceTests extends ESTestCase {
|
|||
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();
|
||||
}
|
||||
}
|
|
@ -29,6 +29,7 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||
|
@ -282,7 +283,7 @@ public class ClusterSettingsIT extends ESIntegTestCase {
|
|||
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() {
|
||||
String key1 = "cluster.routing.allocation.enable";
|
||||
|
|
|
@ -18,47 +18,43 @@
|
|||
*/
|
||||
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.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.network.NetworkService;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.zen.UnicastHostsProvider;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.discovery.zen.ZenPing;
|
||||
import org.elasticsearch.plugins.DiscoveryPlugin;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.NoopDiscovery;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.junit.After;
|
||||
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.when;
|
||||
|
||||
public class DiscoveryModuleTests extends ESTestCase {
|
||||
|
||||
private TransportService transportService;
|
||||
private NamedWriteableRegistry namedWriteableRegistry;
|
||||
private ClusterService clusterService;
|
||||
private MasterService masterService;
|
||||
private ClusterApplier clusterApplier;
|
||||
private ThreadPool threadPool;
|
||||
private ClusterSettings clusterSettings;
|
||||
|
||||
public interface DummyHostsProviderPlugin extends DiscoveryPlugin {
|
||||
Map<String, Supplier<UnicastHostsProvider>> impl();
|
||||
|
@ -74,7 +70,8 @@ public class DiscoveryModuleTests extends ESTestCase {
|
|||
@Override
|
||||
default Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
|
||||
NamedWriteableRegistry namedWriteableRegistry,
|
||||
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
|
||||
MasterService masterService, ClusterApplier clusterApplier,
|
||||
ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
|
||||
return impl();
|
||||
}
|
||||
}
|
||||
|
@ -82,11 +79,11 @@ public class DiscoveryModuleTests extends ESTestCase {
|
|||
@Before
|
||||
public void setupDummyServices() {
|
||||
transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, null, null);
|
||||
clusterService = mock(ClusterService.class);
|
||||
masterService = mock(MasterService.class);
|
||||
namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
|
||||
ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
|
||||
when(clusterService.getClusterSettings()).thenReturn(clusterSettings);
|
||||
clusterApplier = mock(ClusterApplier.class);
|
||||
threadPool = mock(ThreadPool.class);
|
||||
clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -95,7 +92,8 @@ public class DiscoveryModuleTests extends ESTestCase {
|
|||
}
|
||||
|
||||
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() {
|
||||
|
|
|
@ -447,29 +447,30 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
|
|||
ensureGreen("test");
|
||||
|
||||
// verify all cluster states are the same
|
||||
ClusterState state = null;
|
||||
for (String node : nodes) {
|
||||
ClusterState nodeState = getNodeClusterState(node);
|
||||
if (state == null) {
|
||||
state = nodeState;
|
||||
continue;
|
||||
}
|
||||
// assert nodes are identical
|
||||
try {
|
||||
assertEquals("unequal versions", state.version(), nodeState.version());
|
||||
assertEquals("unequal node count", state.nodes().getSize(), nodeState.nodes().getSize());
|
||||
assertEquals("different masters ", state.nodes().getMasterNodeId(), nodeState.nodes().getMasterNodeId());
|
||||
assertEquals("different meta data version", state.metaData().version(), nodeState.metaData().version());
|
||||
if (!state.routingTable().toString().equals(nodeState.routingTable().toString())) {
|
||||
fail("different routing");
|
||||
// use assert busy to wait for cluster states to be applied (as publish_timeout has low value)
|
||||
assertBusy(() -> {
|
||||
ClusterState state = null;
|
||||
for (String node : nodes) {
|
||||
ClusterState nodeState = getNodeClusterState(node);
|
||||
if (state == null) {
|
||||
state = nodeState;
|
||||
continue;
|
||||
}
|
||||
// assert nodes are identical
|
||||
try {
|
||||
assertEquals("unequal versions", state.version(), nodeState.version());
|
||||
assertEquals("unequal node count", state.nodes().getSize(), nodeState.nodes().getSize());
|
||||
assertEquals("different masters ", state.nodes().getMasterNodeId(), nodeState.nodes().getMasterNodeId());
|
||||
assertEquals("different meta data version", state.metaData().version(), nodeState.metaData().version());
|
||||
assertEquals("different routing", state.routingTable().toString(), nodeState.routingTable().toString());
|
||||
} catch (AssertionError t) {
|
||||
fail("failed comparing cluster state: " + t.getMessage() + "\n" +
|
||||
"--- cluster state of node [" + nodes.get(0) + "]: ---\n" + state +
|
||||
"\n--- cluster state [" + node + "]: ---\n" + nodeState);
|
||||
}
|
||||
} catch (AssertionError t) {
|
||||
fail("failed comparing cluster state: " + t.getMessage() + "\n" +
|
||||
"--- cluster state of node [" + nodes.get(0) + "]: ---\n" + state +
|
||||
"\n--- cluster state [" + node + "]: ---\n" + nodeState);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.elasticsearch.cluster.ClusterName;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
|
@ -58,17 +57,14 @@ import java.util.Set;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
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.greaterThanOrEqualTo;
|
||||
|
||||
public class ZenFaultDetectionTests extends ESTestCase {
|
||||
protected ThreadPool threadPool;
|
||||
protected ClusterService clusterServiceA;
|
||||
protected ClusterService clusterServiceB;
|
||||
private CircuitBreakerService circuitBreakerService;
|
||||
|
||||
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();
|
||||
serviceB = build(settingsB, version1);
|
||||
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
|
||||
// will not get this callback called on the connections done in this setup
|
||||
|
@ -133,8 +127,6 @@ public class ZenFaultDetectionTests extends ESTestCase {
|
|||
super.tearDown();
|
||||
serviceA.close();
|
||||
serviceB.close();
|
||||
clusterServiceA.close();
|
||||
clusterServiceB.close();
|
||||
terminate(threadPool);
|
||||
}
|
||||
|
||||
|
@ -241,9 +233,9 @@ public class ZenFaultDetectionTests extends ESTestCase {
|
|||
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m").put("cluster.name", clusterName.value());
|
||||
|
||||
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,
|
||||
clusterServiceA);
|
||||
clusterStateSupplier::get, null, clusterName);
|
||||
masterFD.restart(nodeB, "test");
|
||||
|
||||
final String[] failureReason = new String[1];
|
||||
|
@ -278,7 +270,7 @@ public class ZenFaultDetectionTests extends ESTestCase {
|
|||
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "1s")
|
||||
.put("cluster.name", clusterName.value()).build();
|
||||
final ClusterState stateNodeA = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build();
|
||||
setState(clusterServiceA, stateNodeA);
|
||||
AtomicReference<ClusterState> clusterStateSupplierA = new AtomicReference<>(stateNodeA);
|
||||
|
||||
int minExpectedPings = 2;
|
||||
|
||||
|
@ -289,14 +281,14 @@ public class ZenFaultDetectionTests extends ESTestCase {
|
|||
serviceB.addTracer(pingProbeB);
|
||||
|
||||
MasterFaultDetection masterFDNodeA = new MasterFaultDetection(Settings.builder().put(settingsA).put(settings).build(),
|
||||
threadPool, serviceA, clusterServiceA);
|
||||
threadPool, serviceA, clusterStateSupplierA::get, null, clusterName);
|
||||
masterFDNodeA.restart(nodeB, "test");
|
||||
|
||||
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(),
|
||||
threadPool, serviceB, clusterServiceB);
|
||||
threadPool, serviceB, clusterStateSupplierB::get, null, clusterName);
|
||||
masterFDNodeB.restart(nodeB, "test");
|
||||
|
||||
// let's do a few pings
|
||||
|
|
|
@ -21,14 +21,10 @@ package org.elasticsearch.discovery.single;
|
|||
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
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.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
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.transport.MockTransportService;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
|
@ -36,7 +32,6 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.util.Stack;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
@ -57,37 +52,10 @@ public class SingleNodeDiscoveryTests extends ESTestCase {
|
|||
final ClusterService clusterService = createClusterService(threadPool, node);
|
||||
stack.push(clusterService);
|
||||
final SingleNodeDiscovery discovery =
|
||||
new SingleNodeDiscovery(Settings.EMPTY, clusterService);
|
||||
new SingleNodeDiscovery(Settings.EMPTY, transportService,
|
||||
clusterService.getClusterApplierService());
|
||||
discovery.startInitialJoin();
|
||||
|
||||
// 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();
|
||||
final DiscoveryNodes nodes = discovery.getInitialClusterState().nodes();
|
||||
assertThat(nodes.getSize(), equalTo(1));
|
||||
assertThat(nodes.getMasterNode().getId(), equalTo(node.getId()));
|
||||
} finally {
|
||||
|
|
|
@ -22,8 +22,10 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
|
|||
import org.apache.logging.log4j.util.Supplier;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.NotMasterException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
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.TestShardRouting;
|
||||
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.cluster.service.MasterService;
|
||||
import org.elasticsearch.cluster.service.MasterServiceTests;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.common.util.concurrent.BaseFuture;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.test.ClusterServiceUtils;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.VersionUtils;
|
||||
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_VERSION_CREATED;
|
||||
import static org.elasticsearch.cluster.routing.RoutingTableTests.updateActiveAllocations;
|
||||
import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
|
||||
import static org.elasticsearch.test.ClusterServiceUtils.setState;
|
||||
import static org.elasticsearch.cluster.service.MasterServiceTests.discoveryState;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.empty;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
@ -93,12 +94,12 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
|
||||
private static ThreadPool threadPool;
|
||||
|
||||
private ClusterService clusterService;
|
||||
private MasterService masterService;
|
||||
private NodeJoinController nodeJoinController;
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() {
|
||||
threadPool = new TestThreadPool("ShardReplicationTests");
|
||||
threadPool = new TestThreadPool("NodeJoinControllerTests");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -110,25 +111,39 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
@Before
|
||||
public void setUp() throws Exception {
|
||||
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
|
||||
public void tearDown() throws Exception {
|
||||
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 {
|
||||
setupMasterServiceAndNodeJoinController(initialState(true));
|
||||
List<DiscoveryNode> nodes = new ArrayList<>();
|
||||
nodes.add(clusterService.localNode());
|
||||
nodes.add(discoveryState(masterService).nodes().getLocalNode());
|
||||
|
||||
int nodeId = 0;
|
||||
for (int i = randomInt(5); i > 0; i--) {
|
||||
|
@ -162,9 +177,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testFailingJoinsWhenNotMaster() throws ExecutionException, InterruptedException {
|
||||
// remove current master flag
|
||||
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null);
|
||||
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
|
||||
setupMasterServiceAndNodeJoinController(initialState(false));
|
||||
int nodeId = 0;
|
||||
try {
|
||||
joinNode(newNode(nodeId++));
|
||||
|
@ -194,8 +207,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testSimpleMasterElectionWithoutRequiredJoins() throws InterruptedException, ExecutionException {
|
||||
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null);
|
||||
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
|
||||
setupMasterServiceAndNodeJoinController(initialState(false));
|
||||
int nodeId = 0;
|
||||
final int requiredJoins = 0;
|
||||
logger.debug("--> using requiredJoins [{}]", requiredJoins);
|
||||
|
@ -244,8 +256,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testSimpleMasterElection() throws InterruptedException, ExecutionException {
|
||||
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null);
|
||||
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
|
||||
setupMasterServiceAndNodeJoinController(initialState(false));
|
||||
int nodeId = 0;
|
||||
final int requiredJoins = 1 + randomInt(5);
|
||||
logger.debug("--> using requiredJoins [{}]", requiredJoins);
|
||||
|
@ -356,10 +367,8 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
|
||||
}
|
||||
|
||||
|
||||
public void testMasterElectionTimeout() throws InterruptedException {
|
||||
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null);
|
||||
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodes));
|
||||
setupMasterServiceAndNodeJoinController(initialState(false));
|
||||
int nodeId = 0;
|
||||
final int requiredJoins = 1 + randomInt(5);
|
||||
logger.debug("--> using requiredJoins [{}]", requiredJoins);
|
||||
|
@ -422,22 +431,23 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, ExecutionException {
|
||||
ClusterState state = clusterService.state();
|
||||
ClusterState state = initialState(true);
|
||||
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes());
|
||||
final DiscoveryNode other_node = new DiscoveryNode("other_node", buildNewFakeTransportAddress(),
|
||||
emptyMap(), emptySet(), Version.CURRENT);
|
||||
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);
|
||||
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 {
|
||||
setupMasterServiceAndNodeJoinController(initialState(true));
|
||||
Thread[] threads = new Thread[3 + randomInt(5)];
|
||||
ArrayList<DiscoveryNode> nodes = new ArrayList<>();
|
||||
nodes.add(clusterService.localNode());
|
||||
nodes.add(discoveryState(masterService).nodes().getLocalNode());
|
||||
final CyclicBarrier barrier = new CyclicBarrier(threads.length);
|
||||
final List<Throwable> backgroundExceptions = new CopyOnWriteArrayList<>();
|
||||
for (int i = 0; i < threads.length; i++) {
|
||||
|
@ -472,15 +482,14 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testElectionWithConcurrentJoins() throws InterruptedException, BrokenBarrierException {
|
||||
DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null);
|
||||
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(nodesBuilder));
|
||||
setupMasterServiceAndNodeJoinController(initialState(false));
|
||||
|
||||
nodeJoinController.startElectionContext();
|
||||
|
||||
Thread[] threads = new Thread[3 + randomInt(5)];
|
||||
final int requiredJoins = randomInt(threads.length);
|
||||
ArrayList<DiscoveryNode> nodes = new ArrayList<>();
|
||||
nodes.add(clusterService.localNode());
|
||||
nodes.add(discoveryState(masterService).nodes().getLocalNode());
|
||||
final CyclicBarrier barrier = new CyclicBarrier(threads.length + 1);
|
||||
final List<Throwable> backgroundExceptions = new CopyOnWriteArrayList<>();
|
||||
for (int i = 0; i < threads.length; i++) {
|
||||
|
@ -539,7 +548,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
|
||||
public void testRejectingJoinWithSameAddressButDifferentId() throws InterruptedException, ExecutionException {
|
||||
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 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 {
|
||||
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 other_node = new DiscoveryNode(
|
||||
randomBoolean() ? existing.getName() : "other_name",
|
||||
|
@ -565,7 +574,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
|
||||
public void testRejectingRestartedNodeJoinsBeforeProcessingNodeLeft() throws InterruptedException, ExecutionException {
|
||||
addNodes(randomInt(5));
|
||||
ClusterState state = clusterService.state();
|
||||
ClusterState state = discoveryState(masterService);
|
||||
final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList()));
|
||||
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
|
||||
*/
|
||||
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(),
|
||||
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)
|
||||
// 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.add(otherNode);
|
||||
ClusterState.Builder stateBuilder = ClusterState.builder(clusterService.state()).nodes(discoBuilder);
|
||||
ClusterState.Builder stateBuilder = ClusterState.builder(initialState).nodes(discoBuilder);
|
||||
if (randomBoolean()) {
|
||||
IndexMetaData indexMetaData = IndexMetaData.builder("test").settings(Settings.builder()
|
||||
.put(SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
|
@ -623,7 +633,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
.routingTable(RoutingTable.builder().add(indexRoutingTable).build());
|
||||
}
|
||||
|
||||
setState(clusterService, stateBuilder.build());
|
||||
setupMasterServiceAndNodeJoinController(stateBuilder.build());
|
||||
|
||||
// conflict on node id or address
|
||||
final DiscoveryNode conflictingNode = randomBoolean() ?
|
||||
|
@ -652,7 +662,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
|
||||
joinFuture.get(); // throw any exception
|
||||
|
||||
final ClusterState finalState = clusterService.state();
|
||||
final ClusterState finalState = discoveryState(masterService);
|
||||
final DiscoveryNodes finalNodes = finalState.nodes();
|
||||
assertTrue(finalNodes.isLocalNodeElectedMaster());
|
||||
assertThat(finalNodes.getLocalNode(), equalTo(masterNode));
|
||||
|
@ -666,18 +676,18 @@ public class NodeJoinControllerTests extends ESTestCase {
|
|||
|
||||
|
||||
private void addNodes(int count) {
|
||||
ClusterState state = clusterService.state();
|
||||
ClusterState state = initialState(true);
|
||||
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes());
|
||||
for (int i = 0;i< count;i++) {
|
||||
final DiscoveryNode node = new DiscoveryNode("node_" + state.nodes().getSize() + i, buildNewFakeTransportAddress(),
|
||||
emptyMap(), new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), Version.CURRENT);
|
||||
nodesBuilder.add(node);
|
||||
}
|
||||
setState(clusterService, ClusterState.builder(state).nodes(nodesBuilder));
|
||||
setupMasterServiceAndNodeJoinController(ClusterState.builder(state).nodes(nodesBuilder).build());
|
||||
}
|
||||
|
||||
protected void assertNodesInCurrentState(List<DiscoveryNode> expectedNodes) {
|
||||
final ClusterState state = clusterService.state();
|
||||
final ClusterState state = discoveryState(masterService);
|
||||
logger.info("assert for [{}] in:\n{}", expectedNodes, state);
|
||||
DiscoveryNodes discoveryNodes = state.nodes();
|
||||
for (DiscoveryNode node : expectedNodes) {
|
||||
|
|
|
@ -260,6 +260,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
|
|||
"}";
|
||||
|
||||
internalCluster().startNode();
|
||||
ensureGreen(); // ensures that all events are processed (in particular state recovery fully completed)
|
||||
|
||||
logger.info("--> request node discovery stats");
|
||||
NodesStatsResponse statsResponse = client().admin().cluster().prepareNodesStats().clear().setDiscovery(true).get();
|
||||
|
|
|
@ -37,12 +37,14 @@ import org.elasticsearch.cluster.routing.RoutingTable;
|
|||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||
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.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.zen.PublishClusterStateActionTests.AssertingAckListener;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.test.ClusterServiceUtils;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.VersionUtils;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
|
@ -65,6 +67,7 @@ import java.util.List;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
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_VERSION_CREATED;
|
||||
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.ZenDiscovery.shouldIgnoreOrRejectNewClusterState;
|
||||
import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
|
||||
import static org.elasticsearch.test.ClusterServiceUtils.setState;
|
||||
import static org.hamcrest.Matchers.arrayWithSize;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
|
@ -182,26 +185,31 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
|
|||
DiscoveryNode masterNode = masterTransport.getLocalNode();
|
||||
toClose.addFirst(masterTransport);
|
||||
ClusterState state = ClusterStateCreationUtils.state(masterNode, masterNode, masterNode);
|
||||
// build the zen discovery and cluster service
|
||||
ClusterService masterClusterService = createClusterService(threadPool, masterNode);
|
||||
toClose.addFirst(masterClusterService);
|
||||
// build the zen discovery and discovery service
|
||||
MasterService masterMasterService = ClusterServiceUtils.createMasterService(threadPool, masterNode);
|
||||
toClose.addFirst(masterMasterService);
|
||||
// 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();
|
||||
setState(masterClusterService, state);
|
||||
ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterClusterService, threadPool);
|
||||
state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build();
|
||||
Settings settingsWithClusterName = Settings.builder().put(settings).put(
|
||||
ClusterName.CLUSTER_NAME_SETTING.getKey(), discoveryState(masterMasterService).getClusterName().value()).build();
|
||||
ZenDiscovery masterZen = buildZenDiscovery(
|
||||
settingsWithClusterName,
|
||||
masterTransport, masterMasterService, threadPool);
|
||||
masterZen.setState(state);
|
||||
toClose.addFirst(masterZen);
|
||||
masterTransport.acceptIncomingRequests();
|
||||
|
||||
final MockTransportService otherTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null);
|
||||
otherTransport.start();
|
||||
toClose.addFirst(otherTransport);
|
||||
|
||||
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();
|
||||
ClusterService otherClusterService = createClusterService(threadPool, masterNode);
|
||||
toClose.addFirst(otherClusterService);
|
||||
setState(otherClusterService, otherState);
|
||||
ZenDiscovery otherZen = buildZenDiscovery(settings, otherTransport, otherClusterService, threadPool);
|
||||
MasterService otherMasterService = ClusterServiceUtils.createMasterService(threadPool, otherNode);
|
||||
toClose.addFirst(otherMasterService);
|
||||
ZenDiscovery otherZen = buildZenDiscovery(settingsWithClusterName, otherTransport, otherMasterService, threadPool);
|
||||
otherZen.setState(otherState);
|
||||
toClose.addFirst(otherZen);
|
||||
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
|
||||
// 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())
|
||||
).build();
|
||||
|
||||
|
@ -220,7 +228,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
|
|||
AssertingAckListener listener = new AssertingAckListener(newState.nodes().getSize() - 1);
|
||||
expectedFDNodes = masterZen.getFaultDetectionNodes();
|
||||
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
|
||||
expectedFDNodes = fdNodesForState(newState, masterNode);
|
||||
} catch (Discovery.FailedToCommitClusterStateException e) {
|
||||
|
@ -249,12 +257,12 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
|
|||
DiscoveryNode masterNode = masterTransport.getLocalNode();
|
||||
toClose.addFirst(masterTransport);
|
||||
ClusterState state = ClusterStateCreationUtils.state(masterNode, null, masterNode);
|
||||
// build the zen discovery and cluster service
|
||||
ClusterService masterClusterService = createClusterService(threadPool, masterNode);
|
||||
toClose.addFirst(masterClusterService);
|
||||
state = ClusterState.builder(masterClusterService.getClusterName()).nodes(state.nodes()).build();
|
||||
setState(masterClusterService, state);
|
||||
ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterClusterService, threadPool);
|
||||
// build the zen discovery and master service for the master node
|
||||
MasterService masterMasterService = ClusterServiceUtils.createMasterService(threadPool, masterNode);
|
||||
toClose.addFirst(masterMasterService);
|
||||
state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build();
|
||||
ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterMasterService, threadPool);
|
||||
masterZen.setState(state);
|
||||
toClose.addFirst(masterZen);
|
||||
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
|
||||
// was updated by the presence of this node in NodesFaultDetection)
|
||||
ClusterState newState = ClusterState.builder(masterClusterService.state()).incrementVersion().nodes(
|
||||
DiscoveryNodes.builder(masterClusterService.state().nodes()).masterNodeId(masterNode.getId())
|
||||
ClusterState newState = ClusterState.builder(discoveryState(masterMasterService)).incrementVersion().nodes(
|
||||
DiscoveryNodes.builder(discoveryState(masterMasterService).nodes()).masterNodeId(masterNode.getId())
|
||||
).build();
|
||||
|
||||
|
||||
|
@ -277,9 +285,8 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
|
|||
// publish was a success, check that queue as cleared
|
||||
assertThat(masterZen.pendingClusterStates(), emptyArray());
|
||||
} catch (Discovery.FailedToCommitClusterStateException e) {
|
||||
// not successful, so the pending queue should stay
|
||||
assertThat(masterZen.pendingClusterStates(), arrayWithSize(1));
|
||||
assertThat(masterZen.pendingClusterStates()[0].getClusterName().value(), equalTo("foreign"));
|
||||
// not successful, so the pending queue should be cleaned
|
||||
assertThat(Arrays.toString(masterZen.pendingClusterStates()), masterZen.pendingClusterStates(), arrayWithSize(0));
|
||||
}
|
||||
} finally {
|
||||
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()),
|
||||
clusterService, Collections::emptyList);
|
||||
masterService, (source, clusterStateSupplier, listener) -> listener.clusterStateProcessed(source, clusterStateSupplier.get(), clusterStateSupplier.get()),
|
||||
clusterSettings, Collections::emptyList);
|
||||
zenDiscovery.start();
|
||||
return zenDiscovery;
|
||||
}
|
||||
|
|
|
@ -19,10 +19,7 @@
|
|||
|
||||
package org.elasticsearch.gateway;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
@ -37,7 +34,7 @@ public class GatewayServiceTests extends ESTestCase {
|
|||
private GatewayService createService(Settings.Builder settings) {
|
||||
ClusterService clusterService = new ClusterService(Settings.builder().put("cluster.name", "GatewayServiceTests").build(),
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
|
||||
null, () -> new DiscoveryNode(UUIDs.randomBase64UUID(), buildNewFakeTransportAddress(), Version.CURRENT));
|
||||
null);
|
||||
return new GatewayService(settings.build(),
|
||||
null, clusterService, null, null, null, new NoopDiscovery(), null);
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.elasticsearch.test.ESIntegTestCase;
|
|||
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||
import org.elasticsearch.test.ESIntegTestCase.Scope;
|
||||
import org.elasticsearch.test.InternalTestCluster;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -204,8 +205,8 @@ public class IndexLifecycleActionIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
private String getLocalNodeId(String name) {
|
||||
Discovery discovery = internalCluster().getInstance(Discovery.class, name);
|
||||
String nodeId = discovery.localNode().getId();
|
||||
TransportService transportService = internalCluster().getInstance(TransportService.class, name);
|
||||
String nodeId = transportService.getLocalNode().getId();
|
||||
assertThat(nodeId, not(nullValue()));
|
||||
return nodeId;
|
||||
}
|
||||
|
|
|
@ -20,9 +20,12 @@
|
|||
package org.elasticsearch.indices.store;
|
||||
|
||||
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.state.ClusterStateResponse;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateTaskListener;
|
||||
import org.elasticsearch.cluster.LocalClusterUpdateTask;
|
||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
||||
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.allocation.command.MoveAllocationCommand;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||
import org.elasticsearch.cluster.service.ClusterApplierService;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Priority;
|
||||
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
|
||||
// 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();
|
||||
internalCluster().getInstance(ClusterService.class, nonMasterNode).submitStateUpdateTask("test", new LocalClusterUpdateTask(Priority.IMMEDIATE) {
|
||||
|
||||
ClusterApplierService clusterApplierService = internalCluster().getInstance(ClusterService.class, nonMasterNode).getClusterApplierService();
|
||||
ClusterState currentState = clusterApplierService.state();
|
||||
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index);
|
||||
for (int j = 0; j < numShards; j++) {
|
||||
indexRoutingTableBuilder.addIndexShard(
|
||||
new IndexShardRoutingTable.Builder(new ShardId(index, j))
|
||||
.addShard(TestShardRouting.newShardRouting("test", j, masterId, true, ShardRoutingState.STARTED))
|
||||
.build()
|
||||
);
|
||||
}
|
||||
ClusterState newState = ClusterState.builder(currentState)
|
||||
.incrementVersion()
|
||||
.routingTable(RoutingTable.builder().add(indexRoutingTableBuilder).build())
|
||||
.build();
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
clusterApplierService.onNewClusterState("test", () -> newState, new ClusterStateTaskListener() {
|
||||
@Override
|
||||
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
|
||||
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index);
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
indexRoutingTableBuilder.addIndexShard(
|
||||
new IndexShardRoutingTable.Builder(new ShardId(index, i))
|
||||
.addShard(TestShardRouting.newShardRouting("test", i, masterId, true, ShardRoutingState.STARTED))
|
||||
.build()
|
||||
);
|
||||
}
|
||||
return newState(ClusterState.builder(currentState)
|
||||
.routingTable(RoutingTable.builder().add(indexRoutingTableBuilder).build())
|
||||
.build());
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
latch.countDown();
|
||||
fail("Excepted proper response " + ExceptionsHelper.detailedMessage(e));
|
||||
}
|
||||
});
|
||||
latch.await();
|
||||
waitNoPendingTasksOnAll();
|
||||
logger.info("Checking if shards aren't removed");
|
||||
for (int shard : node2Shards) {
|
||||
|
|
|
@ -59,6 +59,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
|
|||
assertThat(geoCentroid.getName(), equalTo(aggName));
|
||||
GeoPoint centroid = geoCentroid.centroid();
|
||||
assertThat(centroid, equalTo(null));
|
||||
assertEquals(0, geoCentroid.count());
|
||||
}
|
||||
|
||||
public void testUnmapped() throws Exception {
|
||||
|
@ -72,6 +73,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
|
|||
assertThat(geoCentroid.getName(), equalTo(aggName));
|
||||
GeoPoint centroid = geoCentroid.centroid();
|
||||
assertThat(centroid, equalTo(null));
|
||||
assertEquals(0, geoCentroid.count());
|
||||
}
|
||||
|
||||
public void testPartiallyUnmapped() throws Exception {
|
||||
|
@ -86,6 +88,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
|
|||
GeoPoint centroid = geoCentroid.centroid();
|
||||
assertThat(centroid.lat(), closeTo(singleCentroid.lat(), GEOHASH_TOLERANCE));
|
||||
assertThat(centroid.lon(), closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE));
|
||||
assertEquals(numDocs, geoCentroid.count());
|
||||
}
|
||||
|
||||
public void testSingleValuedField() throws Exception {
|
||||
|
@ -101,6 +104,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
|
|||
GeoPoint centroid = geoCentroid.centroid();
|
||||
assertThat(centroid.lat(), closeTo(singleCentroid.lat(), GEOHASH_TOLERANCE));
|
||||
assertThat(centroid.lon(), closeTo(singleCentroid.lon(), GEOHASH_TOLERANCE));
|
||||
assertEquals(numDocs, geoCentroid.count());
|
||||
}
|
||||
|
||||
public void testSingleValueFieldGetProperty() throws Exception {
|
||||
|
@ -130,6 +134,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
|
|||
closeTo(singleCentroid.lon(), 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));
|
||||
assertEquals(numDocs, (long) ((InternalAggregation) global).getProperty(aggName + ".count"));
|
||||
}
|
||||
|
||||
public void testMultiValuedField() throws Exception {
|
||||
|
@ -145,6 +150,7 @@ public class GeoCentroidIT extends AbstractGeoTestCase {
|
|||
GeoPoint centroid = geoCentroid.centroid();
|
||||
assertThat(centroid.lat(), closeTo(multiCentroid.lat(), GEOHASH_TOLERANCE));
|
||||
assertThat(centroid.lon(), closeTo(multiCentroid.lon(), GEOHASH_TOLERANCE));
|
||||
assertEquals(2 * numDocs, geoCentroid.count());
|
||||
}
|
||||
|
||||
public void testSingleValueFieldAsSubAggToGeohashGrid() throws Exception {
|
||||
|
|
|
@ -42,8 +42,11 @@ public class InternalGeoCentroidTests extends InternalAggregationTestCase<Intern
|
|||
centroid.resetLon(GeoEncodingUtils.decodeLongitude(encodedLon));
|
||||
int encodedLat = GeoEncodingUtils.encodeLatitude(centroid.lat());
|
||||
centroid.resetLat(GeoEncodingUtils.decodeLatitude(encodedLat));
|
||||
|
||||
return new InternalGeoCentroid("_name", centroid, 1, Collections.emptyList(), Collections.emptyMap());
|
||||
long count = randomIntBetween(0, 1000);
|
||||
if (count == 0) {
|
||||
centroid = null;
|
||||
}
|
||||
return new InternalGeoCentroid("_name", centroid, count, Collections.emptyList(), Collections.emptyMap());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -53,14 +56,18 @@ public class InternalGeoCentroidTests extends InternalAggregationTestCase<Intern
|
|||
|
||||
@Override
|
||||
protected void assertReduced(InternalGeoCentroid reduced, List<InternalGeoCentroid> inputs) {
|
||||
GeoPoint expected = new GeoPoint(0, 0);
|
||||
int i = 0;
|
||||
double lonSum = 0;
|
||||
double latSum = 0;
|
||||
int totalCount = 0;
|
||||
for (InternalGeoCentroid input : inputs) {
|
||||
expected.reset(expected.lat() + (input.centroid().lat() - expected.lat()) / (i+1),
|
||||
expected.lon() + (input.centroid().lon() - expected.lon()) / (i+1));
|
||||
i++;
|
||||
if (input.count() > 0) {
|
||||
lonSum += (input.count() * input.centroid().getLon());
|
||||
latSum += (input.count() * input.centroid().getLat());
|
||||
}
|
||||
totalCount += input.count();
|
||||
}
|
||||
assertEquals(expected.getLat(), reduced.centroid().getLat(), 1E-5D);
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,19 +18,9 @@
|
|||
*/
|
||||
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.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.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.unit.TimeValue;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
|
@ -47,13 +37,10 @@ import java.nio.file.attribute.BasicFileAttributes;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
|
||||
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) {
|
||||
((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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -724,14 +724,9 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
|
|||
}
|
||||
|
||||
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");
|
||||
internalCluster().startNode(masterSettings);
|
||||
internalCluster().startNode(masterSettings);
|
||||
internalCluster().startNode(dataSettings);
|
||||
internalCluster().startNode(dataSettings);
|
||||
internalCluster().startMasterOnlyNodes(2);
|
||||
internalCluster().startDataOnlyNodes(2);
|
||||
|
||||
final Client client = client();
|
||||
|
||||
|
@ -758,35 +753,17 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
|
|||
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");
|
||||
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");
|
||||
internalCluster().stopCurrentMasterNode();
|
||||
|
||||
logger.info("--> unblocking snapshot execution");
|
||||
snapshotListener.unblock();
|
||||
|
||||
} finally {
|
||||
clusterService.removeListener(snapshotListener);
|
||||
}
|
||||
logger.info("--> stopping master node");
|
||||
internalCluster().stopCurrentMasterNode();
|
||||
|
||||
logger.info("--> wait until the snapshot is done");
|
||||
|
||||
assertBusy(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
GetSnapshotsResponse snapshotsStatusResponse = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get();
|
||||
SnapshotInfo snapshotInfo = snapshotsStatusResponse.getSnapshots().get(0);
|
||||
assertTrue(snapshotInfo.state().completed());
|
||||
}
|
||||
assertBusy(() -> {
|
||||
GetSnapshotsResponse snapshotsStatusResponse = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get();
|
||||
SnapshotInfo snapshotInfo = snapshotsStatusResponse.getSnapshots().get(0);
|
||||
assertTrue(snapshotInfo.state().completed());
|
||||
}, 1, TimeUnit.MINUTES);
|
||||
|
||||
logger.info("--> verify that snapshot was succesful");
|
||||
|
@ -865,7 +842,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
|
|||
restoreResponse.getRestoreInfo().successfulShards());
|
||||
ensureYellow();
|
||||
}
|
||||
|
||||
|
||||
public static class SnapshottableMetadata extends TestCustomMetaData {
|
||||
public static final String TYPE = "test_snapshottable";
|
||||
|
||||
|
|
|
@ -2286,59 +2286,6 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
|
|||
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 {
|
||||
final Client client = client();
|
||||
|
||||
|
|
|
@ -19,27 +19,15 @@
|
|||
package org.elasticsearch.test;
|
||||
|
||||
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.common.component.Lifecycle;
|
||||
import org.elasticsearch.common.component.LifecycleListener;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.DiscoveryStats;
|
||||
|
||||
public class NoopDiscovery implements Discovery {
|
||||
|
||||
|
||||
@Override
|
||||
public DiscoveryNode localNode() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String nodeDescription() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAllocationService(AllocationService allocationService) {
|
||||
|
||||
|
@ -51,12 +39,17 @@ public class NoopDiscovery implements Discovery {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DiscoveryStats stats() {
|
||||
public ClusterState getInitialClusterState() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DiscoverySettings getDiscoverySettings() {
|
||||
public ClusterState clusterState() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DiscoveryStats stats() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -491,61 +491,7 @@ public class UpdateIT extends ESIntegTestCase {
|
|||
assertThat(updateResponse.getGetResult().sourceAsMap().get("bar").toString(), equalTo("baz"));
|
||||
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 {
|
||||
UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1")
|
||||
.setUpsert(XContentFactory.jsonBuilder().startObject().field("bar", "baz").endObject())
|
||||
|
|
|
@ -62,7 +62,8 @@ The response for the above aggregation:
|
|||
"location": {
|
||||
"lat": 51.00982963107526,
|
||||
"lon": 3.9662130922079086
|
||||
}
|
||||
},
|
||||
"count": 6
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -114,7 +115,8 @@ The response for the above aggregation:
|
|||
"location": {
|
||||
"lat": 52.371655656024814,
|
||||
"lon": 4.909563297405839
|
||||
}
|
||||
},
|
||||
"count": 3
|
||||
}
|
||||
},
|
||||
{
|
||||
|
@ -124,7 +126,8 @@ The response for the above aggregation:
|
|||
"location": {
|
||||
"lat": 48.86055548675358,
|
||||
"lon": 2.3316944623366
|
||||
}
|
||||
},
|
||||
"count": 2
|
||||
}
|
||||
},
|
||||
{
|
||||
|
@ -134,7 +137,8 @@ The response for the above aggregation:
|
|||
"location": {
|
||||
"lat": 51.22289997059852,
|
||||
"lon": 4.40519998781383
|
||||
}
|
||||
},
|
||||
"count": 1
|
||||
}
|
||||
}
|
||||
]
|
||||
|
|
|
@ -54,5 +54,6 @@ alias4 test1 - 2 1,2
|
|||
The output shows that `alias2` has configured a filter, and specific routing
|
||||
configurations in `alias3` and `alias4`.
|
||||
|
||||
If you only want to get information about a single alias, you can specify
|
||||
the alias in the URL, for example `/_cat/aliases/alias1`.
|
||||
If you only want to get information about specific aliases, you can specify
|
||||
the aliases in comma-delimited format as a URL parameter, e.g.,
|
||||
/_cat/aliases/aliases/alias1,alias2.
|
|
@ -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
|
||||
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:
|
||||
|
||||
|
@ -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
|
||||
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
|
||||
across nodes:
|
||||
|
|
|
@ -5,3 +5,7 @@
|
|||
|
||||
Document modification operations may no longer specify the `version_type` of
|
||||
`force` to override any previous version checks.
|
||||
|
||||
==== <<upserts>> no longer support versions
|
||||
|
||||
Adding a `version` to an upsert request is no longer supported.
|
||||
|
|
|
@ -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
|
||||
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 levels of nested object fields and inner hits.
|
||||
|
||||
|
|
|
@ -19,22 +19,16 @@
|
|||
|
||||
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.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.cloud.azure.classic.management.AzureComputeService;
|
||||
import org.elasticsearch.cloud.azure.classic.management.AzureComputeServiceImpl;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.cluster.service.ClusterApplier;
|
||||
import org.elasticsearch.cluster.service.MasterService;
|
||||
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;
|
||||
|
@ -42,12 +36,17 @@ import org.elasticsearch.discovery.DiscoveryModule;
|
|||
import org.elasticsearch.discovery.azure.classic.AzureUnicastHostsProvider;
|
||||
import org.elasticsearch.discovery.zen.UnicastHostsProvider;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.discovery.zen.ZenPing;
|
||||
import org.elasticsearch.plugins.DiscoveryPlugin;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
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 static final String AZURE = "azure";
|
||||
|
@ -76,10 +75,12 @@ public class AzureDiscoveryPlugin extends Plugin implements DiscoveryPlugin {
|
|||
@Override
|
||||
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
|
||||
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
|
||||
return Collections.singletonMap(AZURE, () ->
|
||||
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
|
||||
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
|
||||
clusterSettings, hostsProvider));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,6 +20,30 @@
|
|||
package org.elasticsearch.discovery.ec2;
|
||||
|
||||
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.Closeable;
|
||||
import java.io.IOException;
|
||||
|
@ -37,28 +61,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
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 {
|
||||
|
||||
private static Logger logger = Loggers.getLogger(Ec2DiscoveryPlugin.class);
|
||||
|
@ -95,10 +97,12 @@ public class Ec2DiscoveryPlugin extends Plugin implements DiscoveryPlugin, Close
|
|||
@Override
|
||||
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
|
||||
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
|
||||
return Collections.singletonMap(EC2, () ->
|
||||
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
|
||||
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
|
||||
clusterSettings, hostsProvider));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,28 +24,25 @@ import com.google.api.client.util.ClassInfo;
|
|||
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.cloud.gce.GceInstancesService;
|
||||
import org.elasticsearch.cloud.gce.GceInstancesServiceImpl;
|
||||
import org.elasticsearch.cloud.gce.GceMetadataService;
|
||||
import org.elasticsearch.cloud.gce.GceModule;
|
||||
import org.elasticsearch.cloud.gce.network.GceNameResolver;
|
||||
import org.elasticsearch.cloud.gce.util.Access;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.component.LifecycleComponent;
|
||||
import org.elasticsearch.common.inject.Module;
|
||||
import org.elasticsearch.cluster.service.ClusterApplier;
|
||||
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.gce.GceUnicastHostsProvider;
|
||||
import org.elasticsearch.discovery.zen.UnicastHostsProvider;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.discovery.zen.ZenPing;
|
||||
import org.elasticsearch.plugins.DiscoveryPlugin;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -53,11 +50,7 @@ import org.elasticsearch.transport.TransportService;
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.security.AccessController;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -92,10 +85,12 @@ public class GceDiscoveryPlugin extends Plugin implements DiscoveryPlugin, Close
|
|||
@Override
|
||||
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
|
||||
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
|
||||
return Collections.singletonMap(GCE, () ->
|
||||
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
|
||||
new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
|
||||
clusterSettings, hostsProvider));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -126,6 +126,52 @@
|
|||
- match:
|
||||
$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":
|
||||
|
||||
|
|
|
@ -18,82 +18,90 @@
|
|||
*/
|
||||
package org.elasticsearch.test;
|
||||
|
||||
import org.apache.logging.log4j.core.util.Throwables;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
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.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
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.MasterService;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.Discovery.AckListener;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumSet;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.BiConsumer;
|
||||
|
||||
import static junit.framework.TestCase.fail;
|
||||
|
||||
public class ClusterServiceUtils {
|
||||
|
||||
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 MasterService createMasterService(ThreadPool threadPool, ClusterState initialClusterState) {
|
||||
MasterService masterService = new MasterService(Settings.EMPTY, threadPool);
|
||||
AtomicReference<ClusterState> clusterStateRef = new AtomicReference<>(initialClusterState);
|
||||
masterService.setClusterStatePublisher((event, ackListener) -> clusterStateRef.set(event.state()));
|
||||
masterService.setClusterStateSupplier(clusterStateRef::get);
|
||||
masterService.start();
|
||||
return masterService;
|
||||
}
|
||||
|
||||
public static ClusterService createClusterService(ThreadPool threadPool, DiscoveryNode localNode) {
|
||||
return createClusterService(Settings.EMPTY, threadPool, localNode);
|
||||
public static MasterService createMasterService(ThreadPool threadPool, DiscoveryNode 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) {
|
||||
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) {
|
||||
public static void setState(ClusterApplierService executor, ClusterState clusterState) {
|
||||
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
|
||||
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 ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
|
||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||
// 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
|
||||
|
@ -112,4 +120,88 @@ public class ClusterServiceUtils {
|
|||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -570,7 +570,8 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery;
|
||||
assertBusy(() -> {
|
||||
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")),
|
||||
states, emptyArray());
|
||||
});
|
||||
|
|
|
@ -24,8 +24,10 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
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.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
|
@ -56,9 +58,11 @@ public class TestZenDiscovery extends ZenDiscovery {
|
|||
@Override
|
||||
public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
|
||||
NamedWriteableRegistry namedWriteableRegistry,
|
||||
ClusterService clusterService, UnicastHostsProvider hostsProvider) {
|
||||
MasterService masterService, ClusterApplier clusterApplier,
|
||||
ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
|
||||
return Collections.singletonMap("test-zen",
|
||||
() -> new TestZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider));
|
||||
() -> new TestZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService,
|
||||
clusterApplier, clusterSettings, hostsProvider));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -73,9 +77,10 @@ public class TestZenDiscovery extends ZenDiscovery {
|
|||
}
|
||||
|
||||
private TestZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService,
|
||||
NamedWriteableRegistry namedWriteableRegistry, ClusterService clusterService,
|
||||
UnicastHostsProvider hostsProvider) {
|
||||
super(settings, threadPool, transportService, namedWriteableRegistry, clusterService, hostsProvider);
|
||||
NamedWriteableRegistry namedWriteableRegistry, MasterService masterService,
|
||||
ClusterApplier clusterApplier, ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider) {
|
||||
super(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier, clusterSettings,
|
||||
hostsProvider);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -18,8 +18,7 @@
|
|||
*/
|
||||
package org.elasticsearch.test.disruption;
|
||||
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.LocalClusterUpdateTask;
|
||||
import org.apache.logging.log4j.core.util.Throwables;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
@ -58,23 +57,19 @@ public class BlockClusterStateProcessing extends SingleNodeDisruption {
|
|||
boolean success = disruptionLatch.compareAndSet(null, new CountDownLatch(1));
|
||||
assert success : "startDisrupting called without waiting on stopDisrupting to complete";
|
||||
final CountDownLatch started = new CountDownLatch(1);
|
||||
clusterService.submitStateUpdateTask("service_disruption_block", new LocalClusterUpdateTask(Priority.IMMEDIATE) {
|
||||
|
||||
@Override
|
||||
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
|
||||
clusterService.getClusterApplierService().runOnApplierThread("service_disruption_block",
|
||||
currentState -> {
|
||||
started.countDown();
|
||||
CountDownLatch latch = disruptionLatch.get();
|
||||
if (latch != null) {
|
||||
latch.await();
|
||||
try {
|
||||
latch.await();
|
||||
} catch (InterruptedException e) {
|
||||
Throwables.rethrow(e);
|
||||
}
|
||||
}
|
||||
return unchanged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
logger.error("unexpected error during disruption", e);
|
||||
}
|
||||
});
|
||||
}, (source, e) -> logger.error("unexpected error during disruption", e),
|
||||
Priority.IMMEDIATE);
|
||||
try {
|
||||
started.await();
|
||||
} catch (InterruptedException e) {
|
||||
|
|
|
@ -18,8 +18,7 @@
|
|||
*/
|
||||
package org.elasticsearch.test.disruption;
|
||||
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.LocalClusterUpdateTask;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
@ -102,27 +101,23 @@ public class SlowClusterStateProcessing extends SingleNodeDisruption {
|
|||
return false;
|
||||
}
|
||||
final AtomicBoolean stopped = new AtomicBoolean(false);
|
||||
clusterService.submitStateUpdateTask("service_disruption_delay", new LocalClusterUpdateTask(Priority.IMMEDIATE) {
|
||||
|
||||
@Override
|
||||
public ClusterTasksResult<LocalClusterUpdateTask> execute(ClusterState currentState) throws Exception {
|
||||
long count = duration.millis() / 200;
|
||||
// wait while checking for a stopped
|
||||
for (; count > 0 && !stopped.get(); count--) {
|
||||
Thread.sleep(200);
|
||||
clusterService.getClusterApplierService().runOnApplierThread("service_disruption_delay",
|
||||
currentState -> {
|
||||
try {
|
||||
long count = duration.millis() / 200;
|
||||
// wait while checking for a stopped
|
||||
for (; count > 0 && !stopped.get(); count--) {
|
||||
Thread.sleep(200);
|
||||
}
|
||||
if (!stopped.get()) {
|
||||
Thread.sleep(duration.millis() % 200);
|
||||
}
|
||||
countDownLatch.countDown();
|
||||
} catch (InterruptedException e) {
|
||||
ExceptionsHelper.reThrowIfNotNull(e);
|
||||
}
|
||||
if (!stopped.get()) {
|
||||
Thread.sleep(duration.millis() % 200);
|
||||
}
|
||||
countDownLatch.countDown();
|
||||
return unchanged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
countDownLatch.countDown();
|
||||
}
|
||||
});
|
||||
}, (source, e) -> countDownLatch.countDown(),
|
||||
Priority.IMMEDIATE);
|
||||
try {
|
||||
countDownLatch.await();
|
||||
} catch (InterruptedException e) {
|
||||
|
|
Loading…
Reference in New Issue