From b6cbcc49ba590ead3cd6d95953eabe22f7d47fef Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 15 Dec 2016 17:06:25 +0100 Subject: [PATCH] ClusterService should expose "applied" cluster states (i.e., remove ClusterStateStatus) (#21817) `ClusterService` is responsible of updating the cluster state on every node (as a response to an API call on the master and when non-masters receive a new state from the master). When a new cluster state is processed, it is made visible via the `ClusterService#state` method and is sent to series of listeners. Those listeners come in two flavours - one is to change the state of the node in response to the new cluster state (call these cluster state appliers), the other is to start a secondary process. Examples for the later include an indexing operation waiting for a shard to be started or a master node action waiting for a master to be elected. The fact that we expose the state before applying it means that samplers of the cluster state had to worry about two things - working based on a stale CS and working based on a future, i.e., "being applied" CS. The `ClusterStateStatus` was used to allow distinguishing between the two. Working with a stale cluster state is not avoidable. How this PR changes things to make sure consumers don't need to worry about future CS, removing the need for the status and simplifying the waiting logic. This change does come with a price as "cluster state appliers" can't sample the cluster state from `ClusterService` whenever they want as the cluster state isn't exposed yet. However, recent clean ups made this is situation easier and this PR takes the last steps to remove such sampling. This also helps clarify the "information flow" and helps component separation (and thus potential unit testing). It also adds an assertion that will trigger if the cluster state is sampled by such listeners. Note that there are still many "appliers" that could be made a simpler, unrestricted "listener" but this can be done in smaller bits in the future. The commit also makes it clear what the `appliers` and what the `listeners` are by using dedicated interfaces. Also, since I had to change the listener types I went ahead and changed the data structure for temporary/timeout listeners (used for the observer) so addition and removal won't be an O(n) operation. --- .../health/TransportClusterHealthAction.java | 16 +- .../TransportRestoreSnapshotAction.java | 6 +- .../action/bulk/TransportBulkAction.java | 27 +-- .../action/index/TransportIndexAction.java | 4 +- .../action/ingest/IngestActionForwarder.java | 10 +- .../action/support/ActiveShardsObserver.java | 12 +- .../master/TransportMasterNodeAction.java | 30 +-- .../TransportReplicationAction.java | 2 +- ...ransportInstanceSingleOperationAction.java | 4 +- ...teStatus.java => ClusterStateApplier.java} | 18 +- .../cluster/ClusterStateObserver.java | 142 ++++------- .../cluster/InternalClusterInfoService.java | 19 +- .../cluster/MasterNodeChangePredicate.java | 37 +-- .../action/shard/ShardStateAction.java | 26 +- .../metadata/MetaDataCreateIndexService.java | 13 +- .../metadata/MetaDataIndexAliasesService.java | 4 +- .../MetaDataIndexTemplateService.java | 5 +- .../metadata/MetaDataMappingService.java | 5 +- .../MetaDataUpdateSettingsService.java | 2 +- .../routing/DelayedAllocationService.java | 4 +- .../cluster/service/ClusterService.java | 226 +++++++++--------- .../cluster/service/ClusterServiceState.java | 49 ---- .../zen/PublishClusterStateAction.java | 1 - .../gateway/DanglingIndicesState.java | 15 +- .../org/elasticsearch/gateway/Gateway.java | 10 +- .../gateway/GatewayAllocator.java | 31 +-- .../gateway/GatewayMetaState.java | 11 +- .../elasticsearch/gateway/GatewayService.java | 6 +- .../index/CompositeIndexEventListener.java | 37 +-- .../org/elasticsearch/index/IndexService.java | 1 - .../index/query/QueryRewriteContext.java | 15 +- .../index/query/QueryShardContext.java | 7 +- .../index/shard/IndexEventListener.java | 45 +--- .../elasticsearch/index/shard/IndexShard.java | 4 +- .../elasticsearch/indices/IndicesService.java | 64 +---- .../cluster/IndicesClusterStateService.java | 126 ++++++---- .../recovery/PeerRecoveryTargetService.java | 15 +- .../indices/store/IndicesStore.java | 25 +- .../ingest/PipelineExecutionService.java | 6 +- .../elasticsearch/ingest/PipelineStore.java | 20 +- .../java/org/elasticsearch/node/Node.java | 9 +- .../node/service/NodeService.java | 4 +- .../repositories/RepositoriesService.java | 8 +- .../elasticsearch/search/SearchService.java | 25 +- .../snapshots/RestoreService.java | 9 +- .../snapshots/SnapshotShardsService.java | 54 +++-- .../snapshots/SnapshotsService.java | 12 +- .../org/elasticsearch/tasks/TaskManager.java | 6 +- .../org/elasticsearch/tribe/TribeService.java | 2 +- .../node/tasks/TaskManagerTestCase.java | 2 +- .../bulk/TransportBulkActionIngestTests.java | 18 +- .../TransportIndexActionIngestTests.java | 14 +- .../health/ClusterStateHealthTests.java | 22 +- .../DelayedAllocationServiceTests.java | 2 +- .../cluster/service/ClusterServiceTests.java | 40 +++- .../DiscoveryWithServiceDisruptionsIT.java | 39 ++- .../discovery/zen/ZenDiscoveryIT.java | 2 +- .../gateway/DanglingIndicesStateTests.java | 17 +- .../elasticsearch/index/IndexModuleTests.java | 5 +- .../AbstractGeoFieldDataTestCase.java | 1 - .../index/mapper/DateFieldTypeTests.java | 18 +- .../index/mapper/RangeFieldTypeTests.java | 2 +- .../index/query/QueryShardContextTests.java | 3 +- .../index/query/RangeQueryRewriteTests.java | 6 +- .../index/query/SimpleQueryParserTests.java | 24 +- ...dicesLifecycleListenerSingleNodeTests.java | 58 +++-- ...actIndicesClusterStateServiceTestCase.java | 21 +- ...ClusterStateServiceRandomUpdatesTests.java | 23 +- .../search/SearchServiceTests.java | 3 +- .../bucket/histogram/ExtendedBoundsTests.java | 2 +- .../highlight/HighlightBuilderTests.java | 2 +- .../ShardSearchTransportRequestTests.java | 3 +- .../rescore/QueryRescoreBuilderTests.java | 2 +- .../search/sort/AbstractSortTestCase.java | 2 +- .../DedicatedClusterSnapshotRestoreIT.java | 4 +- .../SharedClusterSnapshotRestoreIT.java | 4 +- .../test/AbstractQueryTestCase.java | 6 +- .../test/MockIndexEventListener.java | 21 +- .../search/MockSearchServiceTests.java | 2 +- 79 files changed, 740 insertions(+), 857 deletions(-) rename core/src/main/java/org/elasticsearch/cluster/{service/ClusterStateStatus.java => ClusterStateApplier.java} (59%) delete mode 100644 core/src/main/java/org/elasticsearch/cluster/service/ClusterServiceState.java 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 9773410aacc..7dd2c0df84c 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 @@ -34,8 +34,6 @@ import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.cluster.service.ClusterServiceState; -import org.elasticsearch.cluster.service.ClusterStateStatus; import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; @@ -46,6 +44,8 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.util.function.Predicate; + public class TransportClusterHealthAction extends TransportMasterNodeReadAction { private final GatewayAllocator gatewayAllocator; @@ -142,19 +142,13 @@ public class TransportClusterHealthAction extends TransportMasterNodeReadAction< assert waitFor >= 0; final ClusterStateObserver observer = new ClusterStateObserver(clusterService, logger, threadPool.getThreadContext()); - final ClusterServiceState observedState = observer.observedState(); - final ClusterState state = observedState.getClusterState(); + final ClusterState state = observer.observedState(); if (request.timeout().millis() == 0) { listener.onResponse(getResponse(request, state, waitFor, request.timeout().millis() == 0)); return; } final int concreteWaitFor = waitFor; - final ClusterStateObserver.ChangePredicate validationPredicate = new ClusterStateObserver.ValidationPredicate() { - @Override - protected boolean validate(ClusterServiceState newState) { - return newState.getClusterStateStatus() == ClusterStateStatus.APPLIED && validateRequest(request, newState.getClusterState(), concreteWaitFor); - } - }; + final Predicate validationPredicate = newState -> validateRequest(request, newState, concreteWaitFor); final ClusterStateObserver.Listener stateListener = new ClusterStateObserver.Listener() { @Override @@ -174,7 +168,7 @@ public class TransportClusterHealthAction extends TransportMasterNodeReadAction< listener.onResponse(response); } }; - if (observedState.getClusterStateStatus() == ClusterStateStatus.APPLIED && validateRequest(request, state, concreteWaitFor)) { + if (validationPredicate.test(state)) { stateListener.onNewClusterState(state); } else { observer.waitForNextChange(stateListener, validationPredicate, request.timeout()); diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java index 3ef12dfff0e..5ff5bd17fe5 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java @@ -101,10 +101,10 @@ public class TransportRestoreSnapshotAction extends TransportMasterNodeAction shards = prevEntry.shards(); assert prevEntry.state().completed() : "expected completed snapshot state but was " + prevEntry.state(); assert RestoreService.completed(shards) : "expected all restore entries to be completed"; @@ -121,7 +121,7 @@ public class TransportRestoreSnapshotAction extends TransportMasterNodeAction shardsAllocatedPredicate = newState -> activeShardCount.enoughShardsActive(newState, indexName); final ClusterStateObserver.Listener observerListener = new ClusterStateObserver.Listener() { @Override diff --git a/core/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java b/core/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java index 47c3f6cf8e0..ecb03b5c222 100644 --- a/core/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java @@ -34,7 +34,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.cluster.service.ClusterServiceState; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.Discovery; @@ -46,6 +45,7 @@ import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportService; +import java.util.function.Predicate; import java.util.function.Supplier; /** @@ -111,14 +111,6 @@ public abstract class TransportMasterNodeAction listener) { this.task = task; this.request = request; @@ -134,7 +126,8 @@ public abstract class TransportMasterNodeAction masterChangePredicate = MasterNodeChangePredicate.build(clusterState); final DiscoveryNodes nodes = clusterState.nodes(); if (nodes.isLocalNodeElectedMaster() || localExecute(request)) { // check for block, if blocked, retry, else, execute locally @@ -144,7 +137,10 @@ public abstract class TransportMasterNodeAction { + ClusterBlockException newException = checkBlock(request, newState); + return (newException == null || !newException.retryable()); + }); } } else { ActionListener delegate = new ActionListener() { @@ -158,7 +154,7 @@ public abstract class TransportMasterNodeAction) () -> new ParameterizedMessage("master could not publish cluster state or stepped down before publishing action [{}], scheduling a retry", actionName), t); - retry(t, MasterNodeChangePredicate.INSTANCE); + retry(t, masterChangePredicate); } else { listener.onFailure(t); } @@ -168,14 +164,14 @@ public abstract class TransportMasterNodeAction(listener, TransportMasterNodeAction.this::newResponse) { @@ -186,7 +182,7 @@ public abstract class TransportMasterNodeAction statePredicate) { observer.waitForNextChange( new ClusterStateObserver.Listener() { @Override @@ -214,7 +210,7 @@ public abstract class TransportMasterNodeAction) () -> new ParameterizedMessage("timed out while retrying [{}] after failure (timeout [{}])", actionName, timeout), failure); listener.onFailure(new MasterNotDiscoveredException(failure)); } - }, changePredicate + }, statePredicate ); } } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 085b3c10018..8cff657e3dd 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -630,7 +630,7 @@ public abstract class TransportReplicationAction< @Override protected void doRun() { setPhase(task, "routing"); - final ClusterState state = observer.observedState().getClusterState(); + final ClusterState state = observer.observedState(); if (handleBlockExceptions(state)) { return; } diff --git a/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java b/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java index c5014adf570..7670eab102a 100644 --- a/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationAction.java @@ -39,12 +39,12 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequestHandler; import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import java.util.function.Supplier; @@ -124,7 +124,7 @@ public abstract class TransportInstanceSingleOperationAction MATCH_ALL_CHANGES_PREDICATE = state -> true; private final ClusterService clusterService; private final ThreadContext contextHolder; volatile TimeValue timeOutValue; - final AtomicReference lastObservedState; + final AtomicReference lastObservedState; final TimeoutClusterStateListener clusterStateListener = new ObserverClusterStateListener(); // observingContext is not null when waiting on cluster state changes final AtomicReference observingContext = new AtomicReference<>(null); @@ -70,8 +63,17 @@ public class ClusterStateObserver { * to wait indefinitely */ public ClusterStateObserver(ClusterService clusterService, @Nullable TimeValue timeout, Logger logger, ThreadContext contextHolder) { + this(clusterService.state(), clusterService, timeout, logger, contextHolder); + } + /** + * @param timeout a global timeout for this observer. After it has expired the observer + * will fail any existing or new #waitForNextChange calls. Set to null + * to wait indefinitely + */ + public ClusterStateObserver(ClusterState initialState, ClusterService clusterService, @Nullable TimeValue timeout, Logger logger, + ThreadContext contextHolder) { this.clusterService = clusterService; - this.lastObservedState = new AtomicReference<>(clusterService.clusterServiceState()); + this.lastObservedState = new AtomicReference<>(initialState); this.timeOutValue = timeout; if (timeOutValue != null) { this.startTimeNS = System.nanoTime(); @@ -81,8 +83,8 @@ public class ClusterStateObserver { } /** last cluster state and status observed by this observer. Note that this may not be the current one */ - public ClusterServiceState observedState() { - ClusterServiceState state = lastObservedState.get(); + public ClusterState observedState() { + ClusterState state = lastObservedState.get(); assert state != null; return state; } @@ -100,18 +102,18 @@ public class ClusterStateObserver { waitForNextChange(listener, MATCH_ALL_CHANGES_PREDICATE, timeOutValue); } - public void waitForNextChange(Listener listener, ChangePredicate changePredicate) { - waitForNextChange(listener, changePredicate, null); + public void waitForNextChange(Listener listener, Predicate statePredicate) { + waitForNextChange(listener, statePredicate, null); } /** - * Wait for the next cluster state which satisfies changePredicate + * Wait for the next cluster state which satisfies statePredicate * * @param listener callback listener - * @param changePredicate predicate to check whether cluster state changes are relevant and the callback should be called + * @param statePredicate predicate to check whether cluster state changes are relevant and the callback should be called * @param timeOutValue a timeout for waiting. If null the global observer timeout will be used. */ - public void waitForNextChange(Listener listener, ChangePredicate changePredicate, @Nullable TimeValue timeOutValue) { + public void waitForNextChange(Listener listener, Predicate statePredicate, @Nullable TimeValue timeOutValue) { if (observingContext.get() != null) { throw new ElasticsearchException("already waiting for a cluster state change"); @@ -128,7 +130,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(clusterService.clusterServiceState()); + lastObservedState.set(clusterService.state()); listener.onTimeout(timeOutValue); return; } @@ -143,33 +145,24 @@ public class ClusterStateObserver { } // sample a new state - ClusterServiceState newState = clusterService.clusterServiceState(); - ClusterServiceState lastState = lastObservedState.get(); - if (changePredicate.apply(lastState, newState)) { + ClusterState newState = clusterService.state(); + ClusterState lastState = lastObservedState.get(); + if (newState != lastState && statePredicate.test(newState)) { // good enough, let's go. logger.trace("observer: sampled state accepted by predicate ({})", newState); lastObservedState.set(newState); - listener.onNewClusterState(newState.getClusterState()); + listener.onNewClusterState(newState); } else { logger.trace("observer: sampled state rejected by predicate ({}). adding listener to ClusterService", newState); - ObservingContext context = new ObservingContext(new ContextPreservingListener(listener, contextHolder.newStoredContext()), changePredicate); + ObservingContext context = + new ObservingContext(new ContextPreservingListener(listener, contextHolder.newStoredContext()), statePredicate); if (!observingContext.compareAndSet(null, context)) { throw new ElasticsearchException("already waiting for a cluster state change"); } - clusterService.add(timeoutTimeLeftMS == null ? null : new TimeValue(timeoutTimeLeftMS), clusterStateListener); + clusterService.addTimeoutListener(timeoutTimeLeftMS == null ? null : new TimeValue(timeoutTimeLeftMS), clusterStateListener); } } - /** - * reset this observer to the give cluster state. Any pending waits will be canceled. - */ - public void reset(ClusterServiceState state) { - if (observingContext.getAndSet(null) != null) { - clusterService.remove(clusterStateListener); - } - lastObservedState.set(state); - } - class ObserverClusterStateListener implements TimeoutClusterStateListener { @Override @@ -179,18 +172,18 @@ public class ClusterStateObserver { // No need to remove listener as it is the responsibility of the thread that set observingContext to null return; } - if (context.changePredicate.apply(event)) { + final ClusterState state = event.state(); + if (context.statePredicate.test(state)) { if (observingContext.compareAndSet(context, null)) { - clusterService.remove(this); - ClusterServiceState state = new ClusterServiceState(event.state(), ClusterStateStatus.APPLIED); + clusterService.removeTimeoutListener(this); logger.trace("observer: accepting cluster state change ({})", state); lastObservedState.set(state); - context.listener.onNewClusterState(state.getClusterState()); + context.listener.onNewClusterState(state); } else { - logger.trace("observer: predicate approved change but observing context has changed - ignoring (new cluster state version [{}])", event.state().version()); + logger.trace("observer: predicate approved change but observing context has changed - ignoring (new cluster state version [{}])", state.version()); } } else { - logger.trace("observer: predicate rejected change (new cluster state version [{}])", event.state().version()); + logger.trace("observer: predicate rejected change (new cluster state version [{}])", state.version()); } } @@ -201,15 +194,15 @@ public class ClusterStateObserver { // No need to remove listener as it is the responsibility of the thread that set observingContext to null return; } - ClusterServiceState newState = clusterService.clusterServiceState(); - ClusterServiceState lastState = lastObservedState.get(); - if (context.changePredicate.apply(lastState, newState)) { + ClusterState newState = clusterService.state(); + ClusterState lastState = lastObservedState.get(); + if (newState != lastState && 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.remove(this); + clusterService.removeTimeoutListener(this); lastObservedState.set(newState); - context.listener.onNewClusterState(newState.getClusterState()); + context.listener.onNewClusterState(newState); } else { logger.trace("observer: postAdded - predicate approved state but observing context has changed - ignoring ({})", newState); } @@ -224,7 +217,7 @@ public class ClusterStateObserver { if (context != null) { logger.trace("observer: cluster service closed. notifying listener."); - clusterService.remove(this); + clusterService.removeTimeoutListener(this); context.listener.onClusterServiceClose(); } } @@ -233,11 +226,11 @@ public class ClusterStateObserver { public void onTimeout(TimeValue timeout) { ObservingContext context = observingContext.getAndSet(null); if (context != null) { - clusterService.remove(this); + clusterService.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(clusterService.clusterServiceState()); + lastObservedState.set(clusterService.state()); timedOut = true; context.listener.onTimeout(timeOutValue); } @@ -255,58 +248,13 @@ public class ClusterStateObserver { void onTimeout(TimeValue timeout); } - public interface ChangePredicate { - - /** - * a rough check used when starting to monitor for a new change. Called infrequently can be less accurate. - * - * @return true if newState should be accepted - */ - boolean apply(ClusterServiceState previousState, - ClusterServiceState newState); - - /** - * called to see whether a cluster change should be accepted - * - * @return true if changedEvent.state() should be accepted - */ - boolean apply(ClusterChangedEvent changedEvent); - } - - - public abstract static class ValidationPredicate implements ChangePredicate { - - @Override - public boolean apply(ClusterServiceState previousState, ClusterServiceState newState) { - return (previousState.getClusterState() != newState.getClusterState() || - previousState.getClusterStateStatus() != newState.getClusterStateStatus()) && - validate(newState); - } - - protected abstract boolean validate(ClusterServiceState newState); - - @Override - public boolean apply(ClusterChangedEvent changedEvent) { - return changedEvent.previousState().version() != changedEvent.state().version() && - validate(new ClusterServiceState(changedEvent.state(), ClusterStateStatus.APPLIED)); - } - } - - public abstract static class EventPredicate implements ChangePredicate { - @Override - public boolean apply(ClusterServiceState previousState, ClusterServiceState newState) { - return previousState.getClusterState() != newState.getClusterState() || previousState.getClusterStateStatus() != newState.getClusterStateStatus(); - } - - } - static class ObservingContext { public final Listener listener; - public final ChangePredicate changePredicate; + public final Predicate statePredicate; - public ObservingContext(Listener listener, ChangePredicate changePredicate) { + public ObservingContext(Listener listener, Predicate statePredicate) { this.listener = listener; - this.changePredicate = changePredicate; + this.statePredicate = statePredicate; } } diff --git a/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java b/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java index 70656bb56bd..b8ac2a5eb50 100644 --- a/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java +++ b/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java @@ -19,11 +19,6 @@ package org.elasticsearch.cluster; -import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.LatchedActionListener; @@ -53,6 +48,11 @@ import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ReceiveTimeoutTransportException; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + /** * InternalClusterInfoService provides the ClusterInfoService interface, * routinely updated on a timer. The timer can be dynamically changed by @@ -64,7 +64,8 @@ import org.elasticsearch.transport.ReceiveTimeoutTransportException; * Every time the timer runs, gathers information about the disk usage and * shard sizes across the cluster. */ -public class InternalClusterInfoService extends AbstractComponent implements ClusterInfoService, LocalNodeMasterListener, ClusterStateListener { +public class InternalClusterInfoService extends AbstractComponent + implements ClusterInfoService, LocalNodeMasterListener, ClusterStateListener { public static final Setting INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING = Setting.timeSetting("cluster.info.update.interval", TimeValue.timeValueSeconds(30), TimeValue.timeValueSeconds(10), @@ -105,9 +106,9 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu clusterSettings.addSettingsUpdateConsumer(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, this::setEnabled); // Add InternalClusterInfoService to listen for Master changes - this.clusterService.add((LocalNodeMasterListener)this); + this.clusterService.addLocalNodeMasterListener(this); // Add to listen for state changes (when nodes are added) - this.clusterService.add((ClusterStateListener)this); + this.clusterService.addListener(this); } private void setEnabled(boolean enabled) { @@ -167,7 +168,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu } } - if (this.isMaster && dataNodeAdded && clusterService.state().getNodes().getDataNodes().size() > 1) { + if (this.isMaster && dataNodeAdded && event.state().getNodes().getDataNodes().size() > 1) { if (logger.isDebugEnabled()) { logger.debug("data node was added, retrieving new cluster info"); } diff --git a/core/src/main/java/org/elasticsearch/cluster/MasterNodeChangePredicate.java b/core/src/main/java/org/elasticsearch/cluster/MasterNodeChangePredicate.java index 0ee5c891282..2fec24ec488 100644 --- a/core/src/main/java/org/elasticsearch/cluster/MasterNodeChangePredicate.java +++ b/core/src/main/java/org/elasticsearch/cluster/MasterNodeChangePredicate.java @@ -19,23 +19,32 @@ package org.elasticsearch.cluster; -import org.elasticsearch.cluster.service.ClusterServiceState; +import java.util.function.Predicate; -public enum MasterNodeChangePredicate implements ClusterStateObserver.ChangePredicate { - INSTANCE; +public final class MasterNodeChangePredicate { + + private MasterNodeChangePredicate() { - @Override - public boolean apply( - ClusterServiceState previousState, - ClusterServiceState newState) { - // checking if the masterNodeId changed is insufficient as the - // same master node might get re-elected after a disruption - return newState.getClusterState().nodes().getMasterNodeId() != null && - newState.getClusterState() != previousState.getClusterState(); } - @Override - public boolean apply(ClusterChangedEvent changedEvent) { - return changedEvent.nodesDelta().masterNodeChanged(); + /** + * builds a predicate that will accept a cluster state only if it was generated after the current has + * (re-)joined the master + */ + public static Predicate build(ClusterState currentState) { + final long currentVersion = currentState.version(); + final String currentMaster = currentState.nodes().getMasterNodeId(); + return newState -> { + final String newMaster = newState.nodes().getMasterNodeId(); + final boolean accept; + if (newMaster == null) { + accept = false; + } else if (newMaster.equals(currentMaster) == false){ + accept = true; + } else { + accept = newState.version() > currentVersion; + } + return accept; + }; } } diff --git a/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java b/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java index d7964f0c429..0cf124612d0 100644 --- a/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java +++ b/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java @@ -92,7 +92,7 @@ public class ShardStateAction extends AbstractComponent { } private void sendShardAction(final String actionName, final ClusterStateObserver observer, final ShardEntry shardEntry, final Listener listener) { - DiscoveryNode masterNode = observer.observedState().getClusterState().nodes().getMasterNode(); + DiscoveryNode masterNode = observer.observedState().nodes().getMasterNode(); if (masterNode == null) { logger.warn("{} no master known for action [{}] for shard entry [{}]", shardEntry.shardId, actionName, shardEntry); waitForNewMasterAndRetry(actionName, observer, shardEntry, listener); @@ -142,18 +142,27 @@ public class ShardStateAction extends AbstractComponent { */ public void remoteShardFailed(final ShardId shardId, String allocationId, long primaryTerm, final String message, @Nullable final Exception failure, Listener listener) { assert primaryTerm > 0L : "primary term should be strictly positive"; - shardFailed(shardId, allocationId, primaryTerm, message, failure, listener); + shardFailed(shardId, allocationId, primaryTerm, message, failure, listener, clusterService.state()); } /** * Send a shard failed request to the master node to update the cluster state when a shard on the local node failed. */ public void localShardFailed(final ShardRouting shardRouting, final String message, @Nullable final Exception failure, Listener listener) { - shardFailed(shardRouting.shardId(), shardRouting.allocationId().getId(), 0L, message, failure, listener); + localShardFailed(shardRouting, message, failure, listener, clusterService.state()); } - private void shardFailed(final ShardId shardId, String allocationId, long primaryTerm, final String message, @Nullable final Exception failure, Listener listener) { - ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext()); + /** + * Send a shard failed request to the master node to update the cluster state when a shard on the local node failed. + */ + public void localShardFailed(final ShardRouting shardRouting, final String message, @Nullable final Exception failure, Listener listener, + final ClusterState currentState) { + shardFailed(shardRouting.shardId(), shardRouting.allocationId().getId(), 0L, message, failure, listener, currentState); + } + + private void shardFailed(final ShardId shardId, String allocationId, long primaryTerm, final String message, + @Nullable final Exception failure, Listener listener, ClusterState currentState) { + ClusterStateObserver observer = new ClusterStateObserver(currentState, clusterService, null, logger, threadPool.getThreadContext()); ShardEntry shardEntry = new ShardEntry(shardId, allocationId, primaryTerm, message, failure); sendShardAction(SHARD_FAILED_ACTION_NAME, observer, shardEntry, listener); } @@ -180,7 +189,7 @@ public class ShardStateAction extends AbstractComponent { // we wait indefinitely for a new master assert false; } - }, MasterNodeChangePredicate.INSTANCE); + }, MasterNodeChangePredicate.build(observer.observedState())); } private static class ShardFailedTransportHandler implements TransportRequestHandler { @@ -342,7 +351,10 @@ public class ShardStateAction extends AbstractComponent { } public void shardStarted(final ShardRouting shardRouting, final String message, Listener listener) { - ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext()); + shardStarted(shardRouting, message, listener, clusterService.state()); + } + public void shardStarted(final ShardRouting shardRouting, final String message, Listener listener, ClusterState currentState) { + ClusterStateObserver observer = new ClusterStateObserver(currentState, clusterService, null, logger, threadPool.getThreadContext()); ShardEntry shardEntry = new ShardEntry(shardRouting.shardId(), shardRouting.allocationId().getId(), 0L, message, null); sendShardAction(SHARD_STARTED_ACTION_NAME, observer, shardEntry, listener); } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index b1ffccf6aeb..422492e396b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -71,6 +71,7 @@ import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.indices.IndexCreationException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.InvalidIndexNameException; +import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import org.elasticsearch.threadpool.ThreadPool; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -223,7 +224,8 @@ public class MetaDataCreateIndexService extends AbstractComponent { @Override public ClusterState execute(ClusterState currentState) throws Exception { Index createdIndex = null; - String removalReason = null; + String removalExtraInfo = null; + IndexRemovalReason removalReason = IndexRemovalReason.FAILURE; try { validate(request, currentState); @@ -356,7 +358,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { try { mapperService.merge(mappings, request.updateAllTypes()); } catch (MapperParsingException mpe) { - removalReason = "failed on parsing default mapping/mappings on index creation"; + removalExtraInfo = "failed on parsing default mapping/mappings on index creation"; throw mpe; } @@ -407,7 +409,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { try { indexMetaData = indexMetaDataBuilder.build(); } catch (Exception e) { - removalReason = "failed to build index metadata"; + removalExtraInfo = "failed to build index metadata"; throw e; } @@ -440,12 +442,13 @@ public class MetaDataCreateIndexService extends AbstractComponent { ClusterState.builder(updatedState).routingTable(routingTableBuilder.build()).build(), "index [" + request.index() + "] created"); } - removalReason = "cleaning up after validating index on master"; + removalExtraInfo = "cleaning up after validating index on master"; + removalReason = IndexRemovalReason.NO_LONGER_ASSIGNED; return updatedState; } finally { if (createdIndex != null) { // Index was already partially created - need to clean up - indicesService.removeIndex(createdIndex, removalReason != null ? removalReason : "failed to create index"); + indicesService.removeIndex(createdIndex, removalReason, removalExtraInfo); } } } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java index 0c0d3f576a5..c1de936d9c7 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java @@ -20,7 +20,6 @@ package org.elasticsearch.cluster.metadata; import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesClusterStateUpdateRequest; @@ -50,6 +49,7 @@ import java.util.Set; import java.util.function.Function; import static java.util.Collections.emptyList; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.NO_LONGER_ASSIGNED; /** * Service responsible for submitting add and remove aliases requests @@ -172,7 +172,7 @@ public class MetaDataIndexAliasesService extends AbstractComponent { return currentState; } finally { for (Index index : indicesToClose) { - indicesService.removeIndex(index, "created for alias processing"); + indicesService.removeIndex(index, NO_LONGER_ASSIGNED, "created for alias processing"); } } } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java index 08ba1dea67e..020a1d75231 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java @@ -19,7 +19,6 @@ package org.elasticsearch.cluster.metadata; import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.support.master.MasterNodeRequest; @@ -53,6 +52,8 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.NO_LONGER_ASSIGNED; + /** * Service responsible for submitting index templates updates */ @@ -225,7 +226,7 @@ public class MetaDataIndexTemplateService extends AbstractComponent { } finally { if (createdIndex != null) { - indicesService.removeIndex(createdIndex, " created for parsing template mapping"); + indicesService.removeIndex(createdIndex, NO_LONGER_ASSIGNED, " created for parsing template mapping"); } } } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index 15e384df3eb..4e9b114ff13 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -52,6 +52,9 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; + +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.NO_LONGER_ASSIGNED; + /** * Service responsible for submitting mapping changes */ @@ -158,7 +161,7 @@ public class MetaDataMappingService extends AbstractComponent { } } finally { if (removeIndex) { - indicesService.removeIndex(index, "created for mapping processing"); + indicesService.removeIndex(index, NO_LONGER_ASSIGNED, "created for mapping processing"); } } } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java index 2b937328106..6b19e2d4bf3 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java @@ -71,7 +71,7 @@ public class MetaDataUpdateSettingsService extends AbstractComponent implements IndexScopedSettings indexScopedSettings, IndicesService indicesService) { super(settings); this.clusterService = clusterService; - this.clusterService.add(this); + this.clusterService.addListener(this); this.allocationService = allocationService; this.indexScopedSettings = indexScopedSettings; this.indicesService = indicesService; 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 55c6750b825..4522dfcf98f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java @@ -133,7 +133,7 @@ public class DelayedAllocationService extends AbstractLifecycleComponent impleme this.threadPool = threadPool; this.clusterService = clusterService; this.allocationService = allocationService; - clusterService.addFirst(this); + clusterService.addListener(this); } @Override @@ -146,7 +146,7 @@ public class DelayedAllocationService extends AbstractLifecycleComponent impleme @Override protected void doClose() { - clusterService.remove(this); + clusterService.removeListener(this); removeTaskAndCancel(); } 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 6d26918d012..d213cea4d33 100644 --- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -27,6 +27,7 @@ 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.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; @@ -77,6 +78,7 @@ 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; @@ -87,6 +89,7 @@ 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; @@ -113,20 +116,22 @@ public class ClusterService extends AbstractLifecycleComponent { /** * Those 3 state listeners are changing infrequently - CopyOnWriteArrayList is just fine */ - private final Collection priorityClusterStateListeners = new CopyOnWriteArrayList<>(); - private final Collection clusterStateListeners = new CopyOnWriteArrayList<>(); - private final Collection lastClusterStateListeners = new CopyOnWriteArrayList<>(); + private final Collection highPriorityStateAppliers = new CopyOnWriteArrayList<>(); + private final Collection normalPriorityStateAppliers = new CopyOnWriteArrayList<>(); + private final Collection lowPriorityStateAppliers = new CopyOnWriteArrayList<>(); final Map> updateTasksPerExecutor = new HashMap<>(); - // TODO this is rather frequently changing I guess a Synced Set would be better here and a dedicated remove API - private final Collection postAppliedListeners = new CopyOnWriteArrayList<>(); - private final Iterable preAppliedListeners = Iterables.concat(priorityClusterStateListeners, - clusterStateListeners, lastClusterStateListeners); + 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 AtomicReference state; private final ClusterBlocks.Builder initialBlocks; @@ -140,7 +145,7 @@ public class ClusterService extends AbstractLifecycleComponent { this.clusterSettings = clusterSettings; this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); // will be replaced on doStart. - this.state = new AtomicReference<>(new ClusterServiceState(ClusterState.builder(clusterName).build(), ClusterStateStatus.UNKNOWN)); + this.state = new AtomicReference<>(ClusterState.builder(clusterName).build()); this.clusterSettings.addSettingsUpdateConsumer(CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, this::setSlowTaskLoggingThreshold); @@ -161,43 +166,15 @@ public class ClusterService extends AbstractLifecycleComponent { } public synchronized void setLocalNode(DiscoveryNode localNode) { - assert clusterServiceState().getClusterState().nodes().getLocalNodeId() == null : "local node is already set"; - updateState(css -> { - ClusterState clusterState = css.getClusterState(); + assert state().nodes().getLocalNodeId() == null : "local node is already set"; + updateState(clusterState -> { DiscoveryNodes nodes = DiscoveryNodes.builder(clusterState.nodes()).add(localNode).localNodeId(localNode.getId()).build(); - return new ClusterServiceState(ClusterState.builder(clusterState).nodes(nodes).build(), css.getClusterStateStatus()); + return ClusterState.builder(clusterState).nodes(nodes).build(); }); } - private void updateState(UnaryOperator updateFunction) { - this.state.getAndUpdate(oldClusterServiceState -> { - ClusterServiceState newClusterServiceState = updateFunction.apply(oldClusterServiceState); - assert validStateTransition(oldClusterServiceState, newClusterServiceState) : - "Invalid cluster service state transition from " + oldClusterServiceState + " to " + newClusterServiceState; - return newClusterServiceState; - }); - } - - private static boolean validStateTransition(ClusterServiceState oldClusterServiceState, ClusterServiceState newClusterServiceState) { - if (oldClusterServiceState == null || newClusterServiceState == null) { - return false; - } - ClusterStateStatus oldStatus = oldClusterServiceState.getClusterStateStatus(); - ClusterStateStatus newStatus = newClusterServiceState.getClusterStateStatus(); - // only go from UNKNOWN to UNKNOWN or BEING_APPLIED - if (oldStatus == ClusterStateStatus.UNKNOWN && newStatus == ClusterStateStatus.APPLIED) { - return false; - } - // only go from BEING_APPLIED to APPLIED - if (oldStatus == ClusterStateStatus.BEING_APPLIED && newStatus != ClusterStateStatus.APPLIED) { - return false; - } - // only go from APPLIED to BEING_APPLIED - if (oldStatus == ClusterStateStatus.APPLIED && newStatus != ClusterStateStatus.BEING_APPLIED) { - return false; - } - boolean identicalClusterState = oldClusterServiceState.getClusterState() == newClusterServiceState.getClusterState(); - return identicalClusterState == (oldStatus == ClusterStateStatus.BEING_APPLIED && newStatus == ClusterStateStatus.APPLIED); + private void updateState(UnaryOperator updateFunction) { + this.state.getAndUpdate(updateFunction); } public synchronized void setNodeConnectionsService(NodeConnectionsService nodeConnectionsService) { @@ -235,12 +212,10 @@ public class ClusterService extends AbstractLifecycleComponent { @Override protected synchronized void doStart() { Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting"); - Objects.requireNonNull(clusterServiceState().getClusterState().nodes().getLocalNode(), "please set the local node before starting"); + Objects.requireNonNull(state().nodes().getLocalNode(), "please set the local node before starting"); Objects.requireNonNull(nodeConnectionsService, "please set the node connection service before starting"); - add(localNodeMasterListeners); - updateState(css -> new ClusterServiceState( - ClusterState.builder(css.getClusterState()).blocks(initialBlocks).build(), - css.getClusterStateStatus())); + addListener(localNodeMasterListeners); + updateState(state -> ClusterState.builder(state).blocks(initialBlocks).build()); this.threadPoolExecutor = EsExecutors.newSinglePrioritizing(UPDATE_THREAD_NAME, daemonThreadFactory(settings, UPDATE_THREAD_NAME), threadPool.getThreadContext()); } @@ -258,12 +233,8 @@ public class ClusterService extends AbstractLifecycleComponent { } ThreadPool.terminate(threadPoolExecutor, 10, TimeUnit.SECONDS); // close timeout listeners that did not have an ongoing timeout - postAppliedListeners - .stream() - .filter(listener -> listener instanceof TimeoutClusterStateListener) - .map(listener -> (TimeoutClusterStateListener) listener) - .forEach(TimeoutClusterStateListener::onClose); - remove(localNodeMasterListeners); + timeoutClusterStateListeners.forEach(TimeoutClusterStateListener::onClose); + removeListener(localNodeMasterListeners); } @Override @@ -289,45 +260,59 @@ public class ClusterService extends AbstractLifecycleComponent { * The current cluster state. */ public ClusterState state() { - return clusterServiceState().getClusterState(); - } - - /** - * The current cluster service state comprising cluster state and cluster state status. - */ - public ClusterServiceState clusterServiceState() { + assert assertNotCalledFromClusterStateApplier("the applied cluster state is not yet available"); return this.state.get(); } /** - * Adds a priority listener for updated cluster states. + * Adds a high priority applier of updated cluster states. */ - public void addFirst(ClusterStateListener listener) { - priorityClusterStateListeners.add(listener); + public void addHighPriorityApplier(ClusterStateApplier applier) { + highPriorityStateAppliers.add(applier); } /** - * Adds last listener. + * Adds an applier which will be called after all high priority and normal appliers have been called. */ - public void addLast(ClusterStateListener listener) { - lastClusterStateListeners.add(listener); + public void addLowPriorityApplier(ClusterStateApplier applier) { + lowPriorityStateAppliers.add(applier); } /** - * Adds a listener for updated cluster states. + * Adds a applier of updated cluster states. */ - public void add(ClusterStateListener listener) { + 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 remove(ClusterStateListener listener) { + public void removeListener(ClusterStateListener listener) { clusterStateListeners.remove(listener); - priorityClusterStateListeners.remove(listener); - lastClusterStateListeners.remove(listener); - postAppliedListeners.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)) { @@ -340,25 +325,24 @@ public class ClusterService extends AbstractLifecycleComponent { /** * Add a listener for on/off local node master events */ - public void add(LocalNodeMasterListener listener) { + public void addLocalNodeMasterListener(LocalNodeMasterListener listener) { localNodeMasterListeners.add(listener); } /** * Remove the given listener for on/off local master events */ - public void remove(LocalNodeMasterListener listener) { + public void removeLocalNodeMasterListener(LocalNodeMasterListener listener) { localNodeMasterListeners.remove(listener); } /** - * Adds a cluster state listener that will timeout after the provided timeout, - * and is executed after the clusterstate has been successfully applied ie. is - * in state {@link ClusterStateStatus#APPLIED} - * NOTE: a {@code null} timeout means that the listener will never be removed - * automatically + * 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 remmoved on timeout. This is the responsibility of the caller. */ - public void add(@Nullable final TimeValue timeout, final TimeoutClusterStateListener listener) { + public void addTimeoutListener(@Nullable final TimeValue timeout, final TimeoutClusterStateListener listener) { if (lifecycle.stoppedOrClosed()) { listener.onClose(); return; @@ -373,7 +357,7 @@ public class ClusterService extends AbstractLifecycleComponent { notifyTimeout.future = threadPool.schedule(timeout, ThreadPool.Names.GENERIC, notifyTimeout); onGoingTimeouts.add(notifyTimeout); } - postAppliedListeners.add(listener); + timeoutClusterStateListeners.add(listener); listener.postAdded(); } }); @@ -572,6 +556,19 @@ public class ClusterService extends AbstractLifecycleComponent { return true; } + /** asserts that the current stack trace does NOT invlove a cluster state applier */ + private static boolean assertNotCalledFromClusterStateApplier(String reason) { + if (Thread.currentThread().getName().contains(UPDATE_THREAD_NAME)) { + for (StackTraceElement element: Thread.currentThread().getStackTrace()) { + if (element.getClassName().equals(ClusterService.class.getName()) + && element.getMethodName().equals("callClusterStateAppliers")) { + throw new AssertionError("should not be called by a cluster state applier. reason [" + reason + "]"); + } + } + } + return true; + } + public ClusterName getClusterName() { return clusterName; } @@ -596,8 +593,7 @@ public class ClusterService extends AbstractLifecycleComponent { } logger.debug("processing [{}]: execute", taskInputs.summary); - ClusterServiceState previousClusterServiceState = clusterServiceState(); - ClusterState previousClusterState = previousClusterServiceState.getClusterState(); + ClusterState previousClusterState = state(); if (!previousClusterState.nodes().isLocalNodeElectedMaster() && taskInputs.runOnlyOnMaster()) { logger.debug("failing [{}]: local node is no longer master", taskInputs.summary); @@ -606,7 +602,7 @@ public class ClusterService extends AbstractLifecycleComponent { } long startTimeNS = currentTimeInNanos(); - TaskOutputs taskOutputs = calculateTaskOutputs(taskInputs, previousClusterServiceState, startTimeNS); + TaskOutputs taskOutputs = calculateTaskOutputs(taskInputs, previousClusterState, startTimeNS); taskOutputs.notifyFailedTasks(); if (taskOutputs.clusterStateUnchanged()) { @@ -615,7 +611,7 @@ public class ClusterService extends AbstractLifecycleComponent { logger.debug("processing [{}]: took [{}] no change in cluster_state", taskInputs.summary, executionTime); warnAboutSlowTaskIfNeeded(executionTime, taskInputs.summary); } else { - ClusterState newClusterState = taskOutputs.newClusterServiceState.getClusterState(); + ClusterState newClusterState = taskOutputs.newClusterState; if (logger.isTraceEnabled()) { logger.trace("cluster state updated, source [{}]\n{}", taskInputs.summary, newClusterState); } else if (logger.isDebugEnabled()) { @@ -646,8 +642,7 @@ public class ClusterService extends AbstractLifecycleComponent { } } - public TaskOutputs calculateTaskOutputs(TaskInputs taskInputs, ClusterServiceState previousClusterServiceState, long startTimeNS) { - ClusterState previousClusterState = previousClusterServiceState.getClusterState(); + public TaskOutputs calculateTaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState, long startTimeNS) { BatchResult batchResult = executeTasks(taskInputs, startTimeNS, previousClusterState); ClusterState newClusterState = batchResult.resultingState; // extract those that are waiting for results @@ -662,9 +657,7 @@ public class ClusterService extends AbstractLifecycleComponent { } newClusterState = patchVersions(previousClusterState, newClusterState); - ClusterServiceState newClusterServiceState = new ClusterServiceState(newClusterState, ClusterStateStatus.BEING_APPLIED); - - return new TaskOutputs(taskInputs, previousClusterServiceState, newClusterServiceState, nonFailedTasks, + return new TaskOutputs(taskInputs, previousClusterState, newClusterState, nonFailedTasks, batchResult.executionResults); } @@ -728,8 +721,8 @@ public class ClusterService extends AbstractLifecycleComponent { } private void publishAndApplyChanges(TaskInputs taskInputs, TaskOutputs taskOutputs) { - ClusterState previousClusterState = taskOutputs.previousClusterServiceState.getClusterState(); - ClusterState newClusterState = taskOutputs.newClusterServiceState.getClusterState(); + ClusterState previousClusterState = taskOutputs.previousClusterState; + ClusterState newClusterState = taskOutputs.newClusterState; ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent(taskInputs.summary, newClusterState, previousClusterState); // new cluster state, notify all listeners @@ -767,9 +760,7 @@ public class ClusterService extends AbstractLifecycleComponent { } } - // update the current cluster state - updateState(css -> taskOutputs.newClusterServiceState); - logger.debug("set local cluster state to version {}", newClusterState.version()); + 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()) { @@ -779,27 +770,22 @@ public class ClusterService extends AbstractLifecycleComponent { } catch (Exception ex) { logger.warn("failed to apply cluster settings", ex); } - for (ClusterStateListener listener : preAppliedListeners) { - try { - logger.trace("calling [{}] with change to version [{}]", listener, newClusterState.version()); - listener.clusterChanged(clusterChangedEvent); - } catch (Exception ex) { - logger.warn("failed to notify ClusterStateListener", ex); - } - } + + logger.debug("set local cluster state to version {}", newClusterState.version()); + callClusterStateAppliers(newClusterState, clusterChangedEvent); nodeConnectionsService.disconnectFromNodes(clusterChangedEvent.nodesDelta().removedNodes()); - updateState(css -> new ClusterServiceState(css.getClusterState(), ClusterStateStatus.APPLIED)); + updateState(css -> newClusterState); - for (ClusterStateListener listener : postAppliedListeners) { + 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()) { @@ -826,6 +812,17 @@ public class ClusterService extends AbstractLifecycleComponent { } } + 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 */ @@ -854,17 +851,17 @@ public class ClusterService extends AbstractLifecycleComponent { */ class TaskOutputs { public final TaskInputs taskInputs; - public final ClusterServiceState previousClusterServiceState; - public final ClusterServiceState newClusterServiceState; + public final ClusterState previousClusterState; + public final ClusterState newClusterState; public final List nonFailedTasks; public final Map executionResults; - public TaskOutputs(TaskInputs taskInputs, ClusterServiceState previousClusterServiceState, - ClusterServiceState newClusterServiceState, List nonFailedTasks, + public TaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState, + ClusterState newClusterState, List nonFailedTasks, Map executionResults) { this.taskInputs = taskInputs; - this.previousClusterServiceState = previousClusterServiceState; - this.newClusterServiceState = newClusterServiceState; + this.previousClusterState = previousClusterState; + this.newClusterState = newClusterState; this.nonFailedTasks = nonFailedTasks; this.executionResults = executionResults; } @@ -907,7 +904,7 @@ public class ClusterService extends AbstractLifecycleComponent { } public boolean clusterStateUnchanged() { - return previousClusterServiceState.getClusterState() == newClusterServiceState.getClusterState(); + return previousClusterState == newClusterState; } public void notifyFailedTasks() { @@ -922,13 +919,12 @@ public class ClusterService extends AbstractLifecycleComponent { } public void notifySuccessfulTasksOnUnchangedClusterState() { - ClusterState clusterState = newClusterServiceState.getClusterState(); 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, clusterState, clusterState); + task.listener.clusterStateProcessed(task.source, newClusterState, newClusterState); }); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterServiceState.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterServiceState.java deleted file mode 100644 index 3002941b482..00000000000 --- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterServiceState.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.cluster.service; - -import org.elasticsearch.cluster.ClusterState; - -/** - * A simple immutable container class that comprises a cluster state and cluster state status. Used by {@link ClusterService} - * to provide a snapshot view on which cluster state is currently being applied / already applied. - */ -public class ClusterServiceState { - private final ClusterState clusterState; - private final ClusterStateStatus clusterStateStatus; - - public ClusterServiceState(ClusterState clusterState, ClusterStateStatus clusterStateStatus) { - this.clusterState = clusterState; - this.clusterStateStatus = clusterStateStatus; - } - - public ClusterState getClusterState() { - return clusterState; - } - - public ClusterStateStatus getClusterStateStatus() { - return clusterStateStatus; - } - - @Override - public String toString() { - return "version [" + clusterState.version() + "], status [" + clusterStateStatus + "]"; - } -} diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java b/core/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java index 58ba7bb177e..9f0d3576c4b 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java @@ -29,7 +29,6 @@ import org.elasticsearch.cluster.Diff; import org.elasticsearch.cluster.IncompatibleClusterStateVersionException; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.service.ClusterStateStatus; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.compress.Compressor; diff --git a/core/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java b/core/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java index 0c829e88182..acfcadb2f51 100644 --- a/core/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java +++ b/core/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java @@ -20,9 +20,12 @@ package org.elasticsearch.gateway; import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.metadata.IndexGraveyard; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; @@ -48,7 +51,7 @@ import static java.util.Collections.unmodifiableMap; * their state written on disk, but don't exists in the metadata of the cluster), and importing * them into the cluster. */ -public class DanglingIndicesState extends AbstractComponent { +public class DanglingIndicesState extends AbstractComponent implements ClusterStateListener { private final NodeEnvironment nodeEnv; private final MetaStateService metaStateService; @@ -58,11 +61,12 @@ public class DanglingIndicesState extends AbstractComponent { @Inject public DanglingIndicesState(Settings settings, NodeEnvironment nodeEnv, MetaStateService metaStateService, - LocalAllocateDangledIndices allocateDangledIndices) { + LocalAllocateDangledIndices allocateDangledIndices, ClusterService clusterService) { super(settings); this.nodeEnv = nodeEnv; this.metaStateService = metaStateService; this.allocateDangledIndices = allocateDangledIndices; + clusterService.addListener(this); } /** @@ -174,4 +178,11 @@ public class DanglingIndicesState extends AbstractComponent { logger.warn("failed to send allocate dangled", e); } } + + @Override + public void clusterChanged(ClusterChangedEvent event) { + if (event.state().blocks().disableStatePersistence() == false) { + processDanglingIndices(event.state().metaData()); + } + } } diff --git a/core/src/main/java/org/elasticsearch/gateway/Gateway.java b/core/src/main/java/org/elasticsearch/gateway/Gateway.java index 3a6bfa7aec1..0d562fa4b2b 100644 --- a/core/src/main/java/org/elasticsearch/gateway/Gateway.java +++ b/core/src/main/java/org/elasticsearch/gateway/Gateway.java @@ -25,7 +25,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; @@ -40,7 +40,7 @@ import java.util.Arrays; import java.util.Map; import java.util.function.Supplier; -public class Gateway extends AbstractComponent implements ClusterStateListener { +public class Gateway extends AbstractComponent implements ClusterStateApplier { private final ClusterService clusterService; @@ -60,7 +60,7 @@ public class Gateway extends AbstractComponent implements ClusterStateListener { this.metaState = metaState; this.listGatewayMetaState = listGatewayMetaState; this.minimumMasterNodesProvider = discovery::getMinimumMasterNodes; - clusterService.addLast(this); + clusterService.addLowPriorityApplier(this); } public void performStateRecovery(final GatewayStateRecoveredListener listener) throws GatewayException { @@ -176,10 +176,10 @@ public class Gateway extends AbstractComponent implements ClusterStateListener { } @Override - public void clusterChanged(final ClusterChangedEvent event) { + public void applyClusterState(final ClusterChangedEvent event) { // order is important, first metaState, and then shardsState // so dangling indices will be recorded - metaState.clusterChanged(event); + metaState.applyClusterState(event); } public interface GatewayStateRecoveredListener { diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java b/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java index 5ab3179579b..605fe20a33a 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java @@ -22,8 +22,6 @@ package org.elasticsearch.gateway; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.support.nodes.BaseNodeResponse; import org.elasticsearch.action.support.nodes.BaseNodesResponse; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingService; @@ -79,24 +77,21 @@ public class GatewayAllocator extends AbstractComponent { public void setReallocation(final ClusterService clusterService, final RoutingService routingService) { this.routingService = routingService; - clusterService.add(new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - boolean cleanCache = false; - DiscoveryNode localNode = event.state().nodes().getLocalNode(); - if (localNode != null) { - if (localNode.isMasterNode() && event.localNodeMaster() == false) { - cleanCache = true; - } - } else { + clusterService.addStateApplier(event -> { + boolean cleanCache = false; + DiscoveryNode localNode = event.state().nodes().getLocalNode(); + if (localNode != null) { + if (localNode.isMasterNode() && event.localNodeMaster() == false) { cleanCache = true; } - if (cleanCache) { - Releasables.close(asyncFetchStarted.values()); - asyncFetchStarted.clear(); - Releasables.close(asyncFetchStore.values()); - asyncFetchStore.clear(); - } + } else { + cleanCache = true; + } + if (cleanCache) { + Releasables.close(asyncFetchStarted.values()); + asyncFetchStarted.clear(); + Releasables.close(asyncFetchStore.values()); + asyncFetchStore.clear(); } }); } diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java b/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java index 98b39cd2c8a..153ca8b3a87 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java @@ -23,7 +23,7 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService; @@ -54,11 +54,10 @@ import java.util.Set; import static java.util.Collections.emptySet; import static java.util.Collections.unmodifiableSet; -public class GatewayMetaState extends AbstractComponent implements ClusterStateListener { +public class GatewayMetaState extends AbstractComponent implements ClusterStateApplier { private final NodeEnvironment nodeEnv; private final MetaStateService metaStateService; - private final DanglingIndicesState danglingIndicesState; @Nullable private volatile MetaData previousMetaData; @@ -67,13 +66,12 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL @Inject public GatewayMetaState(Settings settings, NodeEnvironment nodeEnv, MetaStateService metaStateService, - DanglingIndicesState danglingIndicesState, TransportNodesListGatewayMetaState nodesListGatewayMetaState, + TransportNodesListGatewayMetaState nodesListGatewayMetaState, MetaDataIndexUpgradeService metaDataIndexUpgradeService, MetaDataUpgrader metaDataUpgrader) throws Exception { super(settings); this.nodeEnv = nodeEnv; this.metaStateService = metaStateService; - this.danglingIndicesState = danglingIndicesState; nodesListGatewayMetaState.init(this); if (DiscoveryNode.isDataNode(settings)) { @@ -117,7 +115,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL } @Override - public void clusterChanged(ClusterChangedEvent event) { + public void applyClusterState(ClusterChangedEvent event) { final ClusterState state = event.state(); if (state.blocks().disableStatePersistence()) { @@ -181,7 +179,6 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL } } - danglingIndicesState.processDanglingIndices(newMetaData); if (success) { previousMetaData = newMetaData; previouslyWrittenIndices = unmodifiableSet(relevantIndices); diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayService.java b/core/src/main/java/org/elasticsearch/gateway/GatewayService.java index 2e351b2e6bf..c215585c45c 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayService.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayService.java @@ -20,7 +20,6 @@ package org.elasticsearch.gateway; import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -130,12 +129,13 @@ public class GatewayService extends AbstractLifecycleComponent implements Cluste @Override protected void doStart() { - clusterService.addLast(this); + // use post applied so that the state will be visible to the background recovery thread we spawn in performStateRecovery + clusterService.addListener(this); } @Override protected void doStop() { - clusterService.remove(this); + clusterService.removeListener(this); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java b/core/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java index 3b2cf5cbd07..90d8a205e8b 100644 --- a/core/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java +++ b/core/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java @@ -30,6 +30,7 @@ import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import java.util.ArrayList; import java.util.Collection; @@ -176,48 +177,24 @@ final class CompositeIndexEventListener implements IndexEventListener { } @Override - public void beforeIndexClosed(IndexService indexService) { + public void beforeIndexRemoved(IndexService indexService, IndexRemovalReason reason) { for (IndexEventListener listener : listeners) { try { - listener.beforeIndexClosed(indexService); + listener.beforeIndexRemoved(indexService, reason); } catch (Exception e) { - logger.warn("failed to invoke before index closed callback", e); + logger.warn("failed to invoke before index removed callback", e); throw e; } } } @Override - public void beforeIndexDeleted(IndexService indexService) { + public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRemovalReason reason) { for (IndexEventListener listener : listeners) { try { - listener.beforeIndexDeleted(indexService); + listener.afterIndexRemoved(index, indexSettings, reason); } catch (Exception e) { - logger.warn("failed to invoke before index deleted callback", e); - throw e; - } - } - } - - @Override - public void afterIndexDeleted(Index index, Settings indexSettings) { - for (IndexEventListener listener : listeners) { - try { - listener.afterIndexDeleted(index, indexSettings); - } catch (Exception e) { - logger.warn("failed to invoke after index deleted callback", e); - throw e; - } - } - } - - @Override - public void afterIndexClosed(Index index, Settings indexSettings) { - for (IndexEventListener listener : listeners) { - try { - listener.afterIndexClosed(index, indexSettings); - } catch (Exception e) { - logger.warn("failed to invoke after index closed callback", e); + logger.warn("failed to invoke after index removed callback", e); throw e; } } diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index dce125cbdf4..5ba2889f4bb 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -471,7 +471,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust shardId, indexSettings, indexCache.bitsetFilterCache(), indexFieldData, mapperService(), similarityService(), scriptService, queryRegistry, client, indexReader, - clusterService.state(), nowInMillis); } diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index 090a49aa7a6..6521369453c 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -19,11 +19,9 @@ package org.elasticsearch.index.query; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.util.SetOnce; import org.elasticsearch.client.Client; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcherSupplier; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.IndexSettings; @@ -33,9 +31,7 @@ import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.script.ScriptSettings; -import java.util.Collections; import java.util.function.LongSupplier; /** @@ -48,19 +44,17 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier { protected final IndicesQueriesRegistry indicesQueriesRegistry; protected final Client client; protected final IndexReader reader; - protected final ClusterState clusterState; protected final LongSupplier nowInMillis; public QueryRewriteContext(IndexSettings indexSettings, MapperService mapperService, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, Client client, IndexReader reader, - ClusterState clusterState, LongSupplier nowInMillis) { + LongSupplier nowInMillis) { this.mapperService = mapperService; this.scriptService = scriptService; this.indexSettings = indexSettings; this.indicesQueriesRegistry = indicesQueriesRegistry; this.client = client; this.reader = reader; - this.clusterState = clusterState; this.nowInMillis = nowInMillis; } @@ -98,13 +92,6 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier { return this.indexSettings.getParseFieldMatcher(); } - /** - * Returns the cluster state as is when the operation started. - */ - public ClusterState getClusterState() { - return clusterState; - } - /** * Returns a new {@link QueryParseContext} that wraps the provided parser, using the ParseFieldMatcher settings that * are configured in the index settings. The default script language will always default to Painless. diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index e71322eb0ad..1d5fc1bc9a1 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -29,7 +29,6 @@ import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; @@ -101,8 +100,8 @@ public class QueryShardContext extends QueryRewriteContext { public QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache, IndexFieldDataService indexFieldDataService, MapperService mapperService, SimilarityService similarityService, ScriptService scriptService, final IndicesQueriesRegistry indicesQueriesRegistry, Client client, - IndexReader reader, ClusterState clusterState, LongSupplier nowInMillis) { - super(indexSettings, mapperService, scriptService, indicesQueriesRegistry, client, reader, clusterState, nowInMillis); + IndexReader reader, LongSupplier nowInMillis) { + super(indexSettings, mapperService, scriptService, indicesQueriesRegistry, client, reader, nowInMillis); this.shardId = shardId; this.indexSettings = indexSettings; this.similarityService = similarityService; @@ -118,7 +117,7 @@ public class QueryShardContext extends QueryRewriteContext { public QueryShardContext(QueryShardContext source) { this(source.shardId, source.indexSettings, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService, source.similarityService, source.scriptService, source.indicesQueriesRegistry, source.client, - source.reader, source.clusterState, source.nowInMillis); + source.reader, source.nowInMillis); this.types = source.getTypes(); } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexEventListener.java b/core/src/main/java/org/elasticsearch/index/shard/IndexEventListener.java index f5c6dca7d2f..24ed98e9aff 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexEventListener.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexEventListener.java @@ -23,6 +23,8 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; /** * An index event listener is the primary extension point for plugins and build-in services @@ -103,31 +105,32 @@ public interface IndexEventListener { } - /** - * Called before the index shard gets created. - */ - default void beforeIndexShardCreated(ShardId shardId, Settings indexSettings) { - } - - /** * Called before the index get closed. * * @param indexService The index service + * @param reason the reason for index removal */ - default void beforeIndexClosed(IndexService indexService) { + default void beforeIndexRemoved(IndexService indexService, IndexRemovalReason reason) { } /** - * Called after the index has been closed. + * Called after the index has been removed. * * @param index The index + * @param reason the reason for index removal */ - default void afterIndexClosed(Index index, Settings indexSettings) { + default void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRemovalReason reason) { } + /** + * Called before the index shard gets created. + */ + default void beforeIndexShardCreated(ShardId shardId, Settings indexSettings) { + } + /** * Called before the index shard gets deleted from disk * Note: this method is only executed on the first attempt of deleting the shard. Retries are will not invoke @@ -149,28 +152,6 @@ public interface IndexEventListener { default void afterIndexShardDeleted(ShardId shardId, Settings indexSettings) { } - /** - * Called after the index has been deleted. - * This listener method is invoked after {@link #afterIndexClosed(org.elasticsearch.index.Index, org.elasticsearch.common.settings.Settings)} - * when an index is deleted - * - * @param index The index - */ - default void afterIndexDeleted(Index index, Settings indexSettings) { - - } - - /** - * Called before the index gets deleted. - * This listener method is invoked after - * {@link #beforeIndexClosed(org.elasticsearch.index.IndexService)} when an index is deleted - * - * @param indexService The index service - */ - default void beforeIndexDeleted(IndexService indexService) { - - } - /** * Called on the Master node only before the {@link IndexService} instances is created to simulate an index creation. * This happens right before the index and it's metadata is registered in the cluster state diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 45a72d5a8e7..fc6eac196d8 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1612,7 +1612,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl } }); } else { - final Exception e; + final RuntimeException e; if (numShards == -1) { e = new IndexNotFoundException(mergeSourceIndex); } else { @@ -1620,7 +1620,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl + " are started yet, expected " + numShards + " found " + startedShards.size() + " can't recover shard " + shardId()); } - recoveryListener.onRecoveryFailure(recoveryState, new RecoveryFailedException(recoveryState, null, e), true); + throw e; } break; default: diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index 664743efaef..a3361f6b2ed 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -232,9 +232,7 @@ public class IndicesService extends AbstractLifecycleComponent for (final Index index : indices) { indicesStopExecutor.execute(() -> { try { - removeIndex(index, "shutdown", false); - } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("failed to remove index on stop [{}]", index), e); + removeIndex(index, IndexRemovalReason.NO_LONGER_ASSIGNED, "shutdown"); } finally { latch.countDown(); } @@ -525,22 +523,8 @@ public class IndicesService extends AbstractLifecycleComponent return indexShard; } - /** - * Removes the given index from this service and releases all associated resources. Persistent parts of the index - * like the shards files, state and transaction logs are kept around in the case of a disaster recovery. - * @param index the index to remove - * @param reason the high level reason causing this removal - */ @Override - public void removeIndex(Index index, String reason) { - try { - removeIndex(index, reason, false); - } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("failed to remove index ({})", reason), e); - } - } - - private void removeIndex(Index index, String reason, boolean delete) { + public void removeIndex(final Index index, final IndexRemovalReason reason, final String extraInfo) { final String indexName = index.getName(); try { final IndexService indexService; @@ -558,22 +542,18 @@ public class IndicesService extends AbstractLifecycleComponent listener = indexService.getIndexEventListener(); } - listener.beforeIndexClosed(indexService); - if (delete) { - listener.beforeIndexDeleted(indexService); - } - logger.debug("{} closing index service (reason [{}])", index, reason); - indexService.close(reason, delete); - logger.debug("{} closed... (reason [{}])", index, reason); - listener.afterIndexClosed(indexService.index(), indexService.getIndexSettings().getSettings()); - if (delete) { - final IndexSettings indexSettings = indexService.getIndexSettings(); - listener.afterIndexDeleted(indexService.index(), indexSettings.getSettings()); + listener.beforeIndexRemoved(indexService, reason); + logger.debug("{} closing index service (reason [{}][{}])", index, reason, extraInfo); + indexService.close(extraInfo, reason == IndexRemovalReason.DELETED); + logger.debug("{} closed... (reason [{}][{}])", index, reason, extraInfo); + final IndexSettings indexSettings = indexService.getIndexSettings(); + listener.afterIndexRemoved(indexService.index(), indexSettings, reason); + if (reason == IndexRemovalReason.DELETED) { // now we are done - try to wipe data on disk if possible - deleteIndexStore(reason, indexService.index(), indexSettings); + deleteIndexStore(extraInfo, indexService.index(), indexSettings); } - } catch (IOException ex) { - throw new ElasticsearchException("failed to remove index " + index, ex); + } catch (Exception e) { + logger.warn((Supplier) () -> new ParameterizedMessage("failed to remove index {} ([{}][{}])", index, reason, extraInfo), e); } } @@ -613,27 +593,9 @@ public class IndicesService extends AbstractLifecycleComponent } } - /** - * Deletes the given index. Persistent parts of the index - * like the shards files, state and transaction logs are removed once all resources are released. - * - * Equivalent to {@link #removeIndex(Index, String)} but fires - * different lifecycle events to ensure pending resources of this index are immediately removed. - * @param index the index to delete - * @param reason the high level reason causing this delete - */ - @Override - public void deleteIndex(Index index, String reason) { - try { - removeIndex(index, reason, true); - } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("failed to delete index ({})", reason), e); - } - } - /** * Deletes an index that is not assigned to this node. This method cleans up all disk folders relating to the index - * but does not deal with in-memory structures. For those call {@link #deleteIndex(Index, String)} + * but does not deal with in-memory structures. For those call {@link #removeIndex(Index, IndexRemovalReason, String)} */ @Override public void deleteUnassignedIndex(String reason, IndexMetaData metaData, ClusterState clusterState) { diff --git a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index aaecba91f3e..a49c2a97cb0 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -26,7 +26,7 @@ import org.apache.lucene.store.LockObtainFailedException; import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.action.index.NodeMappingRefreshAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -68,7 +68,6 @@ import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.search.SearchService; -import org.elasticsearch.snapshots.RestoreService; import org.elasticsearch.snapshots.SnapshotShardsService; import org.elasticsearch.threadpool.ThreadPool; @@ -86,7 +85,12 @@ import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.stream.Collectors; -public class IndicesClusterStateService extends AbstractLifecycleComponent implements ClusterStateListener { +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.CLOSED; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.FAILURE; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.NO_LONGER_ASSIGNED; + +public class IndicesClusterStateService extends AbstractLifecycleComponent implements ClusterStateApplier { final AllocatedIndices> indicesService; private final ClusterService clusterService; @@ -102,7 +106,6 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple // a list of shards that failed during recovery // we keep track of these shards in order to prevent repeated recovery of these shards on each cluster state update final ConcurrentMap failedShardsCache = ConcurrentCollections.newConcurrentMap(); - private final RestoreService restoreService; private final RepositoriesService repositoriesService; private final FailedShardHandler failedShardHandler = new FailedShardHandler(); @@ -115,13 +118,13 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple ThreadPool threadPool, PeerRecoveryTargetService recoveryTargetService, ShardStateAction shardStateAction, NodeMappingRefreshAction nodeMappingRefreshAction, - RepositoriesService repositoriesService, RestoreService restoreService, + RepositoriesService repositoriesService, SearchService searchService, SyncedFlushService syncedFlushService, PeerRecoverySourceService peerRecoverySourceService, SnapshotShardsService snapshotShardsService, GlobalCheckpointSyncAction globalCheckpointSyncAction) { - this(settings, (AllocatedIndices>) indicesService, + this(settings, indicesService, clusterService, threadPool, recoveryTargetService, shardStateAction, - nodeMappingRefreshAction, repositoriesService, restoreService, searchService, syncedFlushService, peerRecoverySourceService, + nodeMappingRefreshAction, repositoriesService, searchService, syncedFlushService, peerRecoverySourceService, snapshotShardsService, globalCheckpointSyncAction::updateCheckpointForShard); } @@ -132,7 +135,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple ThreadPool threadPool, PeerRecoveryTargetService recoveryTargetService, ShardStateAction shardStateAction, NodeMappingRefreshAction nodeMappingRefreshAction, - RepositoriesService repositoriesService, RestoreService restoreService, + RepositoriesService repositoriesService, SearchService searchService, SyncedFlushService syncedFlushService, PeerRecoverySourceService peerRecoverySourceService, SnapshotShardsService snapshotShardsService, Consumer globalCheckpointSyncer) { @@ -145,7 +148,6 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple this.recoveryTargetService = recoveryTargetService; this.shardStateAction = shardStateAction; this.nodeMappingRefreshAction = nodeMappingRefreshAction; - this.restoreService = restoreService; this.repositoriesService = repositoriesService; this.sendRefreshMapping = this.settings.getAsBoolean("indices.cluster.send_refresh_mapping", true); this.globalCheckpointSyncer = globalCheckpointSyncer; @@ -155,14 +157,14 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple protected void doStart() { // Doesn't make sense to manage shards on non-master and non-data nodes if (DiscoveryNode.isDataNode(settings) || DiscoveryNode.isMasterNode(settings)) { - clusterService.addFirst(this); + clusterService.addHighPriorityApplier(this); } } @Override protected void doStop() { if (DiscoveryNode.isDataNode(settings) || DiscoveryNode.isMasterNode(settings)) { - clusterService.remove(this); + clusterService.removeApplier(this); } } @@ -171,7 +173,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple } @Override - public synchronized void clusterChanged(final ClusterChangedEvent event) { + public synchronized void applyClusterState(final ClusterChangedEvent event) { if (!lifecycle.started()) { return; } @@ -183,7 +185,8 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple // TODO: feels hacky, a block disables state persistence, and then we clean the allocated shards, maybe another flag in blocks? if (state.blocks().disableStatePersistence()) { for (AllocatedIndex indexService : indicesService) { - indicesService.removeIndex(indexService.index(), "cleaning index (disabled block persistence)"); // also cleans shards + indicesService.removeIndex(indexService.index(), NO_LONGER_ASSIGNED, + "cleaning index (disabled block persistence)"); // also cleans shards } return; } @@ -231,7 +234,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple if (masterNode != null) { // TODO: can we remove this? Is resending shard failures the responsibility of shardStateAction? String message = "master " + masterNode + " has not removed previously failed shard. resending shard failure"; logger.trace("[{}] re-sending failed shard [{}], reason [{}]", matchedRouting.shardId(), matchedRouting, message); - shardStateAction.localShardFailed(matchedRouting, message, null, SHARD_STATE_ACTION_LISTENER); + shardStateAction.localShardFailed(matchedRouting, message, null, SHARD_STATE_ACTION_LISTENER, state); } } } @@ -256,7 +259,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple final IndexSettings indexSettings; if (indexService != null) { indexSettings = indexService.getIndexSettings(); - indicesService.deleteIndex(index, "index no longer part of the metadata"); + indicesService.removeIndex(index, DELETED, "index no longer part of the metadata"); } else if (previousState.metaData().hasIndex(index.getName())) { // The deleted index was part of the previous cluster state, but not loaded on the local node final IndexMetaData metaData = previousState.metaData().index(index); @@ -330,11 +333,14 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple // to remove the in-memory structures for the index and not delete the // contents on disk because the index will later be re-imported as a // dangling index - assert state.metaData().index(index) != null || event.isNewCluster() : + final IndexMetaData indexMetaData = state.metaData().index(index); + assert indexMetaData != null || event.isNewCluster() : "index " + index + " does not exist in the cluster state, it should either " + "have been deleted or the cluster must be new"; - logger.debug("{} removing index, no shards allocated", index); - indicesService.removeIndex(index, "removing index (no shards allocated)"); + final AllocatedIndices.IndexRemovalReason reason = + indexMetaData != null && indexMetaData.getState() == IndexMetaData.State.CLOSE ? CLOSED : NO_LONGER_ASSIGNED; + logger.debug("{} removing index, [{}]", index, reason); + indicesService.removeIndex(index, reason, "removing index (no shards allocated)"); } } } @@ -355,7 +361,8 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple failedShardsCache.containsKey(shardId) == false && indicesService.getShardOrNull(shardId) == null) { // the master thinks we are active, but we don't have this shard at all, mark it as failed - sendFailShard(shardRouting, "master marked shard as active, but shard has not been created, mark shard as failed", null); + sendFailShard(shardRouting, "master marked shard as active, but shard has not been created, mark shard as failed", null, + state); } } } @@ -451,10 +458,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple failShardReason = "failed to create index"; } else { failShardReason = "failed to update mapping for index"; - indicesService.removeIndex(index, "removing index (mapping update failed)"); + indicesService.removeIndex(index, FAILURE, "removing index (mapping update failed)"); } for (ShardRouting shardRouting : entry.getValue()) { - sendFailShard(shardRouting, failShardReason, e); + sendFailShard(shardRouting, failShardReason, e, state); } } } @@ -480,14 +487,14 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple ); } } catch (Exception e) { - indicesService.removeIndex(indexService.index(), "removing index (mapping update failed)"); + indicesService.removeIndex(indexService.index(), FAILURE, "removing index (mapping update failed)"); // fail shards that would be created or updated by createOrUpdateShards RoutingNode localRoutingNode = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); if (localRoutingNode != null) { for (final ShardRouting shardRouting : localRoutingNode) { if (shardRouting.index().equals(index) && failedShardsCache.containsKey(shardRouting.shardId()) == false) { - sendFailShard(shardRouting, "failed to update mapping for index", e); + sendFailShard(shardRouting, "failed to update mapping for index", e, state); } } } @@ -513,15 +520,15 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple Shard shard = indexService.getShardOrNull(shardId.id()); if (shard == null) { assert shardRouting.initializing() : shardRouting + " should have been removed by failMissingShards"; - createShard(nodes, routingTable, shardRouting); + createShard(nodes, routingTable, shardRouting, state); } else { - updateShard(nodes, shardRouting, shard, routingTable); + updateShard(nodes, shardRouting, shard, routingTable, state); } } } } - private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardRouting shardRouting) { + private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardRouting shardRouting, ClusterState state) { assert shardRouting.initializing() : "only allow shard creation for initializing shard but was " + shardRouting; DiscoveryNode sourceNode = null; @@ -539,11 +546,12 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple indicesService.createShard(shardRouting, recoveryState, recoveryTargetService, new RecoveryListener(shardRouting), repositoriesService, failedShardHandler); } catch (Exception e) { - failAndRemoveShard(shardRouting, true, "failed to create shard", e); + failAndRemoveShard(shardRouting, true, "failed to create shard", e, state); } } - private void updateShard(DiscoveryNodes nodes, ShardRouting shardRouting, Shard shard, RoutingTable routingTable) { + private void updateShard(DiscoveryNodes nodes, ShardRouting shardRouting, Shard shard, RoutingTable routingTable, + ClusterState clusterState) { final ShardRouting currentRoutingEntry = shard.routingEntry(); assert currentRoutingEntry.isSameAllocation(shardRouting) : "local shard has a different allocation id but wasn't cleaning by removeShards. " @@ -560,7 +568,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple shard.updateAllocationIdsFromMaster(activeIds, initializingIds); } } catch (Exception e) { - failAndRemoveShard(shardRouting, true, "failed updating shard routing entry", e); + failAndRemoveShard(shardRouting, true, "failed updating shard routing entry", e, clusterState); return; } @@ -576,7 +584,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple if (nodes.getMasterNode() != null) { shardStateAction.shardStarted(shardRouting, "master " + nodes.getMasterNode() + " marked shard as initializing, but shard state is [" + state + "], mark shard as started", - SHARD_STATE_ACTION_LISTENER); + SHARD_STATE_ACTION_LISTENER, clusterState); } } } @@ -632,10 +640,11 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple } private synchronized void handleRecoveryFailure(ShardRouting shardRouting, boolean sendShardFailure, Exception failure) { - failAndRemoveShard(shardRouting, sendShardFailure, "failed recovery", failure); + failAndRemoveShard(shardRouting, sendShardFailure, "failed recovery", failure, clusterService.state()); } - private void failAndRemoveShard(ShardRouting shardRouting, boolean sendShardFailure, String message, @Nullable Exception failure) { + private void failAndRemoveShard(ShardRouting shardRouting, boolean sendShardFailure, String message, @Nullable Exception failure, + ClusterState state) { try { AllocatedIndex indexService = indicesService.indexService(shardRouting.shardId().getIndex()); if (indexService != null) { @@ -654,17 +663,17 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple inner); } if (sendShardFailure) { - sendFailShard(shardRouting, message, failure); + sendFailShard(shardRouting, message, failure, state); } } - private void sendFailShard(ShardRouting shardRouting, String message, @Nullable Exception failure) { + private void sendFailShard(ShardRouting shardRouting, String message, @Nullable Exception failure, ClusterState state) { try { logger.warn( (Supplier) () -> new ParameterizedMessage( "[{}] marking and sending shard failed due to [{}]", shardRouting.shardId(), message), failure); failedShardsCache.put(shardRouting.shardId(), shardRouting); - shardStateAction.localShardFailed(shardRouting, message, failure, SHARD_STATE_ACTION_LISTENER); + shardStateAction.localShardFailed(shardRouting, message, failure, SHARD_STATE_ACTION_LISTENER, state); } catch (Exception inner) { if (failure != null) inner.addSuppressed(failure); logger.warn( @@ -683,7 +692,8 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple final ShardRouting shardRouting = shardFailure.routing; threadPool.generic().execute(() -> { synchronized (IndicesClusterStateService.this) { - failAndRemoveShard(shardRouting, true, "shard failure, reason [" + shardFailure.reason + "]", shardFailure.cause); + failAndRemoveShard(shardRouting, true, "shard failure, reason [" + shardFailure.reason + "]", shardFailure.cause, + clusterService.state()); } }); } @@ -783,20 +793,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple */ IndexMetaData verifyIndexIsDeleted(Index index, ClusterState clusterState); - /** - * Deletes the given index. Persistent parts of the index - * like the shards files, state and transaction logs are removed once all resources are released. - * - * Equivalent to {@link #removeIndex(Index, String)} but fires - * different lifecycle events to ensure pending resources of this index are immediately removed. - * @param index the index to delete - * @param reason the high level reason causing this delete - */ - void deleteIndex(Index index, String reason); /** * Deletes an index that is not assigned to this node. This method cleans up all disk folders relating to the index - * but does not deal with in-memory structures. For those call {@link #deleteIndex(Index, String)} + * but does not deal with in-memory structures. For those call {@link #removeIndex(Index, IndexRemovalReason, String)} */ void deleteUnassignedIndex(String reason, IndexMetaData metaData, ClusterState clusterState); @@ -804,9 +804,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple * Removes the given index from this service and releases all associated resources. Persistent parts of the index * like the shards files, state and transaction logs are kept around in the case of a disaster recovery. * @param index the index to remove - * @param reason the high level reason causing this removal + * @param reason the reason to remove the index + * @param extraInfo extra information that will be used for logging and reporting */ - void removeIndex(Index index, String reason); + void removeIndex(Index index, IndexRemovalReason reason, String extraInfo); /** * Returns an IndexService for the specified index if exists otherwise returns null. @@ -833,5 +834,32 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple void processPendingDeletes(Index index, IndexSettings indexSettings, TimeValue timeValue) throws IOException, InterruptedException, ShardLockObtainFailedException; + + enum IndexRemovalReason { + /** + * Shard of this index were previously assigned to this node but all shards have been relocated. + * The index should be removed and all associated resources released. Persistent parts of the index + * like the shards files, state and transaction logs are kept around in the case of a disaster recovery. + */ + NO_LONGER_ASSIGNED, + /** + * The index is deleted. Persistent parts of the index like the shards files, state and transaction logs are removed once + * all resources are released. + */ + DELETED, + + /** + * The index have been closed. The index should be removed and all associated resources released. Persistent parts of the index + * like the shards files, state and transaction logs are kept around in the case of a disaster recovery. + */ + CLOSED, + + /** + * Something around index management has failed and the index should be removed. + * Persistent parts of the index like the shards files, state and transaction logs are kept around in the + * case of a disaster recovery. + */ + FAILURE + } } } diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index e5f8b531f67..84f35ebca43 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -31,7 +31,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.cluster.service.ClusterServiceState; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; @@ -400,7 +399,7 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde private void waitForClusterState(long clusterStateVersion) { ClusterStateObserver observer = new ClusterStateObserver(clusterService, TimeValue.timeValueMinutes(5), logger, threadPool.getThreadContext()); - final ClusterState clusterState = observer.observedState().getClusterState(); + final ClusterState clusterState = observer.observedState(); if (clusterState.getVersion() >= clusterStateVersion) { logger.trace("node has cluster state with version higher than {} (current: {})", clusterStateVersion, clusterState.getVersion()); @@ -424,23 +423,17 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde public void onTimeout(TimeValue timeout) { future.onFailure(new IllegalStateException("cluster state never updated to version " + clusterStateVersion)); } - }, new ClusterStateObserver.ValidationPredicate() { - - @Override - protected boolean validate(ClusterServiceState newState) { - return newState.getClusterState().getVersion() >= clusterStateVersion; - } - }); + }, newState -> newState.getVersion() >= clusterStateVersion); try { future.get(); logger.trace("successfully waited for cluster state with version {} (current: {})", clusterStateVersion, - observer.observedState().getClusterState().getVersion()); + observer.observedState().getVersion()); } catch (Exception e) { logger.debug( (Supplier) () -> new ParameterizedMessage( "failed waiting for cluster state with version {} (current: {})", clusterStateVersion, - observer.observedState().getClusterState().getVersion()), + observer.observedState().getVersion()), e); throw ExceptionsHelper.convertToRuntime(e); } 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 324903eb43a..589d8348981 100644 --- a/core/src/main/java/org/elasticsearch/indices/store/IndicesStore.java +++ b/core/src/main/java/org/elasticsearch/indices/store/IndicesStore.java @@ -35,7 +35,6 @@ import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.cluster.service.ClusterServiceState; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; @@ -102,14 +101,17 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe this.deleteShardTimeout = INDICES_STORE_DELETE_SHARD_TIMEOUT.get(settings); // Doesn't make sense to delete shards on non-data nodes if (DiscoveryNode.isDataNode(settings)) { - clusterService.add(this); + // we double check nothing has changed when responses come back from other nodes. + // it's easier to do that check when the current cluster state is visible. + // also it's good in general to let things settle down + clusterService.addListener(this); } } @Override public void close() { if (DiscoveryNode.isDataNode(settings)) { - clusterService.remove(this); + clusterService.removeListener(this); } } @@ -358,16 +360,13 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe logger.error((Supplier) () -> new ParameterizedMessage("failed send response for shard active while trying to delete shard {} - shard will probably not be removed", request.shardId), e); } } - }, new ClusterStateObserver.ValidationPredicate() { - @Override - protected boolean validate(ClusterServiceState newState) { - // the shard is not there in which case we want to send back a false (shard is not active), so the cluster state listener must be notified - // or the shard is active in which case we want to send back that the shard is active - // here we could also evaluate the cluster state and get the information from there. we - // don't do it because we would have to write another method for this that would have the same effect - IndexShard indexShard = getShard(request); - return indexShard == null || shardActive(indexShard); - } + }, newState -> { + // the shard is not there in which case we want to send back a false (shard is not active), so the cluster state listener must be notified + // or the shard is active in which case we want to send back that the shard is active + // here we could also evaluate the cluster state and get the information from there. we + // don't do it because we would have to write another method for this that would have the same effect + IndexShard currentShard = getShard(request); + return currentShard == null || shardActive(currentShard); }); } } diff --git a/core/src/main/java/org/elasticsearch/ingest/PipelineExecutionService.java b/core/src/main/java/org/elasticsearch/ingest/PipelineExecutionService.java index 0ca89ea37b9..c1b46e49567 100644 --- a/core/src/main/java/org/elasticsearch/ingest/PipelineExecutionService.java +++ b/core/src/main/java/org/elasticsearch/ingest/PipelineExecutionService.java @@ -22,7 +22,7 @@ package org.elasticsearch.ingest; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.common.Strings; import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.metrics.MeanMetric; @@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.Consumer; -public class PipelineExecutionService implements ClusterStateListener { +public class PipelineExecutionService implements ClusterStateApplier { private final PipelineStore store; private final ThreadPool threadPool; @@ -112,7 +112,7 @@ public class PipelineExecutionService implements ClusterStateListener { } @Override - public void clusterChanged(ClusterChangedEvent event) { + public void applyClusterState(ClusterChangedEvent event) { IngestMetadata ingestMetadata = event.state().getMetaData().custom(IngestMetadata.TYPE); if (ingestMetadata != null) { updatePipelineStats(ingestMetadata); diff --git a/core/src/main/java/org/elasticsearch/ingest/PipelineStore.java b/core/src/main/java/org/elasticsearch/ingest/PipelineStore.java index 94850674e75..1e938581b8c 100644 --- a/core/src/main/java/org/elasticsearch/ingest/PipelineStore.java +++ b/core/src/main/java/org/elasticsearch/ingest/PipelineStore.java @@ -19,13 +19,6 @@ package org.elasticsearch.ingest; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; - import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ResourceNotFoundException; @@ -36,7 +29,7 @@ import org.elasticsearch.action.ingest.WritePipelineResponse; import org.elasticsearch.cluster.AckedClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; @@ -45,7 +38,14 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; -public class PipelineStore extends AbstractComponent implements ClusterStateListener { +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class PipelineStore extends AbstractComponent implements ClusterStateApplier { private final Pipeline.Factory factory = new Pipeline.Factory(); private final Map processorFactories; @@ -62,7 +62,7 @@ public class PipelineStore extends AbstractComponent implements ClusterStateList } @Override - public void clusterChanged(ClusterChangedEvent event) { + public void applyClusterState(ClusterChangedEvent event) { innerUpdatePipelines(event.previousState(), event.state()); } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 8756e09e8d8..ec64c0f710a 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -40,7 +40,6 @@ import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.InternalClusterInfoService; -import org.elasticsearch.cluster.MasterNodeChangePredicate; import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.MetaData; @@ -322,7 +321,7 @@ public class Node implements Closeable { final NetworkService networkService = new NetworkService(settings, getCustomNameResolvers(pluginsService.filterPlugins(DiscoveryPlugin.class))); final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool); - clusterService.add(scriptModule.getScriptService()); + clusterService.addListener(scriptModule.getScriptService()); resourcesToClose.add(clusterService); final TribeService tribeService = new TribeService(settings, clusterService, nodeId, s -> newTribeClientNode(s, classpathPlugins)); @@ -578,7 +577,7 @@ public class Node implements Closeable { // playing nice with the life cycle interfaces clusterService.setLocalNode(localNode); transportService.setLocalNode(localNode); - clusterService.add(transportService.getTaskManager()); + clusterService.addStateApplier(transportService.getTaskManager()); clusterService.start(); @@ -591,7 +590,7 @@ public class Node implements Closeable { if (initialStateTimeout.millis() > 0) { final ThreadPool thread = injector.getInstance(ThreadPool.class); ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, thread.getThreadContext()); - if (observer.observedState().getClusterState().nodes().getMasterNodeId() == null) { + if (observer.observedState().nodes().getMasterNodeId() == null) { logger.debug("waiting to join the cluster. timeout [{}]", initialStateTimeout); final CountDownLatch latch = new CountDownLatch(1); observer.waitForNextChange(new ClusterStateObserver.Listener() { @@ -609,7 +608,7 @@ public class Node implements Closeable { initialStateTimeout); latch.countDown(); } - }, MasterNodeChangePredicate.INSTANCE, initialStateTimeout); + }, state -> state.nodes().getMasterNodeId() != null, initialStateTimeout); try { latch.await(); diff --git a/core/src/main/java/org/elasticsearch/node/service/NodeService.java b/core/src/main/java/org/elasticsearch/node/service/NodeService.java index 36a399321e1..7d9a148b271 100644 --- a/core/src/main/java/org/elasticsearch/node/service/NodeService.java +++ b/core/src/main/java/org/elasticsearch/node/service/NodeService.java @@ -78,8 +78,8 @@ public class NodeService extends AbstractComponent implements Closeable { this.ingestService = ingestService; this.settingsFilter = settingsFilter; this.scriptService = scriptService; - clusterService.add(ingestService.getPipelineStore()); - clusterService.add(ingestService.getPipelineExecutionService()); + clusterService.addStateApplier(ingestService.getPipelineStore()); + clusterService.addStateApplier(ingestService.getPipelineExecutionService()); } public NodeInfo info(boolean settings, boolean os, boolean process, boolean jvm, boolean threadPool, diff --git a/core/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/core/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index e5951d48a00..300ba811a1b 100644 --- a/core/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/core/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -25,7 +25,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.AckedClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.ack.ClusterStateUpdateRequest; import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse; import org.elasticsearch.cluster.metadata.MetaData; @@ -53,7 +53,7 @@ import java.util.stream.Collectors; /** * Service responsible for maintaining and providing access to snapshot repositories on nodes. */ -public class RepositoriesService extends AbstractComponent implements ClusterStateListener { +public class RepositoriesService extends AbstractComponent implements ClusterStateApplier { private final Map typesRegistry; @@ -72,7 +72,7 @@ public class RepositoriesService extends AbstractComponent implements ClusterSta // Doesn't make sense to maintain repositories on non-master and non-data nodes // Nothing happens there anyway if (DiscoveryNode.isDataNode(settings) || DiscoveryNode.isMasterNode(settings)) { - clusterService.add(this); + clusterService.addStateApplier(this); } this.verifyAction = new VerifyNodeRepositoryAction(settings, transportService, clusterService, this); } @@ -253,7 +253,7 @@ public class RepositoriesService extends AbstractComponent implements ClusterSta * @param event cluster changed event */ @Override - public void clusterChanged(ClusterChangedEvent event) { + public void applyClusterState(ClusterChangedEvent event) { try { RepositoriesMetaData oldMetaData = event.previousState().getMetaData().custom(RepositoriesMetaData.TYPE); RepositoriesMetaData newMetaData = event.state().getMetaData().custom(RepositoriesMetaData.TYPE); diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 55e4f100e76..fd2779d585d 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -27,7 +27,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.search.SearchTask; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseFieldMatcher; @@ -35,13 +34,13 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.lucene.Lucene; 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.BigArrays; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.ConcurrentMapLong; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.InnerHitBuilder; import org.elasticsearch.index.query.QueryShardContext; @@ -49,6 +48,7 @@ import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.SearchOperationListener; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.SearchScript; @@ -183,23 +183,16 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv } @Override - public void afterIndexClosed(Index index, Settings indexSettings) { - // once an index is closed we can just clean up all the pending search context information + public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRemovalReason reason) { + // once an index is removed due to deletion or closing, we can just clean up all the pending search context information + // if we then close all the contexts we can get some search failures along the way which are not expected. + // it's fine to keep the contexts open if the index is still "alive" + // unfortunately we don't have a clear way to signal today why an index is closed. // to release memory and let references to the filesystem go etc. - IndexMetaData idxMeta = SearchService.this.clusterService.state().metaData().index(index); - if (idxMeta != null && idxMeta.getState() == IndexMetaData.State.CLOSE) { - // we need to check if it's really closed - // since sometimes due to a relocation we already closed the shard and that causes the index to be closed - // if we then close all the contexts we can get some search failures along the way which are not expected. - // it's fine to keep the contexts open if the index is still "alive" - // unfortunately we don't have a clear way to signal today why an index is closed. - afterIndexDeleted(index, indexSettings); + if (reason == IndexRemovalReason.DELETED || reason == IndexRemovalReason.CLOSED) { + freeAllContextForIndex(index); } - } - @Override - public void afterIndexDeleted(Index index, Settings indexSettings) { - freeAllContextForIndex(index); } protected void putContext(SearchContext context) { diff --git a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 7ef62ef2db5..7f84f2d66c8 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -30,7 +30,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; @@ -83,7 +83,6 @@ import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; -import static java.util.Collections.min; import static java.util.Collections.unmodifiableSet; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_CREATION_DATE; @@ -115,7 +114,7 @@ import static org.elasticsearch.common.util.set.Sets.newHashSet; * which removes {@link RestoreInProgress} when all shards are completed. In case of * restore failure a normal recovery fail-over process kicks in. */ -public class RestoreService extends AbstractComponent implements ClusterStateListener { +public class RestoreService extends AbstractComponent implements ClusterStateApplier { private static final Set UNMODIFIABLE_SETTINGS = unmodifiableSet(newHashSet( SETTING_NUMBER_OF_SHARDS, @@ -160,7 +159,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis this.allocationService = allocationService; this.createIndexService = createIndexService; this.metaDataIndexUpgradeService = metaDataIndexUpgradeService; - clusterService.add(this); + clusterService.addStateApplier(this); this.clusterSettings = clusterSettings; this.cleanRestoreStateTaskExecutor = new CleanRestoreStateTaskExecutor(logger); } @@ -793,7 +792,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis } @Override - public void clusterChanged(ClusterChangedEvent event) { + public void applyClusterState(ClusterChangedEvent event) { try { if (event.localNodeMaster()) { cleanupRestoreState(event); diff --git a/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index d939236732e..d6598eb3a12 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -26,10 +26,11 @@ import org.apache.lucene.index.IndexCommit; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.SnapshotsInProgress.ShardSnapshotStatus; +import org.elasticsearch.cluster.SnapshotsInProgress.State; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; @@ -84,7 +85,7 @@ import static org.elasticsearch.cluster.SnapshotsInProgress.completed; * This service runs on data and master nodes and controls currently snapshotted shards on these nodes. It is responsible for * starting and stopping shard level snapshots */ -public class SnapshotShardsService extends AbstractLifecycleComponent implements ClusterStateListener, IndexEventListener { +public class SnapshotShardsService extends AbstractLifecycleComponent implements ClusterStateApplier, IndexEventListener { public static final String UPDATE_SNAPSHOT_ACTION_NAME = "internal:cluster/snapshot/update_snapshot"; @@ -118,8 +119,8 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements this.threadPool = threadPool; if (DiscoveryNode.isDataNode(settings)) { // this is only useful on the nodes that can hold data - // addLast to make sure that Repository will be created before snapshot - clusterService.addLast(this); + // addLowPriorityApplier to make sure that Repository will be created before snapshot + clusterService.addLowPriorityApplier(this); } if (DiscoveryNode.isMasterNode(settings)) { @@ -151,11 +152,11 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements @Override protected void doClose() { - clusterService.remove(this); + clusterService.removeApplier(this); } @Override - public void clusterChanged(ClusterChangedEvent event) { + public void applyClusterState(ClusterChangedEvent event) { try { SnapshotsInProgress prev = event.previousState().custom(SnapshotsInProgress.TYPE); SnapshotsInProgress curr = event.state().custom(SnapshotsInProgress.TYPE); @@ -234,19 +235,20 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements // snapshots in the future Map> newSnapshots = new HashMap<>(); // Now go through all snapshots and update existing or create missing - final String localNodeId = clusterService.localNode().getId(); + final String localNodeId = event.state().nodes().getLocalNodeId(); + final DiscoveryNode masterNode = event.state().nodes().getMasterNode(); final Map> snapshotIndices = new HashMap<>(); if (snapshotsInProgress != null) { for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { snapshotIndices.put(entry.snapshot(), entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity()))); - if (entry.state() == SnapshotsInProgress.State.STARTED) { + if (entry.state() == State.STARTED) { Map startedShards = new HashMap<>(); SnapshotShards snapshotShards = shardSnapshots.get(entry.snapshot()); for (ObjectObjectCursor shard : entry.shards()) { // Add all new shards to start processing on if (localNodeId.equals(shard.value.nodeId())) { - if (shard.value.state() == SnapshotsInProgress.State.INIT && (snapshotShards == null || !snapshotShards.shards.containsKey(shard.key))) { + if (shard.value.state() == State.INIT && (snapshotShards == null || !snapshotShards.shards.containsKey(shard.key))) { logger.trace("[{}] - Adding shard to the queue", shard.key); startedShards.put(shard.key, new IndexShardSnapshotStatus()); } @@ -267,7 +269,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements survivors.put(entry.snapshot(), new SnapshotShards(unmodifiableMap(startedShards))); } } - } else if (entry.state() == SnapshotsInProgress.State.ABORTED) { + } else if (entry.state() == State.ABORTED) { // Abort all running shards for this snapshot SnapshotShards snapshotShards = shardSnapshots.get(entry.snapshot()); if (snapshotShards != null) { @@ -285,12 +287,12 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements case DONE: logger.debug("[{}] trying to cancel snapshot on the shard [{}] that is already done, updating status on the master", entry.snapshot(), shard.key); updateIndexShardSnapshotStatus(entry.snapshot(), shard.key, - new ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.SUCCESS)); + new ShardSnapshotStatus(localNodeId, State.SUCCESS), masterNode); break; case FAILURE: logger.debug("[{}] trying to cancel snapshot on the shard [{}] that has already failed, updating status on the master", entry.snapshot(), shard.key); updateIndexShardSnapshotStatus(entry.snapshot(), shard.key, - new ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.FAILED, snapshotStatus.failure())); + new ShardSnapshotStatus(localNodeId, State.FAILED, snapshotStatus.failure()), masterNode); break; default: throw new IllegalStateException("Unknown snapshot shard stage " + snapshotStatus.stage()); @@ -331,18 +333,21 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements @Override public void doRun() { snapshot(indexShard, entry.getKey(), indexId, shardEntry.getValue()); - updateIndexShardSnapshotStatus(entry.getKey(), shardId, new ShardSnapshotStatus(localNodeId, SnapshotsInProgress.State.SUCCESS)); + updateIndexShardSnapshotStatus(entry.getKey(), shardId, + new ShardSnapshotStatus(localNodeId, State.SUCCESS), masterNode); } @Override public void onFailure(Exception e) { logger.warn((Supplier) () -> new ParameterizedMessage("[{}] [{}] failed to create snapshot", shardId, entry.getKey()), e); - updateIndexShardSnapshotStatus(entry.getKey(), shardId, new ShardSnapshotStatus(localNodeId, SnapshotsInProgress.State.FAILED, ExceptionsHelper.detailedMessage(e))); + updateIndexShardSnapshotStatus(entry.getKey(), shardId, + new ShardSnapshotStatus(localNodeId, State.FAILED, ExceptionsHelper.detailedMessage(e)), masterNode); } }); } catch (Exception e) { - updateIndexShardSnapshotStatus(entry.getKey(), shardId, new ShardSnapshotStatus(localNodeId, SnapshotsInProgress.State.FAILED, ExceptionsHelper.detailedMessage(e))); + updateIndexShardSnapshotStatus(entry.getKey(), shardId, + new ShardSnapshotStatus(localNodeId, State.FAILED, ExceptionsHelper.detailedMessage(e)), masterNode); } } } @@ -401,8 +406,10 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements if (snapshotsInProgress == null) { return; } + final String localNodeId = event.state().nodes().getLocalNodeId(); + final DiscoveryNode masterNode = event.state().nodes().getMasterNode(); for (SnapshotsInProgress.Entry snapshot : snapshotsInProgress.entries()) { - if (snapshot.state() == SnapshotsInProgress.State.STARTED || snapshot.state() == SnapshotsInProgress.State.ABORTED) { + if (snapshot.state() == State.STARTED || snapshot.state() == State.ABORTED) { Map localShards = currentSnapshotShards(snapshot.snapshot()); if (localShards != null) { ImmutableOpenMap masterShards = snapshot.shards(); @@ -416,12 +423,12 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements // but we think the shard is done - we need to make new master know that the shard is done logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard is done locally, updating status on the master", snapshot.snapshot(), shardId); updateIndexShardSnapshotStatus(snapshot.snapshot(), shardId, - new ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.SUCCESS)); + new ShardSnapshotStatus(localNodeId, State.SUCCESS), masterNode); } else if (localShard.getValue().stage() == Stage.FAILURE) { // but we think the shard failed - we need to make new master know that the shard failed logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard failed locally, updating status on master", snapshot.snapshot(), shardId); updateIndexShardSnapshotStatus(snapshot.snapshot(), shardId, - new ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.FAILED, localShardStatus.failure())); + new ShardSnapshotStatus(localNodeId, State.FAILED, localShardStatus.failure()), masterNode); } } @@ -508,15 +515,10 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements /** * Updates the shard status */ - public void updateIndexShardSnapshotStatus(Snapshot snapshot, ShardId shardId, ShardSnapshotStatus status) { + public void updateIndexShardSnapshotStatus(Snapshot snapshot, ShardId shardId, ShardSnapshotStatus status, DiscoveryNode master) { UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status); try { - if (clusterService.state().nodes().isLocalNodeElectedMaster()) { - innerUpdateSnapshotState(request); - } else { - transportService.sendRequest(clusterService.state().nodes().getMasterNode(), - UPDATE_SNAPSHOT_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); - } + transportService.sendRequest(master, UPDATE_SNAPSHOT_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); } catch (Exception e) { logger.warn((Supplier) () -> new ParameterizedMessage("[{}] [{}] failed to update snapshot state", request.snapshot(), request.status()), e); } @@ -579,7 +581,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements } else { // Snapshot is finished - mark it as done // TODO: Add PARTIAL_SUCCESS status? - SnapshotsInProgress.Entry updatedEntry = new SnapshotsInProgress.Entry(entry, SnapshotsInProgress.State.SUCCESS, shards.build()); + SnapshotsInProgress.Entry updatedEntry = new SnapshotsInProgress.Entry(entry, State.SUCCESS, shards.build()); entries.add(updatedEntry); // Finalize snapshot in the repository snapshotsService.endSnapshot(updatedEntry); diff --git a/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index d84e3c79f66..056b2e7b10d 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -30,7 +30,7 @@ import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.SnapshotsInProgress; @@ -102,7 +102,7 @@ import static org.elasticsearch.cluster.SnapshotsInProgress.completed; * notifies all {@link #snapshotCompletionListeners} that snapshot is completed, and finally calls {@link #removeSnapshotFromClusterState(Snapshot, SnapshotInfo, Exception)} to remove snapshot from cluster state * */ -public class SnapshotsService extends AbstractLifecycleComponent implements ClusterStateListener { +public class SnapshotsService extends AbstractLifecycleComponent implements ClusterStateApplier { private final ClusterService clusterService; @@ -123,8 +123,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus this.threadPool = threadPool; if (DiscoveryNode.isMasterNode(settings)) { - // addLast to make sure that Repository will be created before snapshot - clusterService.addLast(this); + // addLowPriorityApplier to make sure that Repository will be created before snapshot + clusterService.addLowPriorityApplier(this); } } @@ -592,7 +592,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus } @Override - public void clusterChanged(ClusterChangedEvent event) { + public void applyClusterState(ClusterChangedEvent event) { try { if (event.localNodeMaster()) { if (event.nodesRemoved()) { @@ -1273,7 +1273,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus @Override protected void doClose() { - clusterService.remove(this); + clusterService.removeApplier(this); } public RepositoriesService getRepositoriesService() { diff --git a/core/src/main/java/org/elasticsearch/tasks/TaskManager.java b/core/src/main/java/org/elasticsearch/tasks/TaskManager.java index 0f2165824fe..61c36f9015d 100644 --- a/core/src/main/java/org/elasticsearch/tasks/TaskManager.java +++ b/core/src/main/java/org/elasticsearch/tasks/TaskManager.java @@ -27,7 +27,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.component.AbstractComponent; @@ -53,7 +53,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; /** * Task Manager service for keeping track of currently running tasks on the nodes */ -public class TaskManager extends AbstractComponent implements ClusterStateListener { +public class TaskManager extends AbstractComponent implements ClusterStateApplier { private static final TimeValue WAIT_FOR_COMPLETION_POLL = timeValueMillis(100); private final ConcurrentMapLong tasks = ConcurrentCollections.newConcurrentMapLongWithAggressiveConcurrency(); @@ -314,7 +314,7 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen } @Override - public void clusterChanged(ClusterChangedEvent event) { + public void applyClusterState(ClusterChangedEvent event) { lastDiscoveryNodes = event.state().getNodes(); if (event.nodesRemoved()) { synchronized (banedParents) { diff --git a/core/src/main/java/org/elasticsearch/tribe/TribeService.java b/core/src/main/java/org/elasticsearch/tribe/TribeService.java index f8d2ceb56d0..d976f9229b8 100644 --- a/core/src/main/java/org/elasticsearch/tribe/TribeService.java +++ b/core/src/main/java/org/elasticsearch/tribe/TribeService.java @@ -293,7 +293,7 @@ public class TribeService extends AbstractLifecycleComponent { public void startNodes() { for (Node node : nodes) { try { - getClusterService(node).add(new TribeClusterStateListener(node)); + getClusterService(node).addListener(new TribeClusterStateListener(node)); node.start(); } catch (Exception e) { // calling close is safe for non started nodes, we can just iterate over all diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java index 19e38343efa..43adf182c8b 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java @@ -183,7 +183,7 @@ public abstract class TaskManagerTestCase extends ESTestCase { } }; transportService.start(); - clusterService.add(transportService.getTaskManager()); + clusterService.addStateApplier(transportService.getTaskManager()); discoveryNode = new DiscoveryNode(name, transportService.boundAddress().publishAddress(), emptyMap(), emptySet(), Version.CURRENT); IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(settings); diff --git a/core/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java b/core/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java index 4106ae2177d..132b770ca15 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java @@ -19,19 +19,13 @@ package org.elasticsearch.action.bulk; -import java.util.Collections; -import java.util.Iterator; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.BiConsumer; -import java.util.function.Consumer; - import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; @@ -49,6 +43,12 @@ import org.mockito.ArgumentCaptor; import org.mockito.Captor; import org.mockito.MockitoAnnotations; +import java.util.Collections; +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.sameInstance; import static org.mockito.Matchers.any; @@ -134,9 +134,9 @@ public class TransportBulkActionIngestTests extends ESTestCase { doAnswer(invocation -> { ClusterChangedEvent event = mock(ClusterChangedEvent.class); when(event.state()).thenReturn(state); - ((ClusterStateListener)invocation.getArguments()[0]).clusterChanged(event); + ((ClusterStateApplier)invocation.getArguments()[0]).applyClusterState(event); return null; - }).when(clusterService).add(any(ClusterStateListener.class)); + }).when(clusterService).addStateApplier(any(ClusterStateApplier.class)); // setup the mocked ingest service for capturing calls ingestService = mock(IngestService.class); executionService = mock(PipelineExecutionService.class); diff --git a/core/src/test/java/org/elasticsearch/action/index/TransportIndexActionIngestTests.java b/core/src/test/java/org/elasticsearch/action/index/TransportIndexActionIngestTests.java index 2ac08f9e894..ebc765243c4 100644 --- a/core/src/test/java/org/elasticsearch/action/index/TransportIndexActionIngestTests.java +++ b/core/src/test/java/org/elasticsearch/action/index/TransportIndexActionIngestTests.java @@ -19,15 +19,11 @@ package org.elasticsearch.action.index; -import java.util.Collections; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; - import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; @@ -44,6 +40,10 @@ import org.mockito.ArgumentCaptor; import org.mockito.Captor; import org.mockito.MockitoAnnotations; +import java.util.Collections; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + import static org.hamcrest.Matchers.sameInstance; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; @@ -126,9 +126,9 @@ public class TransportIndexActionIngestTests extends ESTestCase { doAnswer(invocation -> { ClusterChangedEvent event = mock(ClusterChangedEvent.class); when(event.state()).thenReturn(state); - ((ClusterStateListener)invocation.getArguments()[0]).clusterChanged(event); + ((ClusterStateApplier)invocation.getArguments()[0]).applyClusterState(event); return null; - }).when(clusterService).add(any(ClusterStateListener.class)); + }).when(clusterService).addStateApplier(any(ClusterStateApplier.class)); // setup the mocked ingest service for capturing calls ingestService = mock(IngestService.class); executionService = mock(PipelineExecutionService.class); 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 feaeee703b6..937658736d6 100644 --- a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java @@ -33,6 +33,7 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.RecoverySource; @@ -68,6 +69,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; +import static org.elasticsearch.test.ClusterServiceUtils.setState; import static org.hamcrest.CoreMatchers.allOf; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -114,7 +116,11 @@ public class ClusterStateHealthTests extends ESTestCase { public void testClusterHealthWaitsForClusterStateApplication() throws InterruptedException, ExecutionException { final CountDownLatch applyLatch = new CountDownLatch(1); final CountDownLatch listenerCalled = new CountDownLatch(1); - clusterService.add(event -> { + + setState(clusterService, ClusterState.builder(clusterService.state()) + .nodes(DiscoveryNodes.builder(clusterService.state().nodes()).masterNodeId(null)).build()); + + clusterService.addStateApplier(event -> { listenerCalled.countDown(); try { applyLatch.await(); @@ -123,16 +129,23 @@ public class ClusterStateHealthTests extends ESTestCase { } }); - clusterService.submitStateUpdateTask("test", new ClusterStateUpdateTask() { + logger.info("--> submit task to restore master"); + clusterService.submitStateUpdateTask("restore master", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) throws Exception { - return ClusterState.builder(currentState).build(); + final DiscoveryNodes nodes = currentState.nodes(); + return ClusterState.builder(currentState).nodes(DiscoveryNodes.builder(nodes).masterNodeId(nodes.getLocalNodeId())).build(); } @Override public void onFailure(String source, Exception e) { logger.warn("unexpected failure", e); } + + @Override + public boolean runOnlyOnMaster() { + return false; + } }); logger.info("--> waiting for listener to be called and cluster state being blocked"); @@ -141,10 +154,11 @@ public class ClusterStateHealthTests extends ESTestCase { TransportClusterHealthAction action = new TransportClusterHealthAction(Settings.EMPTY, transportService, clusterService, threadPool, new ActionFilters(new HashSet<>()), indexNameExpressionResolver, new TestGatewayAllocator()); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new ClusterHealthRequest(), listener); + action.execute(new ClusterHealthRequest().waitForGreenStatus(), listener); assertFalse(listener.isDone()); + logger.info("--> realising task to restore master"); applyLatch.countDown(); listener.get(); } 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 a41ecdec79e..994510ec56c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java @@ -70,7 +70,7 @@ public class DelayedAllocationServiceTests extends ESAllocationTestCase { clusterService = mock(ClusterService.class); allocationService = createAllocationService(Settings.EMPTY, new DelayedShardsMockGatewayAllocator()); delayedAllocationService = new TestDelayAllocationService(Settings.EMPTY, threadPool, clusterService, allocationService); - verify(clusterService).addFirst(delayedAllocationService); + verify(clusterService).addListener(delayedAllocationService); } @After diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java index a5d4790f742..57086a0d1bf 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java @@ -1152,7 +1152,7 @@ public class ClusterServiceTests extends ESTestCase { TimedClusterService timedClusterService = createTimedClusterService(false); AtomicBoolean isMaster = new AtomicBoolean(); - timedClusterService.add(new LocalNodeMasterListener() { + timedClusterService.addLocalNodeMasterListener(new LocalNodeMasterListener() { @Override public void onMaster() { isMaster.set(true); @@ -1190,6 +1190,44 @@ public class ClusterServiceTests extends ESTestCase { 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()); + } + private static class SimpleTask { private final int id; diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index ce57e7be05b..90d7e5e5a9e 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -28,9 +28,7 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.block.ClusterBlock; @@ -43,8 +41,6 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.cluster.service.ClusterServiceState; -import org.elasticsearch.cluster.service.ClusterStateStatus; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; @@ -689,29 +685,23 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { String>>>()); for (final String node : majoritySide) { masters.put(node, new ArrayList>()); - internalCluster().getInstance(ClusterService.class, node).add(new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - DiscoveryNode previousMaster = event.previousState().nodes().getMasterNode(); - DiscoveryNode currentMaster = event.state().nodes().getMasterNode(); - if (!Objects.equals(previousMaster, currentMaster)) { - logger.info("node {} received new cluster state: {} \n and had previous cluster state: {}", node, event.state(), - event.previousState()); - String previousMasterNodeName = previousMaster != null ? previousMaster.getName() : null; - String currentMasterNodeName = currentMaster != null ? currentMaster.getName() : null; - masters.get(node).add(new Tuple<>(previousMasterNodeName, currentMasterNodeName)); - } + internalCluster().getInstance(ClusterService.class, node).addListener(event -> { + DiscoveryNode previousMaster = event.previousState().nodes().getMasterNode(); + DiscoveryNode currentMaster = event.state().nodes().getMasterNode(); + if (!Objects.equals(previousMaster, currentMaster)) { + logger.info("node {} received new cluster state: {} \n and had previous cluster state: {}", node, event.state(), + event.previousState()); + String previousMasterNodeName = previousMaster != null ? previousMaster.getName() : null; + String currentMasterNodeName = currentMaster != null ? currentMaster.getName() : null; + masters.get(node).add(new Tuple<>(previousMasterNodeName, currentMasterNodeName)); } }); } final CountDownLatch oldMasterNodeSteppedDown = new CountDownLatch(1); - internalCluster().getInstance(ClusterService.class, oldMasterNode).add(new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - if (event.state().nodes().getMasterNodeId() == null) { - oldMasterNodeSteppedDown.countDown(); - } + internalCluster().getInstance(ClusterService.class, oldMasterNode).addListener(event -> { + if (event.state().nodes().getMasterNodeId() == null) { + oldMasterNodeSteppedDown.countDown(); } }); @@ -1199,9 +1189,8 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { // Don't restart the master node until we know the index deletion has taken effect on master and the master eligible node. assertBusy(() -> { for (String masterNode : allMasterEligibleNodes) { - final ClusterServiceState masterState = internalCluster().clusterService(masterNode).clusterServiceState(); - assertTrue("index not deleted on " + masterNode, masterState.getClusterState().metaData().hasIndex(idxName) == false && - masterState.getClusterStateStatus() == ClusterStateStatus.APPLIED); + final ClusterState masterState = internalCluster().clusterService(masterNode).state(); + assertTrue("index not deleted on " + masterNode, masterState.metaData().hasIndex(idxName) == false); } }); internalCluster().restartNode(masterNode1, InternalTestCluster.EMPTY_CALLBACK); 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 27fb48f764c..cc8d43cc79e 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java @@ -133,7 +133,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase { ClusterService clusterService = internalCluster().getInstance(ClusterService.class, master); final ArrayList statesFound = new ArrayList<>(); final CountDownLatch nodesStopped = new CountDownLatch(1); - clusterService.add(event -> { + clusterService.addStateApplier(event -> { statesFound.add(event.state()); try { // block until both nodes have stopped to accumulate node failures diff --git a/core/src/test/java/org/elasticsearch/gateway/DanglingIndicesStateTests.java b/core/src/test/java/org/elasticsearch/gateway/DanglingIndicesStateTests.java index 581ef0f99a3..91d6d3f08fe 100644 --- a/core/src/test/java/org/elasticsearch/gateway/DanglingIndicesStateTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/DanglingIndicesStateTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexGraveyard; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; @@ -34,6 +35,7 @@ import java.nio.file.StandardCopyOption; import java.util.Map; import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; public class DanglingIndicesStateTests extends ESTestCase { @@ -46,7 +48,7 @@ public class DanglingIndicesStateTests extends ESTestCase { public void testCleanupWhenEmpty() throws Exception { try (NodeEnvironment env = newNodeEnvironment()) { MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env); - DanglingIndicesState danglingState = new DanglingIndicesState(Settings.EMPTY, env, metaStateService, null); + DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService); assertTrue(danglingState.getDanglingIndices().isEmpty()); MetaData metaData = MetaData.builder().build(); @@ -57,7 +59,7 @@ public class DanglingIndicesStateTests extends ESTestCase { public void testDanglingIndicesDiscovery() throws Exception { try (NodeEnvironment env = newNodeEnvironment()) { MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env); - DanglingIndicesState danglingState = new DanglingIndicesState(Settings.EMPTY, env, metaStateService, null); + DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService); assertTrue(danglingState.getDanglingIndices().isEmpty()); MetaData metaData = MetaData.builder().build(); @@ -75,7 +77,7 @@ public class DanglingIndicesStateTests extends ESTestCase { public void testInvalidIndexFolder() throws Exception { try (NodeEnvironment env = newNodeEnvironment()) { MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env); - DanglingIndicesState danglingState = new DanglingIndicesState(Settings.EMPTY, env, metaStateService, null); + DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService); MetaData metaData = MetaData.builder().build(); final String uuid = "test1UUID"; @@ -99,7 +101,7 @@ public class DanglingIndicesStateTests extends ESTestCase { public void testDanglingProcessing() throws Exception { try (NodeEnvironment env = newNodeEnvironment()) { MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env); - DanglingIndicesState danglingState = new DanglingIndicesState(Settings.EMPTY, env, metaStateService, null); + DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService); MetaData metaData = MetaData.builder().build(); @@ -143,7 +145,7 @@ public class DanglingIndicesStateTests extends ESTestCase { public void testDanglingIndicesNotImportedWhenTombstonePresent() throws Exception { try (NodeEnvironment env = newNodeEnvironment()) { MetaStateService metaStateService = new MetaStateService(Settings.EMPTY, env); - DanglingIndicesState danglingState = new DanglingIndicesState(Settings.EMPTY, env, metaStateService, null); + DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService); final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetaData.SETTING_INDEX_UUID, "test1UUID"); IndexMetaData dangledIndex = IndexMetaData.builder("test1").settings(settings).build(); @@ -155,4 +157,9 @@ public class DanglingIndicesStateTests extends ESTestCase { } } + + private DanglingIndicesState createDanglingIndicesState(NodeEnvironment env, MetaStateService metaStateService) { + return new DanglingIndicesState(Settings.EMPTY, env, metaStateService, null, + mock(ClusterService.class)); + } } diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index d747e63a695..6b24c2e356c 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -61,6 +61,7 @@ import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.IndicesQueryCache; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.indices.query.IndicesQueriesRegistry; @@ -189,7 +190,7 @@ public class IndexModuleTests extends ESTestCase { final AtomicBoolean atomicBoolean = new AtomicBoolean(false); final IndexEventListener eventListener = new IndexEventListener() { @Override - public void beforeIndexDeleted(IndexService indexService) { + public void beforeIndexRemoved(IndexService indexService, IndexRemovalReason reason) { atomicBoolean.set(true); } }; @@ -201,7 +202,7 @@ public class IndexModuleTests extends ESTestCase { IndexSettings x = indexService.getIndexSettings(); assertEquals(x.getSettings().getAsMap(), indexSettings.getSettings().getAsMap()); assertEquals(x.getIndex(), index); - indexService.getIndexEventListener().beforeIndexDeleted(null); + indexService.getIndexEventListener().beforeIndexRemoved(null, null); assertTrue(atomicBoolean.get()); indexService.close("simon says", false); } diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/AbstractGeoFieldDataTestCase.java b/core/src/test/java/org/elasticsearch/index/fielddata/AbstractGeoFieldDataTestCase.java index df7df5771cb..21d816e83ea 100644 --- a/core/src/test/java/org/elasticsearch/index/fielddata/AbstractGeoFieldDataTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/fielddata/AbstractGeoFieldDataTestCase.java @@ -22,7 +22,6 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.LatLonDocValuesField; import org.apache.lucene.document.StringField; -import org.elasticsearch.Version; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoUtils; diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java index 8ca6aeba30a..b22b48767a8 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java @@ -18,9 +18,6 @@ */ package org.elasticsearch.index.mapper; -import java.io.IOException; -import java.util.Locale; - import org.apache.lucene.document.LongPoint; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexOptions; @@ -36,8 +33,6 @@ import org.elasticsearch.common.joda.DateMathParser; import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.mapper.DateFieldMapper; -import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.DateFieldMapper.DateFieldType; import org.elasticsearch.index.mapper.MappedFieldType.Relation; import org.elasticsearch.index.mapper.ParseContext.Document; @@ -46,6 +41,9 @@ import org.elasticsearch.index.query.QueryShardContext; import org.joda.time.DateTimeZone; import org.junit.Before; +import java.io.IOException; +import java.util.Locale; + public class DateFieldTypeTests extends FieldTypeTestCase { @Override protected MappedFieldType createDefaultFieldType() { @@ -73,7 +71,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase { } public void testIsFieldWithinQueryEmptyReader() throws IOException { - QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, null, null, null, () -> nowInMillis); + QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, null, null, () -> nowInMillis); IndexReader reader = new MultiReader(); DateFieldType ft = new DateFieldType(); ft.setName("my_date"); @@ -83,7 +81,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase { private void doTestIsFieldWithinQuery(DateFieldType ft, DirectoryReader reader, DateTimeZone zone, DateMathParser alternateFormat) throws IOException { - QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, null, null, null, () -> nowInMillis); + QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, null, null, () -> nowInMillis); assertEquals(Relation.INTERSECTS, ft.isFieldWithinQuery(reader, "2015-10-09", "2016-01-02", randomBoolean(), randomBoolean(), null, null, context)); assertEquals(Relation.INTERSECTS, ft.isFieldWithinQuery(reader, "2016-01-02", "2016-06-20", @@ -130,7 +128,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase { DateFieldType ft2 = new DateFieldType(); ft2.setName("my_date2"); - QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, null, null, null, () -> nowInMillis); + QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, null, null, () -> nowInMillis); assertEquals(Relation.DISJOINT, ft2.isFieldWithinQuery(reader, "2015-10-09", "2016-01-02", false, false, null, null, context)); IOUtils.close(reader, w, dir); } @@ -165,7 +163,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase { QueryShardContext context = new QueryShardContext(0, new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), - null, null, null, null, null, null, null, null, null, () -> nowInMillis); + null, null, null, null, null, null, null, null, () -> nowInMillis); MappedFieldType ft = createDefaultFieldType(); ft.setName("field"); String date = "2015-10-12T14:10:55"; @@ -184,7 +182,7 @@ public class DateFieldTypeTests extends FieldTypeTestCase { .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).build(); QueryShardContext context = new QueryShardContext(0, new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), - null, null, null, null, null, null, null, null, null, () -> nowInMillis); + null, null, null, null, null, null, null, null, () -> nowInMillis); MappedFieldType ft = createDefaultFieldType(); ft.setName("field"); String date1 = "2015-10-12T14:10:55"; diff --git a/core/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java b/core/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java index b4bdea30c30..9010164fe03 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java @@ -74,7 +74,7 @@ public class RangeFieldTypeTests extends FieldTypeTestCase { Settings indexSettings = Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAsciiOfLengthBetween(1, 10), indexSettings); - QueryShardContext context = new QueryShardContext(0, idxSettings, null, null, null, null, null, null, null, null, null, + QueryShardContext context = new QueryShardContext(0, idxSettings, null, null, null, null, null, null, null, null, () -> nowInMillis); RangeFieldMapper.RangeFieldType ft = new RangeFieldMapper.RangeFieldType(type); ft.setName(FIELDNAME); diff --git a/core/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/core/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index 29fe3af19c0..73449dbf5d9 100644 --- a/core/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -48,8 +48,7 @@ public class QueryShardContextTests extends ESTestCase { when(mapperService.getIndexSettings()).thenReturn(indexSettings); final long nowInMillis = randomPositiveLong(); QueryShardContext context = new QueryShardContext( - 0, indexSettings, null, null, mapperService, null, null, null, null, null, null, - () -> nowInMillis); + 0, indexSettings, null, null, mapperService, null, null, null, null, null, () -> nowInMillis); context.setAllowUnmappedFields(false); MappedFieldType fieldType = new TextFieldMapper.TextFieldType(); diff --git a/core/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java b/core/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java index 6c5fe046a4b..fd6f0670a57 100644 --- a/core/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java @@ -37,7 +37,7 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase { IndexService indexService = createIndex("test"); IndexReader reader = new MultiReader(); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(), - null, null, null, null, reader, null, null); + null, null, null, null, reader, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); assertEquals(Relation.DISJOINT, range.getRelation(context)); } @@ -54,7 +54,7 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase { indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE, false); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(), - null, null, null, null, null, null, null); + null, null, null, null, null, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // can't make assumptions on a missing reader, so it must return INTERSECT assertEquals(Relation.INTERSECTS, range.getRelation(context)); @@ -73,7 +73,7 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase { new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE, false); IndexReader reader = new MultiReader(); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(), - null, null, null, null, reader, null, null); + null, null, null, null, reader, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // no values -> DISJOINT assertEquals(Relation.DISJOINT, range.getRelation(context)); diff --git a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java index 253bc95ffb8..5e971abb2bd 100644 --- a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java @@ -22,27 +22,23 @@ package org.elasticsearch.index.query; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.index.Term; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.SynonymQuery; -import org.apache.lucene.search.PrefixQuery; -import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.PrefixQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.SynonymQuery; +import org.apache.lucene.search.TermQuery; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MockFieldMapper; -import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.search.SearchModule; import org.elasticsearch.test.ESTestCase; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.settings.Settings; import java.util.Collections; import java.util.HashMap; @@ -152,7 +148,7 @@ public class SimpleQueryParserTests extends ESTestCase { IndexMetaData indexState = IndexMetaData.builder("index").settings(indexSettings).build(); IndexSettings settings = new IndexSettings(indexState, Settings.EMPTY); QueryShardContext mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, indicesQueriesRegistry, - null, null, null, System::currentTimeMillis) { + null, null, System::currentTimeMillis) { @Override public MappedFieldType fieldMapper(String name) { return new MockFieldMapper.FakeFieldType(); @@ -166,7 +162,7 @@ public class SimpleQueryParserTests extends ESTestCase { // Now check what happens if foo.quote does not exist mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, indicesQueriesRegistry, - null, null, null, System::currentTimeMillis) { + null, null, System::currentTimeMillis) { @Override public MappedFieldType fieldMapper(String name) { if (name.equals("foo.quote")) { diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 46cd30112b5..a2a90a8c753 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -28,9 +28,11 @@ import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -41,11 +43,12 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; public class IndicesLifecycleListenerSingleNodeTests extends ESSingleNodeTestCase { - public void testCloseDeleteCallback() throws Throwable { + public void testStartDeleteIndexEventCallback() throws Throwable { IndicesService indicesService = getInstanceFromNode(IndicesService.class); assertAcked(client().admin().indices().prepareCreate("test") .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0)); @@ -55,45 +58,70 @@ public class IndicesLifecycleListenerSingleNodeTests extends ESSingleNodeTestCas ShardRouting shardRouting = indicesService.indexService(idx).getShard(0).routingEntry(); final AtomicInteger counter = new AtomicInteger(1); IndexEventListener countingListener = new IndexEventListener() { + @Override - public void afterIndexClosed(Index index, Settings indexSettings) { - assertEquals(counter.get(), 5); + public void beforeIndexCreated(Index index, Settings indexSettings) { + assertEquals("test", index.getName()); + assertEquals(1, counter.get()); counter.incrementAndGet(); } @Override - public void beforeIndexClosed(IndexService indexService) { - assertEquals(counter.get(), 1); + public void afterIndexCreated(IndexService indexService) { + assertEquals("test", indexService.index().getName()); + assertEquals(2, counter.get()); counter.incrementAndGet(); } @Override - public void afterIndexDeleted(Index index, Settings indexSettings) { - assertEquals(counter.get(), 6); + public void beforeIndexShardCreated(ShardId shardId, Settings indexSettings) { + assertEquals(3, counter.get()); counter.incrementAndGet(); } @Override - public void beforeIndexDeleted(IndexService indexService) { - assertEquals(counter.get(), 2); + public void afterIndexShardCreated(IndexShard indexShard) { + assertEquals(4, counter.get()); + counter.incrementAndGet(); + } + + @Override + public void afterIndexShardStarted(IndexShard indexShard) { + assertEquals(5, counter.get()); + counter.incrementAndGet(); + } + + @Override + public void beforeIndexRemoved(IndexService indexService, IndexRemovalReason reason) { + assertEquals(DELETED, reason); + assertEquals(6, counter.get()); counter.incrementAndGet(); } @Override public void beforeIndexShardDeleted(ShardId shardId, Settings indexSettings) { - assertEquals(counter.get(), 3); + assertEquals(7, counter.get()); counter.incrementAndGet(); } @Override public void afterIndexShardDeleted(ShardId shardId, Settings indexSettings) { - assertEquals(counter.get(), 4); + assertEquals(8, counter.get()); counter.incrementAndGet(); } + + @Override + public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRemovalReason reason) { + assertEquals(DELETED, reason); + assertEquals(9, counter.get()); + counter.incrementAndGet(); + } + }; - indicesService.deleteIndex(idx, "simon says"); + indicesService.removeIndex(idx, DELETED, "simon says"); try { IndexService index = indicesService.createIndex(metaData, Arrays.asList(countingListener), s -> {}); + assertEquals(3, counter.get()); idx = index.index(); ShardRouting newRouting = shardRouting; String nodeId = newRouting.currentNodeId(); @@ -103,16 +131,18 @@ public class IndicesLifecycleListenerSingleNodeTests extends ESSingleNodeTestCas newRouting = ShardRoutingHelper.initialize(newRouting, nodeId); IndexShard shard = index.createShard(newRouting); shard.updateRoutingEntry(newRouting); + assertEquals(5, counter.get()); final DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); shard.markAsRecovering("store", new RecoveryState(newRouting, localNode, null)); shard.recoverFromStore(); newRouting = ShardRoutingHelper.moveToStarted(newRouting); shard.updateRoutingEntry(newRouting); + assertEquals(6, counter.get()); } finally { - indicesService.deleteIndex(idx, "simon says"); + indicesService.removeIndex(idx, DELETED, "simon says"); } - assertEquals(7, counter.get()); + assertEquals(10, counter.get()); } } diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index 97b3924303d..2449fdfc290 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -197,23 +197,12 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC } @Override - public synchronized void deleteIndex(Index index, String reason) { - if (hasIndex(index) == false) { - return; + public synchronized void removeIndex(Index index, IndexRemovalReason reason, String extraInfo) { + if (hasIndex(index)) { + Map newIndices = new HashMap<>(indices); + newIndices.remove(index.getUUID()); + indices = unmodifiableMap(newIndices); } - Map newIndices = new HashMap<>(indices); - newIndices.remove(index.getUUID()); - indices = unmodifiableMap(newIndices); - } - - @Override - public synchronized void removeIndex(Index index, String reason) { - if (hasIndex(index) == false) { - return; - } - Map newIndices = new HashMap<>(indices); - newIndices.remove(index.getUUID()); - indices = unmodifiableMap(newIndices); } @Override diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index fdbd2d0548a..4b21b8820c7 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -104,7 +104,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice ClusterState previousLocalState = adaptClusterStateToLocalNode(previousState, node); final ClusterChangedEvent event = new ClusterChangedEvent("simulated change " + i, localState, previousLocalState); try { - indicesClusterStateService.clusterChanged(event); + indicesClusterStateService.applyClusterState(event); } catch (AssertionError error) { logger.error((org.apache.logging.log4j.util.Supplier) () -> new ParameterizedMessage( "failed to apply change on [{}].\n *** Previous state ***\n{}\n *** New state ***\n{}", @@ -127,7 +127,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice * are all removed but the on disk contents of those indices remain so that they can later be * imported as dangling indices. Normally, the first cluster state update that the node * receives from the new cluster would contain a cluster block that would cause all in-memory - * structures to be removed (see {@link IndicesClusterStateService#clusterChanged(ClusterChangedEvent)}), + * structures to be removed (see {@link IndicesClusterStateService#applyClusterState(ClusterChangedEvent)}), * but in the case where the node joined and was a few cluster state updates behind, it would * not have received the cluster block, in which case we still need to remove the in-memory * structures while ensuring the data remains on disk. This test executes this particular @@ -155,7 +155,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice ClusterState previousLocalState = adaptClusterStateToLocalNode(initialState, node); IndicesClusterStateService indicesCSSvc = createIndicesClusterStateService(node, RecordingIndicesService::new); indicesCSSvc.start(); - indicesCSSvc.clusterChanged(new ClusterChangedEvent("cluster state change that adds the index", localState, previousLocalState)); + indicesCSSvc.applyClusterState(new ClusterChangedEvent("cluster state change that adds the index", localState, previousLocalState)); // create a new empty cluster state with a brand new cluster UUID ClusterState newClusterState = ClusterState.builder(initialState) @@ -165,8 +165,8 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice // simulate the cluster state change on the node localState = adaptClusterStateToLocalNode(newClusterState, node); previousLocalState = adaptClusterStateToLocalNode(stateWithIndex, node); - indicesCSSvc.clusterChanged(new ClusterChangedEvent("cluster state change with a new cluster UUID (and doesn't contain the index)", - localState, previousLocalState)); + indicesCSSvc.applyClusterState(new ClusterChangedEvent( + "cluster state change with a new cluster UUID (and doesn't contain the index)", localState, previousLocalState)); // check that in memory data structures have been removed once the new cluster state is applied, // but the persistent data is still there @@ -388,7 +388,6 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice null, null, null, - null, shardId -> {}); } @@ -396,11 +395,13 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice private Set deletedIndices = Collections.emptySet(); @Override - public synchronized void deleteIndex(Index index, String reason) { - super.deleteIndex(index, reason); - Set newSet = Sets.newHashSet(deletedIndices); - newSet.add(index); - deletedIndices = Collections.unmodifiableSet(newSet); + public synchronized void removeIndex(Index index, IndexRemovalReason reason, String extraInfo) { + super.removeIndex(index, reason, extraInfo); + if (reason == IndexRemovalReason.DELETED) { + Set newSet = Sets.newHashSet(deletedIndices); + newSet.add(index); + deletedIndices = Collections.unmodifiableSet(newSet); + } } public synchronized boolean isDeleted(Index index) { diff --git a/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 2db44c9d0ba..0d4d740aa35 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -61,6 +61,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import static java.util.Collections.singletonList; import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.is; @@ -152,7 +153,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase { public void run() { startGun.countDown(); while(running.get()) { - service.afterIndexDeleted(indexService.index(), indexService.getIndexSettings().getSettings()); + service.afterIndexRemoved(indexService.index(), indexService.getIndexSettings(), DELETED); if (randomBoolean()) { // here we trigger some refreshes to ensure the IR go out of scope such that we hit ACE if we access a search // context in a non-sane way. diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java index cd19785f3e0..c232068fffd 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java @@ -100,7 +100,7 @@ public class ExtendedBoundsTests extends ESTestCase { SearchContext context = mock(SearchContext.class); QueryShardContext qsc = new QueryShardContext(0, new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), null, null, null, null, - null, null, null, null, null, () -> now); + null, null, null, null, () -> now); when(context.getQueryShardContext()).thenReturn(qsc); FormatDateTimeFormatter formatter = Joda.forPattern("dateOptionalTime"); DocValueFormat format = new DocValueFormat.DateTime(formatter, DateTimeZone.UTC); diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java index 88f696f1697..07a70c91ef9 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java @@ -266,7 +266,7 @@ public class HighlightBuilderTests extends ESTestCase { IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings); // shard context will only need indicesQueriesRegistry for building Query objects nested in highlighter QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, indicesQueriesRegistry, - null, null, null, System::currentTimeMillis) { + null, null, System::currentTimeMillis) { @Override public MappedFieldType fieldMapper(String name) { TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name); diff --git a/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java b/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java index 168d8f869ba..735c84a3319 100644 --- a/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java +++ b/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java @@ -197,8 +197,7 @@ public class ShardSearchTransportRequestTests extends AbstractSearchTestCase { IndexSettings indexSettings = new IndexSettings(indexMetadata.build(), Settings.EMPTY); final long nowInMillis = randomPositiveLong(); QueryShardContext context = new QueryShardContext( - 0, indexSettings, null, null, null, null, null, queriesRegistry, null, null, null, - () -> nowInMillis); + 0, indexSettings, null, null, null, null, null, queriesRegistry, null, null, () -> nowInMillis); readRequest.rewrite(context); QueryBuilder queryBuilder = readRequest.filteringAliases(); assertEquals(queryBuilder, QueryBuilders.boolQuery() diff --git a/core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java b/core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java index 5fc9e6c6dae..22f2dd53808 100644 --- a/core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java @@ -139,7 +139,7 @@ public class QueryRescoreBuilderTests extends ESTestCase { IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAsciiOfLengthBetween(1, 10), indexSettings); // shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, indicesQueriesRegistry, - null, null, null, () -> nowInMillis) { + null, null, () -> nowInMillis) { @Override public MappedFieldType fieldMapper(String name) { TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name); diff --git a/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java b/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java index ec29911d8ca..d545a082b55 100644 --- a/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java +++ b/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java @@ -211,7 +211,7 @@ public abstract class AbstractSortTestCase> extends EST }); long nowInMillis = randomPositiveLong(); return new QueryShardContext(0, idxSettings, bitsetFilterCache, ifds, null, null, scriptService, - indicesQueriesRegistry, null, null, null, () -> nowInMillis) { + indicesQueriesRegistry, null, null, () -> nowInMillis) { @Override public MappedFieldType fieldMapper(String name) { return provideMappedFieldType(name); diff --git a/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index b1c46b35b62..1d0ddf39b01 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -714,7 +714,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest final ClusterService clusterService = internalCluster().clusterService(internalCluster().getMasterName()); BlockingClusterStateListener snapshotListener = new BlockingClusterStateListener(clusterService, "update_snapshot [", "update snapshot state", Priority.HIGH); try { - clusterService.addFirst(snapshotListener); + clusterService.addListener(snapshotListener); logger.info("--> snapshot"); dataNodeClient().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get(); @@ -728,7 +728,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest snapshotListener.unblock(); } finally { - clusterService.remove(snapshotListener); + clusterService.removeListener(snapshotListener); } logger.info("--> wait until the snapshot is done"); diff --git a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 4765292be1d..dda634023b6 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -2335,7 +2335,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas final ClusterService clusterService = internalCluster().clusterService(internalCluster().getMasterName()); BlockingClusterStateListener snapshotListener = new BlockingClusterStateListener(clusterService, "update_snapshot [", "update snapshot state", Priority.HIGH); try { - clusterService.addFirst(snapshotListener); + clusterService.addListener(snapshotListener); logger.info("--> snapshot"); ListenableActionFuture snapshotFuture = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").execute(); @@ -2350,7 +2350,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas assertEquals(numberOfShards, createSnapshotResponse.getSnapshotInfo().successfulShards()); } finally { - clusterService.remove(snapshotListener); + clusterService.removeListener(snapshotListener); } // Check that we didn't timeout diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java index ba4a33d4496..2055cde567e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java @@ -20,7 +20,6 @@ package org.elasticsearch.test; import com.fasterxml.jackson.core.io.JsonStringEncoder; - import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; @@ -36,8 +35,6 @@ import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.ParseFieldMatcher; @@ -1135,9 +1132,8 @@ public abstract class AbstractQueryTestCase> } QueryShardContext createShardContext() { - ClusterState state = ClusterState.builder(new ClusterName("_name")).build(); return new QueryShardContext(0, idxSettings, bitsetFilterCache, indexFieldDataService, mapperService, similarityService, - scriptService, indicesQueriesRegistry, this.client, null, state, () -> nowInMillis); + scriptService, indicesQueriesRegistry, this.client, null, () -> nowInMillis); } ScriptModule createScriptModule(List scriptPlugins) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/MockIndexEventListener.java b/test/framework/src/main/java/org/elasticsearch/test/MockIndexEventListener.java index c4a9515f545..13caec4adc8 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/MockIndexEventListener.java +++ b/test/framework/src/main/java/org/elasticsearch/test/MockIndexEventListener.java @@ -24,14 +24,15 @@ import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsModule; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import org.elasticsearch.plugins.Plugin; import java.util.Arrays; @@ -133,13 +134,13 @@ public final class MockIndexEventListener { } @Override - public void beforeIndexClosed(IndexService indexService) { - delegate.beforeIndexClosed(indexService); + public void beforeIndexRemoved(IndexService indexService, IndexRemovalReason reason) { + delegate.beforeIndexRemoved(indexService, reason); } @Override - public void afterIndexClosed(Index index, Settings indexSettings) { - delegate.afterIndexClosed(index, indexSettings); + public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRemovalReason reason) { + delegate.afterIndexRemoved(index, indexSettings, reason); } @Override @@ -152,16 +153,6 @@ public final class MockIndexEventListener { delegate.afterIndexShardDeleted(shardId, indexSettings); } - @Override - public void afterIndexDeleted(Index index, Settings indexSettings) { - delegate.afterIndexDeleted(index, indexSettings); - } - - @Override - public void beforeIndexDeleted(IndexService indexService) { - delegate.beforeIndexDeleted(indexService); - } - @Override public void beforeIndexAddedToCluster(Index index, Settings indexSettings) { delegate.beforeIndexAddedToCluster(index, indexSettings); diff --git a/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java b/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java index 45ea62f31ea..0776fa1edb2 100644 --- a/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java +++ b/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java @@ -35,7 +35,7 @@ public class MockSearchServiceTests extends ESTestCase { public void testAssertNoInFlightContext() { final long nowInMillis = randomPositiveLong(); SearchContext s = new TestSearchContext(new QueryShardContext(0, new IndexSettings(IndexMetaData.PROTO, Settings.EMPTY), null, null, - null, null, null, null, null, null, null, () -> nowInMillis)) { + null, null, null, null, null, null, () -> nowInMillis)) { @Override public SearchShardTarget shardTarget() { return new SearchShardTarget("node", new Index("idx", "ignored"), 0);