Remove cluster state initial customs (#32501)
This infrastructure was introduced in #26144 and made obsolete in #30743
This commit is contained in:
parent
097c4287f3
commit
db6e8c736d
|
@ -116,23 +116,6 @@ public class ClusterModule extends AbstractModule {
|
|||
this.allocationService = new AllocationService(settings, allocationDeciders, shardsAllocator, clusterInfoService);
|
||||
}
|
||||
|
||||
public static Map<String, Supplier<ClusterState.Custom>> getClusterStateCustomSuppliers(List<ClusterPlugin> clusterPlugins) {
|
||||
final Map<String, Supplier<ClusterState.Custom>> customSupplier = new HashMap<>();
|
||||
customSupplier.put(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress::new);
|
||||
customSupplier.put(RestoreInProgress.TYPE, RestoreInProgress::new);
|
||||
customSupplier.put(SnapshotsInProgress.TYPE, SnapshotsInProgress::new);
|
||||
for (ClusterPlugin plugin : clusterPlugins) {
|
||||
Map<String, Supplier<ClusterState.Custom>> initialCustomSupplier = plugin.getInitialClusterStateCustomSupplier();
|
||||
for (String key : initialCustomSupplier.keySet()) {
|
||||
if (customSupplier.containsKey(key)) {
|
||||
throw new IllegalStateException("custom supplier key [" + key + "] is registered more than once");
|
||||
}
|
||||
}
|
||||
customSupplier.putAll(initialCustomSupplier);
|
||||
}
|
||||
return Collections.unmodifiableMap(customSupplier);
|
||||
}
|
||||
|
||||
public static List<Entry> getNamedWriteables() {
|
||||
List<Entry> entries = new ArrayList<>();
|
||||
// Cluster State
|
||||
|
|
|
@ -39,11 +39,6 @@ public interface ClusterApplier {
|
|||
*/
|
||||
void onNewClusterState(String source, Supplier<ClusterState> clusterStateSupplier, ClusterApplyListener listener);
|
||||
|
||||
/**
|
||||
* Creates a new cluster state builder that is initialized with the cluster name and all initial cluster state customs.
|
||||
*/
|
||||
ClusterState.Builder newClusterStateBuilder();
|
||||
|
||||
/**
|
||||
* Listener for results of cluster state application
|
||||
*/
|
||||
|
|
|
@ -96,17 +96,14 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
|
|||
private final AtomicReference<ClusterState> state; // last applied state
|
||||
|
||||
private NodeConnectionsService nodeConnectionsService;
|
||||
private Supplier<ClusterState.Builder> stateBuilderSupplier;
|
||||
|
||||
public ClusterApplierService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool, Supplier<ClusterState
|
||||
.Builder> stateBuilderSupplier) {
|
||||
public ClusterApplierService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) {
|
||||
super(settings);
|
||||
this.clusterSettings = clusterSettings;
|
||||
this.threadPool = threadPool;
|
||||
this.state = new AtomicReference<>();
|
||||
this.slowTaskLoggingThreshold = CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings);
|
||||
this.localNodeMasterListeners = new LocalNodeMasterListeners(threadPool);
|
||||
this.stateBuilderSupplier = stateBuilderSupplier;
|
||||
}
|
||||
|
||||
public void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
|
||||
|
@ -652,8 +649,4 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
|
|||
return System.nanoTime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState.Builder newClusterStateBuilder() {
|
||||
return stateBuilderSupplier.get();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -42,7 +42,6 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class ClusterService extends AbstractLifecycleComponent {
|
||||
|
||||
|
@ -59,10 +58,8 @@ public class ClusterService extends AbstractLifecycleComponent {
|
|||
private final OperationRouting operationRouting;
|
||||
|
||||
private final ClusterSettings clusterSettings;
|
||||
private final Map<String, Supplier<ClusterState.Custom>> initialClusterStateCustoms;
|
||||
|
||||
public ClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool,
|
||||
Map<String, Supplier<ClusterState.Custom>> initialClusterStateCustoms) {
|
||||
public ClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) {
|
||||
super(settings);
|
||||
this.masterService = new MasterService(settings, threadPool);
|
||||
this.operationRouting = new OperationRouting(settings, clusterSettings);
|
||||
|
@ -70,19 +67,7 @@ public class ClusterService extends AbstractLifecycleComponent {
|
|||
this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);
|
||||
this.clusterSettings.addSettingsUpdateConsumer(CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING,
|
||||
this::setSlowTaskLoggingThreshold);
|
||||
this.initialClusterStateCustoms = initialClusterStateCustoms;
|
||||
this.clusterApplierService = new ClusterApplierService(settings, clusterSettings, threadPool, this::newClusterStateBuilder);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new cluster state builder that is initialized with the cluster name and all initial cluster state customs.
|
||||
*/
|
||||
public ClusterState.Builder newClusterStateBuilder() {
|
||||
ClusterState.Builder builder = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings));
|
||||
for (Map.Entry<String, Supplier<ClusterState.Custom>> entry : initialClusterStateCustoms.entrySet()) {
|
||||
builder.putCustom(entry.getKey(), entry.getValue().get());
|
||||
}
|
||||
return builder;
|
||||
this.clusterApplierService = new ClusterApplierService(settings, clusterSettings, threadPool);
|
||||
}
|
||||
|
||||
private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.discovery.single;
|
|||
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
|
@ -113,7 +114,7 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
|
|||
}
|
||||
|
||||
protected ClusterState createInitialState(DiscoveryNode localNode) {
|
||||
ClusterState.Builder builder = clusterApplier.newClusterStateBuilder();
|
||||
ClusterState.Builder builder = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings));
|
||||
return builder.nodes(DiscoveryNodes.builder().add(localNode)
|
||||
.localNodeId(localNode.getId())
|
||||
.masterNodeId(localNode.getId())
|
||||
|
|
|
@ -252,7 +252,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|||
// set initial state
|
||||
assert committedState.get() == null;
|
||||
assert localNode != null;
|
||||
ClusterState.Builder builder = clusterApplier.newClusterStateBuilder();
|
||||
ClusterState.Builder builder = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings));
|
||||
ClusterState initialState = builder
|
||||
.blocks(ClusterBlocks.builder()
|
||||
.addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.carrotsearch.hppc.ObjectFloatHashMap;
|
|||
import com.carrotsearch.hppc.cursors.ObjectCursor;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.action.FailedNodeException;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
|
@ -147,7 +148,7 @@ public class Gateway extends AbstractComponent {
|
|||
metaDataBuilder.transientSettings(),
|
||||
e -> logUnknownSetting("transient", e),
|
||||
(e, ex) -> logInvalidSetting("transient", e, ex)));
|
||||
ClusterState.Builder builder = clusterService.newClusterStateBuilder();
|
||||
ClusterState.Builder builder = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings));
|
||||
builder.metaData(metaDataBuilder);
|
||||
listener.onSuccess(builder.build());
|
||||
}
|
||||
|
|
|
@ -349,8 +349,7 @@ public class Node implements Closeable {
|
|||
getCustomNameResolvers(pluginsService.filterPlugins(DiscoveryPlugin.class)));
|
||||
|
||||
List<ClusterPlugin> clusterPlugins = pluginsService.filterPlugins(ClusterPlugin.class);
|
||||
final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool,
|
||||
ClusterModule.getClusterStateCustomSuppliers(clusterPlugins));
|
||||
final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool);
|
||||
clusterService.addStateApplier(scriptModule.getScriptService());
|
||||
resourcesToClose.add(clusterService);
|
||||
final IngestService ingestService = new IngestService(settings, threadPool, this.environment,
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Collections;
|
|||
import java.util.Map;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
|
@ -66,12 +65,4 @@ public interface ClusterPlugin {
|
|||
default void onNodeStarted() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a map of {@link ClusterState.Custom} supplier that should be invoked to initialize the initial clusterstate.
|
||||
* This allows custom clusterstate extensions to be always present and prevents invariants where clusterstates are published
|
||||
* but customs are not initialized.
|
||||
*
|
||||
* TODO: Remove this whole concept of InitialClusterStateCustomSupplier, it's not used anymore
|
||||
*/
|
||||
default Map<String, Supplier<ClusterState.Custom>> getInitialClusterStateCustomSupplier() { return Collections.emptyMap(); }
|
||||
}
|
||||
|
|
|
@ -63,7 +63,7 @@ import java.util.function.Supplier;
|
|||
public class ClusterModuleTests extends ModuleTestCase {
|
||||
private ClusterInfoService clusterInfoService = EmptyClusterInfoService.INSTANCE;
|
||||
private ClusterService clusterService = new ClusterService(Settings.EMPTY,
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null, Collections.emptyMap());
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null);
|
||||
static class FakeAllocationDecider extends AllocationDecider {
|
||||
protected FakeAllocationDecider(Settings settings) {
|
||||
super(settings);
|
||||
|
@ -202,57 +202,6 @@ public class ClusterModuleTests extends ModuleTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testCustomSuppliers() {
|
||||
Map<String, Supplier<ClusterState.Custom>> customSuppliers = ClusterModule.getClusterStateCustomSuppliers(Collections.emptyList());
|
||||
assertEquals(3, customSuppliers.size());
|
||||
assertTrue(customSuppliers.containsKey(SnapshotsInProgress.TYPE));
|
||||
assertTrue(customSuppliers.containsKey(SnapshotDeletionsInProgress.TYPE));
|
||||
assertTrue(customSuppliers.containsKey(RestoreInProgress.TYPE));
|
||||
|
||||
customSuppliers = ClusterModule.getClusterStateCustomSuppliers(Collections.singletonList(new ClusterPlugin() {
|
||||
@Override
|
||||
public Map<String, Supplier<ClusterState.Custom>> getInitialClusterStateCustomSupplier() {
|
||||
return Collections.singletonMap("foo", () -> null);
|
||||
}
|
||||
}));
|
||||
assertEquals(4, customSuppliers.size());
|
||||
assertTrue(customSuppliers.containsKey(SnapshotsInProgress.TYPE));
|
||||
assertTrue(customSuppliers.containsKey(SnapshotDeletionsInProgress.TYPE));
|
||||
assertTrue(customSuppliers.containsKey(RestoreInProgress.TYPE));
|
||||
assertTrue(customSuppliers.containsKey("foo"));
|
||||
|
||||
{
|
||||
// Eclipse Neon 2 didn't compile the plugins definition inside the lambda expression,
|
||||
// probably due to https://bugs.eclipse.org/bugs/show_bug.cgi?id=511750, which is
|
||||
// fixed in Eclipse Oxygon. Pulled out the plugins definition to make it work in older versions
|
||||
List<ClusterPlugin> plugins = Collections.singletonList(new ClusterPlugin() {
|
||||
@Override
|
||||
public Map<String, Supplier<ClusterState.Custom>> getInitialClusterStateCustomSupplier() {
|
||||
return Collections.singletonMap(SnapshotsInProgress.TYPE, () -> null);
|
||||
}
|
||||
});
|
||||
IllegalStateException ise = expectThrows(IllegalStateException.class,
|
||||
() -> ClusterModule.getClusterStateCustomSuppliers(plugins));
|
||||
assertEquals(ise.getMessage(), "custom supplier key [snapshots] is registered more than once");
|
||||
}
|
||||
{
|
||||
List<ClusterPlugin> plugins = Arrays.asList(new ClusterPlugin() {
|
||||
@Override
|
||||
public Map<String, Supplier<ClusterState.Custom>> getInitialClusterStateCustomSupplier() {
|
||||
return Collections.singletonMap("foo", () -> null);
|
||||
}
|
||||
}, new ClusterPlugin() {
|
||||
@Override
|
||||
public Map<String, Supplier<ClusterState.Custom>> getInitialClusterStateCustomSupplier() {
|
||||
return Collections.singletonMap("foo", () -> null);
|
||||
}
|
||||
});
|
||||
IllegalStateException ise = expectThrows(IllegalStateException.class,
|
||||
() -> ClusterModule.getClusterStateCustomSuppliers(plugins));
|
||||
assertEquals(ise.getMessage(), "custom supplier key [foo] is registered more than once");
|
||||
}
|
||||
}
|
||||
|
||||
public void testPre63CustomsFiltering() {
|
||||
final String whiteListedClusterCustom = randomFrom(ClusterModule.PRE_6_3_CLUSTER_CUSTOMS_WHITE_LIST);
|
||||
final String whiteListedMetaDataCustom = randomFrom(ClusterModule.PRE_6_3_METADATA_CUSTOMS_WHITE_LIST);
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.elasticsearch.cluster.metadata.MappingMetaData;
|
|||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
|
@ -38,14 +39,20 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.env.NodeEnvironment;
|
||||
import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.plugins.ClusterPlugin;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.hamcrest.CollectionAssertions;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.watcher.ResourceWatcherService;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -53,9 +60,9 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertIndexTemplateExists;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
@ -279,13 +286,11 @@ public class SimpleClusterStateIT extends ESIntegTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testPrivateCustomsAreExcluded() {
|
||||
public void testPrivateCustomsAreExcluded() throws Exception {
|
||||
// ensure that the custom is injected into the cluster state
|
||||
assertBusy(() -> assertTrue(clusterService().state().customs().containsKey("test")));
|
||||
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().setCustoms(true).get();
|
||||
assertFalse(clusterStateResponse.getState().customs().containsKey("test"));
|
||||
// just to make sure there is something
|
||||
assertTrue(clusterStateResponse.getState().customs().containsKey(SnapshotDeletionsInProgress.TYPE));
|
||||
ClusterState state = internalCluster().getInstance(ClusterService.class).state();
|
||||
assertTrue(state.customs().containsKey("test"));
|
||||
}
|
||||
|
||||
private static class TestCustom extends AbstractNamedDiffable<ClusterState.Custom> implements ClusterState.Custom {
|
||||
|
@ -333,11 +338,6 @@ public class SimpleClusterStateIT extends ESIntegTestCase {
|
|||
|
||||
public PrivateCustomPlugin() {}
|
||||
|
||||
@Override
|
||||
public Map<String, Supplier<ClusterState.Custom>> getInitialClusterStateCustomSupplier() {
|
||||
return Collections.singletonMap("test", () -> new TestCustom(1));
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
|
||||
List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
|
||||
|
@ -345,5 +345,54 @@ public class SimpleClusterStateIT extends ESIntegTestCase {
|
|||
entries.add(new NamedWriteableRegistry.Entry(NamedDiff.class, "test", TestCustom::readDiffFrom));
|
||||
return entries;
|
||||
}
|
||||
|
||||
private final AtomicBoolean installed = new AtomicBoolean();
|
||||
|
||||
@Override
|
||||
public Collection<Object> createComponents(
|
||||
final Client client,
|
||||
final ClusterService clusterService,
|
||||
final ThreadPool threadPool,
|
||||
final ResourceWatcherService resourceWatcherService,
|
||||
final ScriptService scriptService,
|
||||
final NamedXContentRegistry xContentRegistry,
|
||||
final Environment environment,
|
||||
final NodeEnvironment nodeEnvironment,
|
||||
final NamedWriteableRegistry namedWriteableRegistry) {
|
||||
clusterService.addListener(event -> {
|
||||
final ClusterState state = event.state();
|
||||
if (state.getBlocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK)) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (state.nodes().isLocalNodeElectedMaster()) {
|
||||
if (state.custom("test") == null) {
|
||||
if (installed.compareAndSet(false, true)) {
|
||||
clusterService.submitStateUpdateTask("install-metadata-custom", new ClusterStateUpdateTask(Priority.URGENT) {
|
||||
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
if (currentState.custom("test") == null) {
|
||||
final ClusterState.Builder builder = ClusterState.builder(currentState);
|
||||
builder.putCustom("test", new TestCustom(42));
|
||||
return builder.build();
|
||||
} else {
|
||||
return currentState;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
});
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -412,7 +412,7 @@ public class ClusterApplierServiceTests extends ESTestCase {
|
|||
public volatile Long currentTimeOverride = null;
|
||||
|
||||
TimedClusterApplierService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) {
|
||||
super(settings, clusterSettings, threadPool, () -> ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings)));
|
||||
super(settings, clusterSettings, threadPool);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1,68 +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.Version;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.Diff;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
public class ClusterSerivceTests extends ESTestCase {
|
||||
|
||||
public void testNewBuilderContainsCustoms() {
|
||||
ClusterState.Custom custom = new ClusterState.Custom() {
|
||||
@Override
|
||||
public Diff<ClusterState.Custom> diff(ClusterState.Custom previousState) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Version getMinimalSupportedVersion() {
|
||||
return Version.CURRENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
ClusterService service = new ClusterService(Settings.EMPTY,
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null, Collections.singletonMap("foo", () ->
|
||||
custom));
|
||||
ClusterState.Builder builder = service.newClusterStateBuilder();
|
||||
assertSame(builder.build().custom("foo"), custom);
|
||||
}
|
||||
}
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.discovery.single;
|
|||
|
||||
import org.elasticsearch.core.internal.io.IOUtils;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
|
@ -64,11 +63,6 @@ public class SingleNodeDiscoveryTests extends ESTestCase {
|
|||
clusterState.set(initialState);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState.Builder newClusterStateBuilder() {
|
||||
return ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNewClusterState(String source, Supplier<ClusterState> clusterStateSupplier,
|
||||
ClusterApplyListener listener) {
|
||||
|
|
|
@ -306,11 +306,6 @@ public class ZenDiscoveryUnitTests extends ESTestCase {
|
|||
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState.Builder newClusterStateBuilder() {
|
||||
return ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNewClusterState(String source, Supplier<ClusterState> clusterStateSupplier, ClusterApplyListener listener) {
|
||||
listener.onSuccess(source);
|
||||
|
|
|
@ -27,14 +27,13 @@ import org.elasticsearch.test.ESTestCase;
|
|||
import org.hamcrest.Matchers;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
public class GatewayServiceTests extends ESTestCase {
|
||||
|
||||
private GatewayService createService(Settings.Builder settings) {
|
||||
ClusterService clusterService = new ClusterService(Settings.builder().put("cluster.name", "GatewayServiceTests").build(),
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
|
||||
null, Collections.emptyMap());
|
||||
null);
|
||||
return new GatewayService(settings.build(),
|
||||
null, clusterService, null, null, null, null);
|
||||
}
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
|
@ -53,7 +52,7 @@ public class ResponseCollectorServiceTests extends ESTestCase {
|
|||
threadpool = new TestThreadPool("response_collector_tests");
|
||||
clusterService = new ClusterService(Settings.EMPTY,
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
|
||||
threadpool, Collections.emptyMap());
|
||||
threadpool);
|
||||
collector = new ResponseCollectorService(Settings.EMPTY, clusterService);
|
||||
}
|
||||
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotR
|
|||
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStats;
|
||||
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus;
|
||||
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.support.ActiveShardCount;
|
||||
import org.elasticsearch.client.AdminClient;
|
||||
|
@ -40,8 +39,6 @@ import org.elasticsearch.client.node.NodeClient;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.NamedDiff;
|
||||
import org.elasticsearch.cluster.RestoreInProgress;
|
||||
import org.elasticsearch.cluster.SnapshotDeletionsInProgress;
|
||||
import org.elasticsearch.cluster.SnapshotsInProgress;
|
||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
|
@ -165,24 +162,6 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
|
|||
return Arrays.asList(MockRepository.Plugin.class, TestCustomMetaDataPlugin.class);
|
||||
}
|
||||
|
||||
public void testClusterStateHasCustoms() throws Exception {
|
||||
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().all().get();
|
||||
assertNotNull(clusterStateResponse.getState().custom(SnapshotsInProgress.TYPE));
|
||||
assertNotNull(clusterStateResponse.getState().custom(RestoreInProgress.TYPE));
|
||||
assertNotNull(clusterStateResponse.getState().custom(SnapshotDeletionsInProgress.TYPE));
|
||||
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||
if (randomBoolean()) {
|
||||
internalCluster().fullRestart();
|
||||
} else {
|
||||
internalCluster().rollingRestart();
|
||||
}
|
||||
|
||||
clusterStateResponse = client().admin().cluster().prepareState().all().get();
|
||||
assertNotNull(clusterStateResponse.getState().custom(SnapshotsInProgress.TYPE));
|
||||
assertNotNull(clusterStateResponse.getState().custom(RestoreInProgress.TYPE));
|
||||
assertNotNull(clusterStateResponse.getState().custom(SnapshotDeletionsInProgress.TYPE));
|
||||
}
|
||||
|
||||
public void testRestorePersistentSettings() throws Exception {
|
||||
logger.info("--> start 2 nodes");
|
||||
internalCluster().startNode();
|
||||
|
|
|
@ -132,7 +132,7 @@ public class ClusterServiceUtils {
|
|||
|
||||
public static ClusterService createClusterService(ThreadPool threadPool, DiscoveryNode localNode, ClusterSettings clusterSettings) {
|
||||
ClusterService clusterService = new ClusterService(Settings.builder().put("cluster.name", "ClusterServiceTests").build(),
|
||||
clusterSettings, threadPool, Collections.emptyMap());
|
||||
clusterSettings, threadPool);
|
||||
clusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) {
|
||||
@Override
|
||||
public void connectToNodes(DiscoveryNodes discoveryNodes) {
|
||||
|
|
|
@ -349,13 +349,6 @@ public class LocalStateCompositeXPackPlugin extends XPackPlugin implements Scrip
|
|||
filterPlugins(Plugin.class).stream().forEach(p -> p.onIndexModule(indexModule));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Supplier<ClusterState.Custom>> getInitialClusterStateCustomSupplier() {
|
||||
Map<String, Supplier<ClusterState.Custom>> suppliers = new HashMap<>();
|
||||
filterPlugins(ClusterPlugin.class).stream().forEach(p -> suppliers.putAll(p.getInitialClusterStateCustomSupplier()));
|
||||
return suppliers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<String, Predicate<String>> getFieldFilter() {
|
||||
List<Function<String, Predicate<String>>> items = filterPlugins(MapperPlugin.class).stream().map(p ->
|
||||
|
|
Loading…
Reference in New Issue