From e4bb360ae0fae753747eebec9ef88cf2530334e0 Mon Sep 17 00:00:00 2001 From: Toby McLaughlin Date: Fri, 28 Apr 2017 14:48:52 +1000 Subject: [PATCH 01/10] Fix typo in node environment exception message This commit fixes a typo in an exception message when trying to create a node environment. Relates #24381 --- core/src/main/java/org/elasticsearch/node/Node.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 01ccf68dc36..764ece5b868 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -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); From 35f78d098a45cee6b26998a8d0f11343ca67dada Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 28 Apr 2017 09:34:31 +0200 Subject: [PATCH 02/10] Separate publishing from applying cluster states (#24236) Separates cluster state publishing from applying cluster states: - ClusterService is split into two classes MasterService and ClusterApplierService. MasterService has the responsibility to calculate cluster state updates for actions that want to change the cluster state (create index, update shard routing table, etc.). ClusterApplierService has the responsibility to apply cluster states that have been successfully published and invokes the cluster state appliers and listeners. - ClusterApplierService keeps track of the last applied state, but MasterService is stateless and uses the last cluster state that is provided by the discovery module to calculate the next prospective state. The ClusterService class is still kept around, which now just delegates actions to ClusterApplierService and MasterService. - The discovery implementation is now responsible for managing the last cluster state that is used by the consensus layer and the master service. It also exposes the initial cluster state which is used by the ClusterApplierService. The discovery implementation is also responsible for adding the right cluster-level blocks to the initial state. - NoneDiscovery has been renamed to TribeDiscovery as it is exclusively used by TribeService. It adds the tribe blocks to the initial state. - ZenDiscovery is synchronized on state changes to the last cluster state that is used by the consensus layer and the master service, and does not submit cluster state update tasks anymore to make changes to the disco state (except when becoming master). Control flow for cluster state updates is now as follows: - State updates are sent to MasterService - MasterService gets the latest committed cluster state from the discovery implementation and calculates the next cluster state to publish - MasterService submits the new prospective cluster state to the discovery implementation for publishing - Discovery implementation publishes cluster states to all nodes and, once the state is committed, asks the ClusterApplierService to apply the newly committed state. - ClusterApplierService applies state to local node. --- .../resources/checkstyle_suppressions.xml | 2 +- .../health/TransportClusterHealthAction.java | 8 +- .../TransportPendingClusterTasksAction.java | 2 +- .../elasticsearch/cluster/ClusterState.java | 1 - .../cluster/ClusterStateObserver.java | 30 +- .../cluster/ClusterStateTaskExecutor.java | 9 +- .../cluster/ClusterStateTaskListener.java | 5 +- .../cluster/LocalClusterUpdateTask.java | 16 +- .../routing/DelayedAllocationService.java | 8 +- .../cluster/service/ClusterApplier.java | 37 + .../service/ClusterApplierService.java | 655 ++++++++++ .../cluster/service/ClusterService.java | 1070 +---------------- .../cluster/service/MasterService.java | 752 ++++++++++++ .../org/elasticsearch/common/Randomness.java | 2 +- .../common/util/concurrent/BaseFuture.java | 9 +- .../elasticsearch/discovery/Discovery.java | 21 +- .../discovery/DiscoveryModule.java | 17 +- .../discovery/NoneDiscovery.java | 102 -- .../discovery/TribeDiscovery.java | 73 ++ .../discovery/single/SingleNodeDiscovery.java | 139 ++- .../discovery/zen/MasterFaultDetection.java | 20 +- .../discovery/zen/NodeJoinController.java | 26 +- .../discovery/zen/ZenDiscovery.java | 615 +++++----- .../elasticsearch/gateway/GatewayService.java | 3 - .../indices/store/IndicesStore.java | 19 +- .../java/org/elasticsearch/node/Node.java | 26 +- .../org/elasticsearch/node/NodeService.java | 4 +- .../plugins/DiscoveryPlugin.java | 14 +- .../repositories/RepositoriesService.java | 7 +- .../org/elasticsearch/tribe/TribeService.java | 26 +- .../cluster/ClusterModuleTests.java | 6 +- .../health/ClusterStateHealthTests.java | 18 +- .../DelayedAllocationServiceTests.java | 4 +- .../service/ClusterApplierServiceTests.java | 427 +++++++ .../cluster/service/ClusterServiceIT.java | 6 +- ...viceTests.java => MasterServiceTests.java} | 600 +++------ .../cluster/settings/ClusterSettingsIT.java | 3 +- .../discovery/DiscoveryModuleTests.java | 42 +- .../DiscoveryWithServiceDisruptionsIT.java | 43 +- .../discovery/ZenFaultDetectionTests.java | 22 +- .../single/SingleNodeDiscoveryTests.java | 38 +- .../zen/NodeJoinControllerTests.java | 102 +- .../discovery/zen/ZenDiscoveryIT.java | 1 + .../discovery/zen/ZenDiscoveryUnitTests.java | 66 +- .../gateway/GatewayServiceTests.java | 5 +- .../IndexLifecycleActionIT.java | 5 +- .../store/IndicesStoreIntegrationIT.java | 39 +- .../AbstractSnapshotIntegTestCase.java | 130 -- .../DedicatedClusterSnapshotRestoreIT.java | 43 +- .../SharedClusterSnapshotRestoreIT.java | 53 - .../org/elasticsearch/test/NoopDiscovery.java | 23 +- .../azure/classic/AzureDiscoveryPlugin.java | 25 +- .../discovery/ec2/Ec2DiscoveryPlugin.java | 52 +- .../discovery/gce/GceDiscoveryPlugin.java | 19 +- .../test/ClusterServiceUtils.java | 190 ++- .../elasticsearch/test/ESIntegTestCase.java | 3 +- .../test/discovery/TestZenDiscovery.java | 17 +- .../BlockClusterStateProcessing.java | 25 +- .../SlowClusterStateProcessing.java | 39 +- 59 files changed, 3174 insertions(+), 2590 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java create mode 100644 core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java create mode 100644 core/src/main/java/org/elasticsearch/cluster/service/MasterService.java delete mode 100644 core/src/main/java/org/elasticsearch/discovery/NoneDiscovery.java create mode 100644 core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java create mode 100644 core/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java rename core/src/test/java/org/elasticsearch/cluster/service/{ClusterServiceTests.java => MasterServiceTests.java} (59%) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 6e62b8ec346..2ed1b68fb8a 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -1047,7 +1047,7 @@ - + diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java index 44c604dc8b8..8924f81a86c 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java @@ -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 diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java index c15758de3cb..cd58bb8d6d4 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java @@ -65,7 +65,7 @@ public class TransportPendingClusterTasksAction extends TransportMasterNodeReadA @Override protected void masterOperation(PendingClusterTasksRequest request, ClusterState state, ActionListener listener) { logger.trace("fetching pending tasks from cluster service"); - final List pendingTasks = clusterService.pendingTasks(); + final List pendingTasks = clusterService.getMasterService().pendingTasks(); logger.trace("done fetching pending tasks from cluster service"); listener.onResponse(new PendingClusterTasksResponse(pendingTasks)); } diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java index 5ef06f178a8..89b3727c9f4 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterStateObserver.java b/core/src/main/java/org/elasticsearch/cluster/ClusterStateObserver.java index e0c35a12c22..d191bc01756 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterStateObserver.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterStateObserver.java @@ -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 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); } diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterStateTaskExecutor.java b/core/src/main/java/org/elasticsearch/cluster/ClusterStateTaskExecutor.java index 3693447cfb6..8e50fddb9b1 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterStateTaskExecutor.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterStateTaskExecutor.java @@ -71,21 +71,18 @@ public interface ClusterStateTaskExecutor { * @param the type of the cluster state update task */ class ClusterTasksResult { - public final boolean noMaster; @Nullable public final ClusterState resultingState; public final Map 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 executionResults) { + ClusterTasksResult(ClusterState resultingState, Map executionResults) { this.resultingState = resultingState; this.executionResults = executionResults; - this.noMaster = noMaster; } public static Builder builder() { @@ -124,11 +121,11 @@ public interface ClusterStateTaskExecutor { } public ClusterTasksResult build(ClusterState resultingState) { - return new ClusterTasksResult<>(false, resultingState, executionResults); + return new ClusterTasksResult<>(resultingState, executionResults); } ClusterTasksResult build(ClusterTasksResult result, ClusterState previousState) { - return new ClusterTasksResult<>(result.noMaster, result.resultingState == null ? previousState : result.resultingState, + return new ClusterTasksResult<>(result.resultingState == null ? previousState : result.resultingState, executionResults); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterStateTaskListener.java b/core/src/main/java/org/elasticsearch/cluster/ClusterStateTaskListener.java index 757c8b0c82e..b2ab13fac2a 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterStateTaskListener.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterStateTaskListener.java @@ -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 + "]")); diff --git a/core/src/main/java/org/elasticsearch/cluster/LocalClusterUpdateTask.java b/core/src/main/java/org/elasticsearch/cluster/LocalClusterUpdateTask.java index 9692ff8d4e1..89d85fa2e14 100644 --- a/core/src/main/java/org/elasticsearch/cluster/LocalClusterUpdateTask.java +++ b/core/src/main/java/org/elasticsearch/cluster/LocalClusterUpdateTask.java @@ -50,25 +50,11 @@ public abstract class LocalClusterUpdateTask implements ClusterStateTaskConfig, return ClusterTasksResult.builder().successes(tasks).build(result, currentState); } - /** - * node stepped down as master or has lost connection to the master - */ - public static ClusterTasksResult 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 unchanged() { - return new ClusterTasksResult(false, null, null); - } - - /** - * locally apply cluster state received from a master - */ - public static ClusterTasksResult newState(ClusterState clusterState) { - return new ClusterTasksResult(false, clusterState, null); + return new ClusterTasksResult<>(null, null); } @Override diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java b/core/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java index 4522dfcf98f..fd7f8f6811f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java @@ -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(); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java new file mode 100644 index 00000000000..ef3135af24d --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java @@ -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 clusterStateSupplier, ClusterStateTaskListener listener); +} diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java new file mode 100644 index 00000000000..540881718fc --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java @@ -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 highPriorityStateAppliers = new CopyOnWriteArrayList<>(); + private final Collection normalPriorityStateAppliers = new CopyOnWriteArrayList<>(); + private final Collection lowPriorityStateAppliers = new CopyOnWriteArrayList<>(); + private final Iterable clusterStateAppliers = Iterables.concat(highPriorityStateAppliers, + normalPriorityStateAppliers, lowPriorityStateAppliers); + + private final Collection clusterStateListeners = new CopyOnWriteArrayList<>(); + private final Collection timeoutClusterStateListeners = + Collections.newSetFromMap(new ConcurrentHashMap()); + + private final LocalNodeMasterListeners localNodeMasterListeners; + + private final Queue onGoingTimeouts = ConcurrentCollections.newQueue(); + + private final AtomicReference 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 { + final ClusterStateTaskListener listener; + final Function updateFunction; + + UpdateTask(Priority priority, String source, ClusterStateTaskListener listener, + Function 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 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 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 clusterStateConsumer, + final ClusterStateTaskListener listener) { + runOnApplierThread(source, clusterStateConsumer, listener, Priority.HIGH); + } + + @Override + public void onNewClusterState(final String source, final java.util.function.Supplier clusterStateSupplier, + final ClusterStateTaskListener listener) { + Function 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 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 NOT 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 NOT 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 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(); + } +} diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java index dd7d7bcbd60..18445e62c7e 100644 --- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -19,264 +19,82 @@ 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.ClusterName; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterState.Builder; import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.ClusterStateObserver; 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.LocalNodeMasterListener; import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.TimeoutClusterStateListener; -import org.elasticsearch.cluster.block.ClusterBlock; -import org.elasticsearch.cluster.block.ClusterBlocks; -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.OperationRouting; -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.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.text.Text; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -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.common.util.iterable.Iterables; -import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.threadpool.ThreadPool; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Locale; import java.util.Map; -import java.util.Objects; -import java.util.Queue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; -import java.util.concurrent.Future; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; -import java.util.function.UnaryOperator; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory; public class ClusterService extends AbstractLifecycleComponent { + private final MasterService masterService; + + private final ClusterApplierService clusterApplierService; + public static final Setting CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING = Setting.positiveTimeSetting("cluster.service.slow_task_logging_threshold", TimeValue.timeValueSeconds(30), Property.Dynamic, Property.NodeScope); - public static final String UPDATE_THREAD_NAME = "clusterService#updateTask"; - private final ThreadPool threadPool; private final ClusterName clusterName; - private final Supplier localNodeSupplier; - - private BiConsumer clusterStatePublisher; private final OperationRouting operationRouting; private final ClusterSettings clusterSettings; - private TimeValue slowTaskLoggingThreshold; - - private volatile PrioritizedEsThreadPoolExecutor threadPoolExecutor; - private volatile ClusterServiceTaskBatcher taskBatcher; - - /** - * Those 3 state listeners are changing infrequently - CopyOnWriteArrayList is just fine - */ - private final Collection highPriorityStateAppliers = new CopyOnWriteArrayList<>(); - private final Collection normalPriorityStateAppliers = new CopyOnWriteArrayList<>(); - private final Collection lowPriorityStateAppliers = new CopyOnWriteArrayList<>(); - private final Iterable clusterStateAppliers = Iterables.concat(highPriorityStateAppliers, - normalPriorityStateAppliers, lowPriorityStateAppliers); - - private final Collection clusterStateListeners = new CopyOnWriteArrayList<>(); - private final Collection timeoutClusterStateListeners = - Collections.newSetFromMap(new ConcurrentHashMap()); - - private final LocalNodeMasterListeners localNodeMasterListeners; - - private final Queue onGoingTimeouts = ConcurrentCollections.newQueue(); - - private final AtomicReference state; - - private final ClusterBlocks.Builder initialBlocks; - - private NodeConnectionsService nodeConnectionsService; - - private DiscoverySettings discoverySettings; - - public ClusterService(Settings settings, - ClusterSettings clusterSettings, ThreadPool threadPool, Supplier localNodeSupplier) { + public ClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) { super(settings); - this.localNodeSupplier = localNodeSupplier; + this.clusterApplierService = new ClusterApplierService(settings, clusterSettings, threadPool); + this.masterService = new MasterService(settings, threadPool); this.operationRouting = new OperationRouting(settings, clusterSettings); - this.threadPool = threadPool; this.clusterSettings = clusterSettings; this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); - // will be replaced on doStart. - this.state = new AtomicReference<>(ClusterState.builder(clusterName).build()); - this.clusterSettings.addSettingsUpdateConsumer(CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, - this::setSlowTaskLoggingThreshold); - - this.slowTaskLoggingThreshold = CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings); - - localNodeMasterListeners = new LocalNodeMasterListeners(threadPool); - - initialBlocks = ClusterBlocks.builder(); + this::setSlowTaskLoggingThreshold); } private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) { - this.slowTaskLoggingThreshold = slowTaskLoggingThreshold; - } - - public synchronized void setClusterStatePublisher(BiConsumer publisher) { - clusterStatePublisher = publisher; - } - - private void updateState(UnaryOperator updateFunction) { - this.state.getAndUpdate(updateFunction); + masterService.setSlowTaskLoggingThreshold(slowTaskLoggingThreshold); + clusterApplierService.setSlowTaskLoggingThreshold(slowTaskLoggingThreshold); } public synchronized void setNodeConnectionsService(NodeConnectionsService nodeConnectionsService) { - assert this.nodeConnectionsService == null : "nodeConnectionsService is already set"; - this.nodeConnectionsService = nodeConnectionsService; - } - - /** - * Adds an initial block to be set on the first cluster state created. - */ - public synchronized void addInitialStateBlock(ClusterBlock block) throws IllegalStateException { - if (lifecycle.started()) { - throw new IllegalStateException("can't set initial block when started"); - } - initialBlocks.addGlobalBlock(block); - } - - /** - * Remove an initial block to be set on the first cluster state created. - */ - public synchronized void removeInitialStateBlock(ClusterBlock block) throws IllegalStateException { - removeInitialStateBlock(block.id()); - } - - /** - * Remove an initial block to be set on the first cluster state created. - */ - public synchronized void removeInitialStateBlock(int blockId) throws IllegalStateException { - if (lifecycle.started()) { - throw new IllegalStateException("can't set initial block when started"); - } - initialBlocks.removeGlobalBlock(blockId); + clusterApplierService.setNodeConnectionsService(nodeConnectionsService); } @Override protected synchronized void doStart() { - Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting"); - Objects.requireNonNull(nodeConnectionsService, "please set the node connection service before starting"); - Objects.requireNonNull(discoverySettings, "please set discovery settings before starting"); - addListener(localNodeMasterListeners); - DiscoveryNode localNode = localNodeSupplier.get(); - assert localNode != null; - updateState(state -> { - assert state.nodes().getLocalNodeId() == null : "local node is already set"; - DiscoveryNodes nodes = DiscoveryNodes.builder(state.nodes()).add(localNode).localNodeId(localNode.getId()).build(); - return ClusterState.builder(state).nodes(nodes).blocks(initialBlocks).build(); - }); - this.threadPoolExecutor = EsExecutors.newSinglePrioritizing(UPDATE_THREAD_NAME, - daemonThreadFactory(settings, UPDATE_THREAD_NAME), threadPool.getThreadContext(), threadPool.scheduler()); - this.taskBatcher = new ClusterServiceTaskBatcher(logger, threadPoolExecutor); + clusterApplierService.start(); + masterService.start(); } @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); + masterService.stop(); + clusterApplierService.stop(); } @Override protected synchronized void doClose() { - } - - class ClusterServiceTaskBatcher extends TaskBatcher { - - ClusterServiceTaskBatcher(Logger logger, PrioritizedEsThreadPoolExecutor threadExecutor) { - super(logger, threadExecutor); - } - - @Override - protected void onTimeout(List 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 tasks, String tasksSummary) { - ClusterStateTaskExecutor taskExecutor = (ClusterStateTaskExecutor) batchingKey; - List updateTasks = (List) tasks; - runTasks(new ClusterService.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 tasks) { - return ((ClusterStateTaskExecutor) batchingKey).describeTasks( - tasks.stream().map(BatchedTask::getTask).collect(Collectors.toList())); - } - } + masterService.close(); + clusterApplierService.close(); } /** @@ -295,83 +113,74 @@ public class ClusterService extends AbstractLifecycleComponent { } /** - * The current cluster state. + * The currently applied cluster state. + * TODO: Should be renamed to appliedState / appliedClusterState */ public ClusterState state() { - assert assertNotCalledFromClusterStateApplier("the applied cluster state is not yet available"); - return this.state.get(); + return clusterApplierService.state(); } /** * Adds a high priority applier of updated cluster states. */ public void addHighPriorityApplier(ClusterStateApplier applier) { - highPriorityStateAppliers.add(applier); + clusterApplierService.addHighPriorityApplier(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); + clusterApplierService.addLowPriorityApplier(applier); } /** * Adds a applier of updated cluster states. */ public void addStateApplier(ClusterStateApplier applier) { - normalPriorityStateAppliers.add(applier); + clusterApplierService.addStateApplier(applier); } /** * Removes an applier of updated cluster states. */ public void removeApplier(ClusterStateApplier applier) { - normalPriorityStateAppliers.remove(applier); - highPriorityStateAppliers.remove(applier); - lowPriorityStateAppliers.remove(applier); + clusterApplierService.removeApplier(applier); } /** * Add a listener for updated cluster states */ public void addListener(ClusterStateListener listener) { - clusterStateListeners.add(listener); + clusterApplierService.addListener(listener); } /** * Removes a listener for updated cluster states. */ public void removeListener(ClusterStateListener listener) { - clusterStateListeners.remove(listener); + clusterApplierService.removeListener(listener); } /** * Removes a timeout listener for updated cluster states. */ public void removeTimeoutListener(TimeoutClusterStateListener listener) { - timeoutClusterStateListeners.remove(listener); - for (Iterator it = onGoingTimeouts.iterator(); it.hasNext(); ) { - NotifyTimeout timeout = it.next(); - if (timeout.listener.equals(listener)) { - timeout.cancel(); - it.remove(); - } - } + clusterApplierService.removeTimeoutListener(listener); } /** * Add a listener for on/off local node master events */ public void addLocalNodeMasterListener(LocalNodeMasterListener listener) { - localNodeMasterListeners.add(listener); + clusterApplierService.addLocalNodeMasterListener(listener); } /** * Remove the given listener for on/off local master events */ public void removeLocalNodeMasterListener(LocalNodeMasterListener listener) { - localNodeMasterListeners.remove(listener); + clusterApplierService.removeLocalNodeMasterListener(listener); } /** @@ -381,32 +190,34 @@ public class ClusterService extends AbstractLifecycleComponent { * 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; - } + clusterApplierService.addTimeoutListener(timeout, listener); + } - // 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 MasterService getMasterService() { + return masterService; + } + + public ClusterApplierService getClusterApplierService() { + return clusterApplierService; + } + + public static boolean assertClusterOrMasterStateThread() { + assert Thread.currentThread().getName().contains(ClusterApplierService.CLUSTER_UPDATE_THREAD_NAME) || + Thread.currentThread().getName().contains(MasterService.MASTER_UPDATE_THREAD_NAME) : + "not called from the master/cluster state update thread"; + return true; + } + + public ClusterName getClusterName() { + return clusterName; + } + + public ClusterSettings getClusterSettings() { + return clusterSettings; + } + + public Settings getSettings() { + return settings; } /** @@ -418,8 +229,8 @@ public class ClusterService extends AbstractLifecycleComponent { * task * */ - public & ClusterStateTaskListener> void submitStateUpdateTask( - final String source, final T updateTask) { + public & ClusterStateTaskListener> + void submitStateUpdateTask(String source, T updateTask) { submitStateUpdateTask(source, updateTask, updateTask, updateTask, updateTask); } @@ -442,10 +253,10 @@ public class ClusterService extends AbstractLifecycleComponent { * @param the type of the cluster state update task state * */ - public void submitStateUpdateTask(final String source, final T task, - final ClusterStateTaskConfig config, - final ClusterStateTaskExecutor executor, - final ClusterStateTaskListener listener) { + public void submitStateUpdateTask(String source, T task, + ClusterStateTaskConfig config, + ClusterStateTaskExecutor executor, + ClusterStateTaskListener listener) { submitStateUpdateTasks(source, Collections.singletonMap(task, listener), config, executor); } @@ -465,761 +276,6 @@ public class ClusterService extends AbstractLifecycleComponent { public void submitStateUpdateTasks(final String source, final Map tasks, final ClusterStateTaskConfig config, final ClusterStateTaskExecutor executor) { - if (!lifecycle.started()) { - return; - } - try { - List safeTasks = tasks.entrySet().stream() - .map(e -> taskBatcher.new UpdateTask(config.priority(), source, e.getKey(), safe(e.getValue(), logger), 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; - } - } - } - - /** - * Returns the tasks that are pending. - */ - public List 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(); - } - - /** asserts that the current thread is the cluster state update thread */ - public static boolean assertClusterStateThread() { - assert Thread.currentThread().getName().contains(ClusterService.UPDATE_THREAD_NAME) : - "not called from the cluster state update thread"; - return true; - } - - /** asserts that the current thread is NOT the cluster state update thread */ - public static boolean assertNotClusterStateUpdateThread(String reason) { - assert Thread.currentThread().getName().contains(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 NOT involve a cluster state applier */ - private static boolean assertNotCalledFromClusterStateApplier(String reason) { - if (Thread.currentThread().getName().contains(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(ClusterService.class.getName()) - && methodName.equals("callClusterStateAppliers")) { - throw new AssertionError("should not be called by a cluster state applier. reason [" + reason + "]"); - } - } - } - return true; - } - - public ClusterName getClusterName() { - return clusterName; - } - - public void setDiscoverySettings(DiscoverySettings discoverySettings) { - this.discoverySettings = discoverySettings; - } - - void runTasks(TaskInputs taskInputs) { - if (!lifecycle.started()) { - logger.debug("processing [{}]: ignoring, cluster service not started", taskInputs.summary); - return; - } - - logger.debug("processing [{}]: execute", taskInputs.summary); - ClusterState previousClusterState = state(); - - if (!previousClusterState.nodes().isLocalNodeElectedMaster() && taskInputs.runOnlyOnMaster()) { - logger.debug("failing [{}]: local node is no longer master", taskInputs.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", taskInputs.summary, executionTime); - warnAboutSlowTaskIfNeeded(executionTime, taskInputs.summary); - } else { - ClusterState newClusterState = taskOutputs.newClusterState; - if (logger.isTraceEnabled()) { - logger.trace("cluster state updated, source [{}]\n{}", taskInputs.summary, newClusterState); - } else if (logger.isDebugEnabled()) { - logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), taskInputs.summary); - } - try { - publishAndApplyChanges(taskInputs, taskOutputs); - TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS))); - logger.debug("processing [{}]: took [{}] done applying updated cluster_state (version: {}, uuid: {})", taskInputs.summary, - executionTime, newClusterState.version(), newClusterState.stateUUID()); - warnAboutSlowTaskIfNeeded(executionTime, taskInputs.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 apply updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}", - executionTime, - version, - stateUUID, - taskInputs.summary, - fullState), - e); - // TODO: do we want to call updateTask.onFailure here? - } - } - } - - public TaskOutputs calculateTaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState, long startTimeNS) { - ClusterTasksResult clusterTasksResult = executeTasks(taskInputs, startTimeNS, previousClusterState); - // extract those that are waiting for results - List nonFailedTasks = new ArrayList<>(); - for (ClusterServiceTaskBatcher.UpdateTask updateTask : taskInputs.updateTasks) { - assert clusterTasksResult.executionResults.containsKey(updateTask.task) : "missing " + updateTask; - final ClusterStateTaskExecutor.TaskResult taskResult = - clusterTasksResult.executionResults.get(updateTask.task); - if (taskResult.isSuccess()) { - nonFailedTasks.add(updateTask); - } - } - ClusterState newClusterState = patchVersionsAndNoMasterBlocks(previousClusterState, clusterTasksResult); - - return new TaskOutputs(taskInputs, previousClusterState, newClusterState, nonFailedTasks, - clusterTasksResult.executionResults); - } - - private ClusterTasksResult executeTasks(TaskInputs taskInputs, long startTimeNS, ClusterState previousClusterState) { - ClusterTasksResult clusterTasksResult; - try { - List inputs = taskInputs.updateTasks.stream() - .map(ClusterServiceTaskBatcher.UpdateTask::getTask).collect(Collectors.toList()); - clusterTasksResult = taskInputs.executor.execute(previousClusterState, inputs); - } 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(ClusterServiceTaskBatcher.UpdateTask::getTask)::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) { - for (ClusterServiceTaskBatcher.UpdateTask updateTask : taskInputs.updateTasks) { - assert clusterTasksResult.executionResults.containsKey(updateTask.task) : - "missing task result for " + updateTask; - } - } - - return clusterTasksResult; - } - - private ClusterState patchVersionsAndNoMasterBlocks(ClusterState previousClusterState, ClusterTasksResult executionResult) { - ClusterState newClusterState = executionResult.resultingState; - - if (executionResult.noMaster) { - assert newClusterState == previousClusterState : "state can only be changed by ClusterService when noMaster = true"; - if (previousClusterState.nodes().getMasterNodeId() != null) { - // remove block if it already exists before adding new one - assert previousClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock().id()) == false : - "NO_MASTER_BLOCK should only be added by ClusterService"; - ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(previousClusterState.blocks()) - .addGlobalBlock(discoverySettings.getNoMasterBlock()) - .build(); - - DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder(previousClusterState.nodes()).masterNodeId(null).build(); - newClusterState = ClusterState.builder(previousClusterState) - .blocks(clusterBlocks) - .nodes(discoveryNodes) - .build(); - } - } else if (newClusterState.nodes().isLocalNodeElectedMaster() && 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)); - } - - // remove the no master block, if it exists - if (newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock().id())) { - builder.blocks(ClusterBlocks.builder().blocks(newClusterState.blocks()) - .removeGlobalBlock(discoverySettings.getNoMasterBlock().id())); - } - - newClusterState = builder.build(); - } - - assert newClusterState.nodes().getMasterNodeId() == null || - newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock().id()) == false : - "cluster state with master node must not have NO_MASTER_BLOCK"; - - return newClusterState; - } - - private void publishAndApplyChanges(TaskInputs taskInputs, TaskOutputs taskOutputs) { - ClusterState previousClusterState = taskOutputs.previousClusterState; - ClusterState newClusterState = taskOutputs.newClusterState; - - ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent(taskInputs.summary, 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, taskInputs.summary); - } - } - - final Discovery.AckListener ackListener = newClusterState.nodes().isLocalNodeElectedMaster() ? - taskOutputs.createAckListener(threadPool, newClusterState) : - null; - - nodeConnectionsService.connectToNodes(newClusterState.nodes()); - - // if we are the master, publish the new state to all nodes - // we publish here before we send a notification to all the listeners, since if it fails - // we don't want to notify - if (newClusterState.nodes().isLocalNodeElectedMaster()) { - logger.debug("publishing cluster state version [{}]", newClusterState.version()); - try { - clusterStatePublisher.accept(clusterChangedEvent, ackListener); - } catch (Discovery.FailedToCommitClusterStateException t) { - final long version = newClusterState.version(); - logger.warn( - (Supplier) () -> new ParameterizedMessage( - "failing [{}]: failed to commit cluster state version [{}]", taskInputs.summary, version), - t); - // ensure that list of connected nodes in NodeConnectionsService is in-sync with the nodes of the current cluster state - nodeConnectionsService.connectToNodes(previousClusterState.nodes()); - nodeConnectionsService.disconnectFromNodesExcept(previousClusterState.nodes()); - taskOutputs.publishingFailed(t); - return; - } - } - - 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("set local cluster state to version {}", newClusterState.version()); - callClusterStateAppliers(newClusterState, clusterChangedEvent); - - nodeConnectionsService.disconnectFromNodesExcept(newClusterState.nodes()); - - updateState(css -> newClusterState); - - Stream.concat(clusterStateListeners.stream(), timeoutClusterStateListeners.stream()).forEach(listener -> { - try { - logger.trace("calling [{}] with change to version [{}]", listener, newClusterState.version()); - listener.clusterChanged(clusterChangedEvent); - } catch (Exception ex) { - logger.warn("failed to notify ClusterStateListener", ex); - } - }); - - //manual ack only from the master at the end of the publish - if (newClusterState.nodes().isLocalNodeElectedMaster()) { - try { - ackListener.onNodeAck(newClusterState.nodes().getLocalNode(), null); - } catch (Exception e) { - final DiscoveryNode localNode = newClusterState.nodes().getLocalNode(); - logger.debug( - (Supplier) () -> new ParameterizedMessage("error while processing ack for master node [{}]", localNode), - e); - } - } - - taskOutputs.processedDifferentClusterState(previousClusterState, newClusterState); - - if (newClusterState.nodes().isLocalNodeElectedMaster()) { - try { - taskOutputs.clusterStatePublished(clusterChangedEvent); - } catch (Exception e) { - logger.error( - (Supplier) () -> new ParameterizedMessage( - "exception thrown while notifying executor of new cluster state publication [{}]", - taskInputs.summary), - e); - } - } - } - - private void callClusterStateAppliers(ClusterState newClusterState, ClusterChangedEvent clusterChangedEvent) { - for (ClusterStateApplier applier : clusterStateAppliers) { - try { - logger.trace("calling [{}] with change to version [{}]", applier, newClusterState.version()); - applier.applyClusterState(clusterChangedEvent); - } catch (Exception ex) { - logger.warn("failed to notify ClusterStateApplier", ex); - } - } - } - - /** - * Represents a set of tasks to be processed together with their executor - */ - class TaskInputs { - public final String summary; - public final List updateTasks; - public final ClusterStateTaskExecutor executor; - - TaskInputs(ClusterStateTaskExecutor executor, List updateTasks, String summary) { - this.summary = summary; - this.executor = executor; - this.updateTasks = updateTasks; - } - - public boolean runOnlyOnMaster() { - return executor.runOnlyOnMaster(); - } - - public void onNoLongerMaster() { - updateTasks.stream().forEach(task -> task.listener.onNoLongerMaster(task.source)); - } - } - - /** - * 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 nonFailedTasks; - public final Map executionResults; - - TaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState, - ClusterState newClusterState, List nonFailedTasks, - Map 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 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 (ClusterServiceTaskBatcher.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); - }); - } - } - - // this one is overridden in tests so we can control time - protected long currentTimeInNanos() { - return System.nanoTime(); - } - - private static SafeClusterStateTaskListener safe(ClusterStateTaskListener listener, Logger logger) { - 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(); - } - } - - private 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); - } - } - - 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 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) { - Executor executor = threadPool.executor(listener.executorName()); - executor.execute(new OnMasterRunnable(listener)); - } - return; - } - - if (master && !event.localNodeMaster()) { - master = false; - for (LocalNodeMasterListener listener : listeners) { - 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(); - } - } - - private static class DelegetingAckListener implements Discovery.AckListener { - - private final List listeners; - - private DelegetingAckListener(List 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(); - } - } - } - - public ClusterSettings getClusterSettings() { - return clusterSettings; - } - - public Settings getSettings() { - return settings; + masterService.submitStateUpdateTasks(source, tasks, config, executor); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/service/MasterService.java b/core/src/main/java/org/elasticsearch/cluster/service/MasterService.java new file mode 100644 index 00000000000..46d9eaef7ab --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/service/MasterService.java @@ -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 clusterStatePublisher; + + private java.util.function.Supplier 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 publisher) { + clusterStatePublisher = publisher; + } + + public synchronized void setClusterStateSupplier(java.util.function.Supplier 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 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 tasks, String tasksSummary) { + ClusterStateTaskExecutor taskExecutor = (ClusterStateTaskExecutor) batchingKey; + List updateTasks = (List) 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 tasks) { + return ((ClusterStateTaskExecutor) 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 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 & 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 the type of the cluster state update task state + * + */ + public void submitStateUpdateTask(String source, T task, + ClusterStateTaskConfig config, + ClusterStateTaskExecutor 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 nonFailedTasks; + public final Map executionResults; + + TaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState, + ClusterState newClusterState, + List nonFailedTasks, + Map 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 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 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 listeners; + + private DelegetingAckListener(List 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 executeTasks(TaskInputs taskInputs, long startTimeNS, ClusterState previousClusterState) { + ClusterTasksResult clusterTasksResult; + try { + List 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 finalClusterTasksResult = clusterTasksResult; + taskInputs.updateTasks.forEach(updateTask -> { + assert finalClusterTasksResult.executionResults.containsKey(updateTask.task) : + "missing task result for " + updateTask; + }); + } + + return clusterTasksResult; + } + + public List getNonFailedTasks(TaskInputs taskInputs, + ClusterTasksResult 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 updateTasks; + public final ClusterStateTaskExecutor executor; + + TaskInputs(ClusterStateTaskExecutor executor, List 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 the type of the cluster state update task state + * + */ + public void submitStateUpdateTasks(final String source, + final Map tasks, final ClusterStateTaskConfig config, + final ClusterStateTaskExecutor executor) { + if (!lifecycle.started()) { + return; + } + try { + List 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(); + } +} diff --git a/core/src/main/java/org/elasticsearch/common/Randomness.java b/core/src/main/java/org/elasticsearch/common/Randomness.java index 295d7a6bcda..05ebe1a7377 100644 --- a/core/src/main/java/org/elasticsearch/common/Randomness.java +++ b/core/src/main/java/org/elasticsearch/common/Randomness.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/BaseFuture.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/BaseFuture.java index ee9aea9ed70..3436ccdf7ad 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/BaseFuture.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/BaseFuture.java @@ -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 implements Future { 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 implements Future { 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(); } diff --git a/core/src/main/java/org/elasticsearch/discovery/Discovery.java b/core/src/main/java/org/elasticsearch/discovery/Discovery.java index e4addfdd02f..4c28b51bc4f 100644 --- a/core/src/main/java/org/elasticsearch/discovery/Discovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/Discovery.java @@ -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 */ diff --git a/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index 2328b5a8616..43fccab1a28 100644 --- a/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -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 plugins) { + NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService, MasterService masterService, + ClusterApplier clusterApplier, ClusterSettings clusterSettings, List plugins) { final UnicastHostsProvider hostsProvider; Map> hostProviders = new HashMap<>(); @@ -80,12 +82,13 @@ public class DiscoveryModule { Map> 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"); } diff --git a/core/src/main/java/org/elasticsearch/discovery/NoneDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/NoneDiscovery.java deleted file mode 100644 index 91b04ce396b..00000000000 --- a/core/src/main/java/org/elasticsearch/discovery/NoneDiscovery.java +++ /dev/null @@ -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() { - - } -} diff --git a/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java new file mode 100644 index 00000000000..751eb94a9f2 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java @@ -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 + } +} diff --git a/core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java index f4735c8bf3a..d7275e7be91 100644 --- a/core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java @@ -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 executor = - new ClusterStateTaskExecutor() { - - @Override - public ClusterTasksResult execute( - final ClusterState current, - final List 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 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 diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java b/core/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java index 81d4c19d33e..fff5e7cb5c9 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java @@ -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 clusterStateSupplier; private final CopyOnWriteArrayList 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 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 { diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 354425a3dca..566886ecdb6 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -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 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")); } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index f5dbaac6242..5da446aa5b9 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -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 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 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,117 @@ 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); + publishClusterState.pendingStatesQueue().markAsCommitted(newState.stateUUID(), + new PendingClusterStatesQueue.StateProcessedListener() { + @Override + public void onNewClusterStateProcessed() { + 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) { + 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 processed = processNextCommittedClusterState("master " + newState.nodes().getMasterNode() + + " committed version [" + newState.version() + "] source [" + clusterChangedEvent.source() + "]"); + if (processed == false) { + assert false : "CS published to itself not processed"; + 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 getFaultDetectionNodes() { + Set getFaultDetectionNodes() { return nodesFD.getNodes(); } @@ -347,7 +410,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover return new DiscoveryStats(queueStats); } - @Override public DiscoverySettings getDiscoverySettings() { return discoverySettings; } @@ -391,22 +453,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 +482,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 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 +521,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 +551,16 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover } private void submitRejoin(String source) { - clusterService.submitStateUpdateTask(source, new LocalClusterUpdateTask(Priority.IMMEDIATE) { - @Override - public ClusterTasksResult 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 +657,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 +700,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 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 +720,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 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 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 +933,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 pingResponses = filterPingResponses(fullPingResponses, masterElectionIgnoreNonMasters, logger); @@ -956,10 +1001,9 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover return pingResponses; } - protected ClusterStateTaskExecutor.ClusterTasksResult 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 +1012,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 +1087,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 +1128,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 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 +1173,14 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover class RejoinClusterRequestHandler implements TransportRequestHandler { @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 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 +1206,15 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover } /** cleans any running joining thread and calls {@link #rejoin} */ - public ClusterStateTaskExecutor.ClusterTasksResult 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 +1247,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 +1256,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); } diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayService.java b/core/src/main/java/org/elasticsearch/gateway/GatewayService.java index c215585c45c..0353deab6e5 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayService.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayService.java @@ -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 diff --git a/core/src/main/java/org/elasticsearch/indices/store/IndicesStore.java b/core/src/main/java/org/elasticsearch/indices/store/IndicesStore.java index 9c9731bc155..2ae8d12a9fe 100644 --- a/core/src/main/java/org/elasticsearch/indices/store/IndicesStore.java +++ b/core/src/main/java/org/elasticsearch/indices/store/IndicesStore.java @@ -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 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) + ); } } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 764ece5b868..cbb68dc2924 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -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 diff --git a/core/src/main/java/org/elasticsearch/node/NodeService.java b/core/src/main/java/org/elasticsearch/node/NodeService.java index cb245487152..1c283522cc6 100644 --- a/core/src/main/java/org/elasticsearch/node/NodeService.java +++ b/core/src/main/java/org/elasticsearch/node/NodeService.java @@ -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, diff --git a/core/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java b/core/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java index 61e87d83a18..0569547b822 100644 --- a/core/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java +++ b/core/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java @@ -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> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService, NamedWriteableRegistry namedWriteableRegistry, - ClusterService clusterService, UnicastHostsProvider hostsProvider) { + MasterService masterService, + ClusterApplier clusterApplier, + ClusterSettings clusterSettings, + UnicastHostsProvider hostsProvider) { return Collections.emptyMap(); } diff --git a/core/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/core/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index 7aea5d18afb..2efbae5961e 100644 --- a/core/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/core/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -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(); } }); } diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index fb9d7babf20..cec01732c42 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -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 tribeClientNodes = TribeService.this.nodes; Map 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) diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java index 51c66033ca9..9e45b0d3828 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java @@ -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); diff --git a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java index f2cd88a27a7..f2537e746ad 100644 --- a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java @@ -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 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(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java index f4ce40e16db..e38f09e567a 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java @@ -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 } diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java new file mode 100644 index 00000000000..9782dcdc858 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java @@ -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 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 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(); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceIT.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceIT.java index f056eded34b..8514cb4ac2e 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceIT.java @@ -382,7 +382,7 @@ public class ClusterServiceIT extends ESIntegTestCase { // The tasks can be re-ordered, so we need to check out-of-order Set controlSources = new HashSet<>(Arrays.asList("1", "2", "3", "4", "5", "6", "7", "8", "9", "10")); - List pendingClusterTasks = clusterService.pendingTasks(); + List 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) { diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/service/MasterServiceTests.java similarity index 59% rename from core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java rename to core/src/test/java/org/elasticsearch/cluster/service/MasterServiceTests.java index 8afadff0825..1b747f22687 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/MasterServiceTests.java @@ -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 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 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() { + 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 execute(ClusterState currentState, List 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 future = new BaseFuture() {}; - 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 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 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 future = new BaseFuture() {}; + 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 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 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 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 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 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(); + } } diff --git a/core/src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsIT.java b/core/src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsIT.java index c52768d7b7e..bab53b8f35c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsIT.java @@ -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"; diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java index 378557b4047..9460261e547 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java @@ -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> impl(); @@ -74,7 +70,8 @@ public class DiscoveryModuleTests extends ESTestCase { @Override default Map> 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 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() { diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index 021e2be85ed..9d46b35377c 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -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); - } - } + } + }); } /** diff --git a/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java b/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java index 59fe5872911..d32f8cba334 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java @@ -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 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 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 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 diff --git a/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java b/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java index a0e0b699d78..917894b60d7 100644 --- a/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java @@ -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 { diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index c5c78189ee1..c25152a4426 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -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 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 nodes = new ArrayList<>(); - nodes.add(clusterService.localNode()); + nodes.add(discoveryState(masterService).nodes().getLocalNode()); final CyclicBarrier barrier = new CyclicBarrier(threads.length); final List 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 nodes = new ArrayList<>(); - nodes.add(clusterService.localNode()); + nodes.add(discoveryState(masterService).nodes().getLocalNode()); final CyclicBarrier barrier = new CyclicBarrier(threads.length + 1); final List 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 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) { diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java index dfe1078aaa0..7ac463616fc 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java @@ -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(); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index 5bbd0f19d2f..0d0d391663d 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -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; } diff --git a/core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java b/core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java index a4e3aad8bec..2bec3d5eded 100644 --- a/core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java @@ -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); } diff --git a/core/src/test/java/org/elasticsearch/indexlifecycle/IndexLifecycleActionIT.java b/core/src/test/java/org/elasticsearch/indexlifecycle/IndexLifecycleActionIT.java index db03e51d4d9..1bd255349ca 100644 --- a/core/src/test/java/org/elasticsearch/indexlifecycle/IndexLifecycleActionIT.java +++ b/core/src/test/java/org/elasticsearch/indexlifecycle/IndexLifecycleActionIT.java @@ -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; } diff --git a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java index 13f00b152df..4e1be614fa5 100644 --- a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java @@ -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 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) { diff --git a/core/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/core/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index 5562e2d4026..aa9b4ba4a16 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/core/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -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 blockOn; - private final Predicate 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; - } - } } diff --git a/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index 2c1dfc899b6..355b75219a0 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -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"; diff --git a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index d8a0036a45c..1783363e7ce 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -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 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(); diff --git a/core/src/test/java/org/elasticsearch/test/NoopDiscovery.java b/core/src/test/java/org/elasticsearch/test/NoopDiscovery.java index c36082f1475..1a661b509a2 100644 --- a/core/src/test/java/org/elasticsearch/test/NoopDiscovery.java +++ b/core/src/test/java/org/elasticsearch/test/NoopDiscovery.java @@ -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; } diff --git a/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/plugin/discovery/azure/classic/AzureDiscoveryPlugin.java b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/plugin/discovery/azure/classic/AzureDiscoveryPlugin.java index 38e29e04583..6bff34a667c 100644 --- a/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/plugin/discovery/azure/classic/AzureDiscoveryPlugin.java +++ b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/plugin/discovery/azure/classic/AzureDiscoveryPlugin.java @@ -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> 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 diff --git a/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryPlugin.java b/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryPlugin.java index d9923fe0f81..3c80d0eda06 100644 --- a/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryPlugin.java +++ b/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryPlugin.java @@ -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> 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 diff --git a/plugins/discovery-gce/src/main/java/org/elasticsearch/plugin/discovery/gce/GceDiscoveryPlugin.java b/plugins/discovery-gce/src/main/java/org/elasticsearch/plugin/discovery/gce/GceDiscoveryPlugin.java index acad7e099f5..44fb5c8d15e 100644 --- a/plugins/discovery-gce/src/main/java/org/elasticsearch/plugin/discovery/gce/GceDiscoveryPlugin.java +++ b/plugins/discovery-gce/src/main/java/org/elasticsearch/plugin/discovery/gce/GceDiscoveryPlugin.java @@ -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> 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 diff --git a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java index b2eced6fd35..f6c595b9b97 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java @@ -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 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 = 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 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 createClusterStatePublisher(ClusterApplier clusterApplier) { + return (event, ackListener) -> { + CountDownLatch latch = new CountDownLatch(1); + AtomicReference 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); + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 0240a8c4315..bd5a809d8cd 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -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()); }); diff --git a/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java b/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java index c2a237ef337..cb4b8e098ae 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java +++ b/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java @@ -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> 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 diff --git a/test/framework/src/main/java/org/elasticsearch/test/disruption/BlockClusterStateProcessing.java b/test/framework/src/main/java/org/elasticsearch/test/disruption/BlockClusterStateProcessing.java index 8a6be290502..f144cb0b118 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/disruption/BlockClusterStateProcessing.java +++ b/test/framework/src/main/java/org/elasticsearch/test/disruption/BlockClusterStateProcessing.java @@ -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 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) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/disruption/SlowClusterStateProcessing.java b/test/framework/src/main/java/org/elasticsearch/test/disruption/SlowClusterStateProcessing.java index 61afa4f77f3..03ffe1d690a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/disruption/SlowClusterStateProcessing.java +++ b/test/framework/src/main/java/org/elasticsearch/test/disruption/SlowClusterStateProcessing.java @@ -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 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) { From b77254871b714a558d39f268f2beaf5a7516302a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 28 Apr 2017 11:09:24 +0200 Subject: [PATCH 03/10] docs: document alternative for nested inner hits source Closes #24110 --- .../search/request/inner-hits.asciidoc | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/docs/reference/search/request/inner-hits.asciidoc b/docs/reference/search/request/inner-hits.asciidoc index 28914cd9b5e..1b118a419ed 100644 --- a/docs/reference/search/request/inner-hits.asciidoc +++ b/docs/reference/search/request/inner-hits.asciidoc @@ -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. From a72db191f296336305ba1e349ec443c75ada3ce1 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 28 Apr 2017 11:16:07 +0200 Subject: [PATCH 04/10] Weaken assertion in ZenDiscovery.publish The previous commit (35f78d098a) introduced an assertion in ZenDiscovery that was overly restrictive - it could trip when a cluster state that was successfully published would not be applied locally because a master with a better cluster state came along in the meantime. --- .../org/elasticsearch/discovery/zen/ZenDiscovery.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 5da446aa5b9..f4bef4b2d91 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -335,16 +335,19 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover 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); } @Override public void onNewClusterStateFailed(Exception e) { + processedOrFailed.set(true); latch.countDown(); ackListener.onNodeAck(localNode, e); logger.warn( @@ -360,10 +363,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover throw new FailedToCommitClusterStateException("local state was mutated while CS update was published to other nodes"); } - boolean processed = processNextCommittedClusterState("master " + newState.nodes().getMasterNode() + + boolean sentToApplier = processNextCommittedClusterState("master " + newState.nodes().getMasterNode() + " committed version [" + newState.version() + "] source [" + clusterChangedEvent.source() + "]"); - if (processed == false) { - assert false : "CS published to itself not processed"; + 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; } } From a5bd2012b6046be76fbc6a60b8463e9eb88fdb9c Mon Sep 17 00:00:00 2001 From: Kunal Kapoor Date: Fri, 28 Apr 2017 17:32:09 +0530 Subject: [PATCH 05/10] Added validation for upserd request (#24282) The version on an update request is a syntactic sugar for get of a specific version, doc merge and a version index. This changes it to reject requests with both upsert and a version. If the upsert index request is versioned, we also reject the op. --- .../action/update/UpdateRequest.java | 6 ++ .../action/bulk/BulkRequestTests.java | 36 ++++++++++++ .../action/update/UpdateRequestTests.java | 17 ++++++ .../org/elasticsearch/update/UpdateIT.java | 56 +------------------ 4 files changed, 60 insertions(+), 55 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java b/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java index 2f153cdbef7..fa8c46edf5b 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java @@ -98,6 +98,12 @@ public class UpdateRequest extends InstanceShardOperationRequest @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); } diff --git a/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java b/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java index d2b63ad3965..76810056485 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java @@ -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 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")); + } + } diff --git a/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index d6a087bec45..339976a7086 100644 --- a/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -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")); + } } diff --git a/core/src/test/java/org/elasticsearch/update/UpdateIT.java b/core/src/test/java/org/elasticsearch/update/UpdateIT.java index 893bbbd6115..10d235d3a85 100644 --- a/core/src/test/java/org/elasticsearch/update/UpdateIT.java +++ b/core/src/test/java/org/elasticsearch/update/UpdateIT.java @@ -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()) From 2412574e49f7bcfc63fda85f21121618aa3d3f79 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 28 Apr 2017 08:43:46 -0400 Subject: [PATCH 06/10] Docs: Upserts no longer support version Closes #16671 --- docs/reference/migration/migrate_6_0/docs.asciidoc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/reference/migration/migrate_6_0/docs.asciidoc b/docs/reference/migration/migrate_6_0/docs.asciidoc index 5d19c000ad7..b7a08ea8930 100644 --- a/docs/reference/migration/migrate_6_0/docs.asciidoc +++ b/docs/reference/migration/migrate_6_0/docs.asciidoc @@ -5,3 +5,7 @@ Document modification operations may no longer specify the `version_type` of `force` to override any previous version checks. + +==== <> no longer support versions + +Adding a `version` to an upsert request is no longer supported. From 382a617d346ffdcd785e385bc1ec9fd542a0d454 Mon Sep 17 00:00:00 2001 From: Guillaume Le Floch Date: Fri, 28 Apr 2017 15:21:44 +0200 Subject: [PATCH 07/10] Handle multiple aliases in _cat/aliases api (#23698) The alias parameter was documented as a list in our rest-spec, yet only the first value out of a list was getting read and processed. This commit adds support for multiple aliases to _cat/aliases Closes #23661 --- .../rest/action/cat/RestAliasAction.java | 2 +- docs/reference/cat/alias.asciidoc | 5 +- .../test/cat.aliases/10_basic.yaml | 46 +++++++++++++++++++ 3 files changed, 50 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestAliasAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestAliasAction.java index 120678d2e82..a783a9c2a82 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestAliasAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestAliasAction.java @@ -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())); diff --git a/docs/reference/cat/alias.asciidoc b/docs/reference/cat/alias.asciidoc index 93a1fdf3eb6..84d567d110a 100644 --- a/docs/reference/cat/alias.asciidoc +++ b/docs/reference/cat/alias.asciidoc @@ -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. \ No newline at end of file diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml index fc7eb456892..12879fa412a 100755 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml @@ -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": From e3b7b88756790712717301b34285a1bb2128bd2b Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 28 Apr 2017 09:52:14 -0400 Subject: [PATCH 08/10] Fix compilation in Ecipse (#24391) Eclipse doesn't allow extra semicolons after an import statement: ``` import foo.Bar;; // <-- syntax error! ``` Here is the Eclipse bug: https://bugs.eclipse.org/bugs/show_bug.cgi?id=425140 which the Eclipse folks closed as "the spec doesn't allow these semicolons so why should we?" Which is fair. Here is the bug against javac for allowing them: https://bugs.openjdk.java.net/browse/JDK-8027682 which hasn't been touched since 2013 without explanation. There is, however, a rather educations mailing list thread: http://mail.openjdk.java.net/pipermail/compiler-dev/2013-August/006956.html which contains gems like, "In general, it is better/simpler to change javac to conform to the spec. (Except when it is not.)" I suspect the reason this hasn't been fixed is: ``` FWIW, if we change javac such that the set of programs accepted by javac is changed, we have an process (currently Oracle internal) to get approval for such a change. So, we would not simply change javac on a whim to meet the spec; we would at least have other eyes looking at the behavioral change to determine if it is "acceptable". ``` from http://mail.openjdk.java.net/pipermail/compiler-dev/2013-August/006973.html --- .../main/java/org/elasticsearch/discovery/TribeDiscovery.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java index 751eb94a9f2..9f802cc270a 100644 --- a/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java @@ -24,7 +24,7 @@ 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.cluster.service.ClusterApplier; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.single.SingleNodeDiscovery; From 94e3796908f7f11c6522fd6f533dd4f0972394f5 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 28 Apr 2017 09:56:19 -0400 Subject: [PATCH 09/10] Docs tests: cat/health can have max_task_wait_time Make the doc test assertions ok with a non `-` value for `max_task_wait_time`. These are rare, but possible: https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+master+multijob-unix-compatibility/os=oraclelinux/900/consoleFull --- docs/reference/cat/health.asciidoc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/reference/cat/health.asciidoc b/docs/reference/cat/health.asciidoc index a87fe4e5e4a..ca2a1838adb 100644 --- a/docs/reference/cat/health.asciidoc +++ b/docs/reference/cat/health.asciidoc @@ -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: From 16a7cbe4635747909f4112eb4e162fb40cef9da6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 28 Apr 2017 16:25:22 +0200 Subject: [PATCH 10/10] Add `count` value to rest output of `geo_centroid` (#24387) Currently we don't write the count value to the geo_centroid aggregation rest response, but it is provided via the java api and the count() method in the GeoCentroid interface. We should add this parameter to the rest output and also provide it via the getProperty() method. --- .../geocentroid/InternalGeoCentroid.java | 24 +++++++++++++++--- .../aggregations/metrics/GeoCentroidIT.java | 6 +++++ .../geocentroid/InternalGeoCentroidTests.java | 25 ++++++++++++------- .../metrics/geocentroid-aggregation.asciidoc | 12 ++++++--- 4 files changed, 50 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroid.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroid.java index da69115ac6b..c5578813c84 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroid.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroid.java @@ -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; } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java index 6f2c979936f..32b036606d3 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java @@ -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 { diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java index c409d2aa795..3bbe1a1b462 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroidTests.java @@ -42,8 +42,11 @@ public class InternalGeoCentroidTests extends InternalAggregationTestCase 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()); } } diff --git a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc index 4756ebdafc8..7a355c7b126 100644 --- a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc @@ -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 } } ]