浏览代码

Skip settings validation during desired nodes updates (#99946)

This commit skips settings validation during desired nodes updates.
The issue comes when a setting that needs to be validated depends
on a secure setting that cannot be read while the desired nodes are
updated. To avoid such issues, we'll skip the settings validations
completely.

Closes #96127
Francisco Fernández Castaño 2 年之前
父节点
当前提交
ae743e673d

+ 5 - 0
docs/changelog/99946.yaml

@@ -0,0 +1,5 @@
+pr: 99946
+summary: Skip settings validation during desired nodes updates
+area: Distributed
+type: bug
+issues: []

+ 34 - 0
rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.desired_nodes/10_basic.yml

@@ -329,6 +329,9 @@ teardown:
           - { settings: { node: { external_id: "instance-000187" } }, processors: 8.0, memory: "64gb", storage: "128gb", node_version: $es_version }
           - { settings: { node: { external_id: "instance-000187" } }, processors: 8.0, memory: "64gb", storage: "128gb", node_version: $es_version }
 ---
 ---
 "Test settings are validated":
 "Test settings are validated":
+  - skip:
+      version: "8.9.99 - "
+      reason: "We started skipping setting validations in 8.10"
   - do:
   - do:
       cluster.state: {}
       cluster.state: {}
 
 
@@ -352,6 +355,9 @@ teardown:
   - match: { error.suppressed.0.reason: "Failed to parse value [-1000] for setting [http.tcp.keep_idle] must be >= -1" }
   - match: { error.suppressed.0.reason: "Failed to parse value [-1000] for setting [http.tcp.keep_idle] must be >= -1" }
 ---
 ---
 "Test unknown settings are forbidden in known versions":
 "Test unknown settings are forbidden in known versions":
+  - skip:
+      version: "8.9.99 - "
+      reason: "We started skipping setting validations in 8.10"
   - do:
   - do:
       cluster.state: {}
       cluster.state: {}
 
 
@@ -375,6 +381,9 @@ teardown:
   - match: { error.suppressed.0.reason: "unknown setting [unknown_setting] please check that any required plugins are installed, or check the breaking changes documentation for removed settings" }
   - match: { error.suppressed.0.reason: "unknown setting [unknown_setting] please check that any required plugins are installed, or check the breaking changes documentation for removed settings" }
 ---
 ---
 "Test unknown settings are allowed in future versions":
 "Test unknown settings are allowed in future versions":
+  - skip:
+      version: "8.9.99 - "
+      reason: "We started skipping setting validations in 8.10"
   - do:
   - do:
       _internal.update_desired_nodes:
       _internal.update_desired_nodes:
         history_id: "test"
         history_id: "test"
@@ -385,6 +394,9 @@ teardown:
   - match: { replaced_existing_history_id: false }
   - match: { replaced_existing_history_id: false }
 ---
 ---
 "Test some settings can be overridden":
 "Test some settings can be overridden":
+  - skip:
+      version: "8.9.99 - "
+      reason: "We started skipping setting validations in 8.10"
   - do:
   - do:
       cluster.state: {}
       cluster.state: {}
 
 
@@ -1154,3 +1166,25 @@ teardown:
             - { settings: { "node.name": "instance-000187" }, processors: 1.0, processors_range: {min: 1.0, max: 2.0}, memory: "64gb", storage: "128gb", node_version: $es_version }
             - { settings: { "node.name": "instance-000187" }, processors: 1.0, processors_range: {min: 1.0, max: 2.0}, memory: "64gb", storage: "128gb", node_version: $es_version }
   - match: { status: 400 }
   - match: { status: 400 }
   - match: { error.type: x_content_parse_exception }
   - match: { error.type: x_content_parse_exception }
+---
+"Test node roles are validated":
+  - do:
+      cluster.state: {}
+
+  # Get master node id
+  - set: { master_node: master }
+
+  - do:
+      nodes.info: {}
+  - set: { nodes.$master.version: es_version }
+
+  - do:
+      catch: bad_request
+      _internal.update_desired_nodes:
+        history_id: "test"
+        version: 1
+        body:
+          nodes:
+            - { settings: { "node.name": "instance-000187", "node.roles": "unknown,other" }, processors: 8.5, memory: "64gb", storage: "128gb", node_version: $es_version }
+  - match: { status: 400 }
+  - match: { error.type: x_content_parse_exception }

+ 3 - 0
rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.desired_nodes/20_dry_run.yml

@@ -87,6 +87,9 @@ teardown:
           - { settings: { node: { name: "instance-000187" } }, processors: 8.5, memory: "64gb", storage: "128gb", node_version: $es_version }
           - { settings: { node: { name: "instance-000187" } }, processors: 8.5, memory: "64gb", storage: "128gb", node_version: $es_version }
 ---
 ---
 "Test validation works for dry run updates":
 "Test validation works for dry run updates":
+  - skip:
+      version: "8.9.99 - "
+      reason: "We started skipping setting validations in 8.10"
   - do:
   - do:
       cluster.state: { }
       cluster.state: { }
 
 

+ 35 - 122
server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportDesiredNodesActionsIT.java

@@ -29,13 +29,11 @@ import org.junit.After;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.List;
 import java.util.List;
-import java.util.Locale;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
 import static org.elasticsearch.cluster.metadata.DesiredNodesTestCase.randomDesiredNode;
 import static org.elasticsearch.cluster.metadata.DesiredNodesTestCase.randomDesiredNode;
 import static org.elasticsearch.common.util.concurrent.EsExecutors.NODE_PROCESSORS_SETTING;
 import static org.elasticsearch.common.util.concurrent.EsExecutors.NODE_PROCESSORS_SETTING;
-import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_TCP_KEEP_IDLE;
 import static org.elasticsearch.node.NodeRoleSettings.NODE_ROLES_SETTING;
 import static org.elasticsearch.node.NodeRoleSettings.NODE_ROLES_SETTING;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.containsString;
@@ -83,19 +81,6 @@ public class TransportDesiredNodesActionsIT extends ESIntegTestCase {
         assertEquals(getLatestDesiredNodes(), desiredNodes);
         assertEquals(getLatestDesiredNodes(), desiredNodes);
     }
     }
 
 
-    public void testSettingsAreValidatedWithDryRun() {
-        var exception = expectThrows(
-            IllegalArgumentException.class,
-            () -> updateDesiredNodes(
-                randomDryRunUpdateDesiredNodesRequest(
-                    Version.CURRENT,
-                    Settings.builder().put(SETTING_HTTP_TCP_KEEP_IDLE.getKey(), Integer.MIN_VALUE).build()
-                )
-            )
-        );
-        assertThat(exception.getMessage(), containsString("contain invalid settings"));
-    }
-
     public void testUpdateDesiredNodesIsIdempotent() {
     public void testUpdateDesiredNodesIsIdempotent() {
         final var updateDesiredNodesRequest = randomUpdateDesiredNodesRequest();
         final var updateDesiredNodesRequest = randomUpdateDesiredNodesRequest();
         updateDesiredNodes(updateDesiredNodesRequest);
         updateDesiredNodes(updateDesiredNodesRequest);
@@ -195,52 +180,6 @@ public class TransportDesiredNodesActionsIT extends ESIntegTestCase {
         }
         }
     }
     }
 
 
-    public void testSettingsAreValidated() {
-        final var updateDesiredNodesRequest = randomUpdateDesiredNodesRequest(
-            Settings.builder().put(SETTING_HTTP_TCP_KEEP_IDLE.getKey(), Integer.MIN_VALUE).build()
-        );
-
-        final IllegalArgumentException exception = expectThrows(
-            IllegalArgumentException.class,
-            () -> updateDesiredNodes(updateDesiredNodesRequest)
-        );
-        assertThat(exception.getMessage(), containsString("Nodes with ids"));
-        assertThat(exception.getMessage(), containsString("contain invalid settings"));
-        assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
-        assertThat(
-            exception.getSuppressed()[0].getMessage(),
-            containsString("Failed to parse value [-2147483648] for setting [http.tcp.keep_idle] must be >= -1")
-        );
-    }
-
-    public void testNodeVersionIsValidated() {
-        final var updateDesiredNodesRequest = randomUpdateDesiredNodesRequest(Version.CURRENT.previousMajor(), Settings.EMPTY);
-
-        final IllegalArgumentException exception = expectThrows(
-            IllegalArgumentException.class,
-            () -> updateDesiredNodes(updateDesiredNodesRequest)
-        );
-        assertThat(exception.getMessage(), containsString("Nodes with ids"));
-        assertThat(exception.getMessage(), containsString("contain invalid settings"));
-        assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
-        assertThat(exception.getSuppressed()[0].getMessage(), containsString("Illegal node version"));
-    }
-
-    public void testUnknownSettingsAreForbiddenInKnownVersions() {
-        final var updateDesiredNodesRequest = randomUpdateDesiredNodesRequest(
-            Settings.builder().put("desired_nodes.random_setting", Integer.MIN_VALUE).build()
-        );
-
-        final IllegalArgumentException exception = expectThrows(
-            IllegalArgumentException.class,
-            () -> updateDesiredNodes(updateDesiredNodesRequest)
-        );
-        assertThat(exception.getMessage(), containsString("Nodes with ids"));
-        assertThat(exception.getMessage(), containsString("contain invalid settings"));
-        assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
-        assertThat(exception.getSuppressed()[0].getMessage(), containsString("unknown setting [desired_nodes.random_setting]"));
-    }
-
     public void testUnknownSettingsAreAllowedInFutureVersions() {
     public void testUnknownSettingsAreAllowedInFutureVersions() {
         final var updateDesiredNodesRequest = randomUpdateDesiredNodesRequest(
         final var updateDesiredNodesRequest = randomUpdateDesiredNodesRequest(
             Version.fromString("99.9.0"),
             Version.fromString("99.9.0"),
@@ -255,74 +194,48 @@ public class TransportDesiredNodesActionsIT extends ESIntegTestCase {
 
 
     public void testNodeProcessorsGetValidatedWithDesiredNodeProcessors() {
     public void testNodeProcessorsGetValidatedWithDesiredNodeProcessors() {
         final int numProcessors = Math.max(Runtime.getRuntime().availableProcessors() + 1, 2048);
         final int numProcessors = Math.max(Runtime.getRuntime().availableProcessors() + 1, 2048);
-
-        {
-            final var updateDesiredNodesRequest = new UpdateDesiredNodesRequest(
-                UUIDs.randomBase64UUID(),
-                randomIntBetween(1, 20),
-                randomList(
-                    1,
-                    20,
-                    () -> randomDesiredNode(
-                        Version.CURRENT,
-                        Settings.builder().put(NODE_PROCESSORS_SETTING.getKey(), numProcessors + 1).build(),
-                        numProcessors
-                    )
-                ),
-                false
-            );
-
-            final IllegalArgumentException exception = expectThrows(
-                IllegalArgumentException.class,
-                () -> updateDesiredNodes(updateDesiredNodesRequest)
-            );
-            assertThat(exception.getMessage(), containsString("Nodes with ids"));
-            assertThat(exception.getMessage(), containsString("contain invalid settings"));
-            assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
-            assertThat(
-                exception.getSuppressed()[0].getMessage(),
-                containsString(
-                    String.format(
-                        Locale.ROOT,
-                        "Failed to parse value [%d] for setting [node.processors] must be <= %d",
-                        numProcessors + 1,
-                        numProcessors
-                    )
+        // This test verifies that the validation doesn't throw on desired nodes
+        // with a higher number of available processors than the node running the tests.
+        final var updateDesiredNodesRequest = new UpdateDesiredNodesRequest(
+            UUIDs.randomBase64UUID(),
+            randomIntBetween(1, 20),
+            randomList(
+                1,
+                20,
+                () -> randomDesiredNode(
+                    Version.CURRENT,
+                    Settings.builder().put(NODE_PROCESSORS_SETTING.getKey(), numProcessors).build(),
+                    numProcessors
                 )
                 )
-            );
-        }
-
-        {
-            // This test verifies that the validation doesn't throw on desired nodes
-            // with a higher number of available processors than the node running the tests.
-            final var updateDesiredNodesRequest = new UpdateDesiredNodesRequest(
-                UUIDs.randomBase64UUID(),
-                randomIntBetween(1, 20),
-                randomList(
-                    1,
-                    20,
-                    () -> randomDesiredNode(
-                        Version.CURRENT,
-                        Settings.builder().put(NODE_PROCESSORS_SETTING.getKey(), numProcessors).build(),
-                        numProcessors
-                    )
-                ),
-                false
-            );
+            ),
+            false
+        );
 
 
-            updateDesiredNodes(updateDesiredNodesRequest);
+        updateDesiredNodes(updateDesiredNodesRequest);
 
 
-            final DesiredNodes latestDesiredNodes = getLatestDesiredNodes();
-            assertStoredDesiredNodesAreCorrect(updateDesiredNodesRequest, latestDesiredNodes);
+        final DesiredNodes latestDesiredNodes = getLatestDesiredNodes();
+        assertStoredDesiredNodesAreCorrect(updateDesiredNodesRequest, latestDesiredNodes);
 
 
-            assertThat(latestDesiredNodes.nodes().isEmpty(), is(equalTo(false)));
-            for (final var desiredNodeWithStatus : latestDesiredNodes) {
-                final var desiredNode = desiredNodeWithStatus.desiredNode();
-                assertThat(desiredNode.settings().get(NODE_PROCESSORS_SETTING.getKey()), is(equalTo(Integer.toString(numProcessors))));
-            }
+        assertThat(latestDesiredNodes.nodes().isEmpty(), is(equalTo(false)));
+        for (final var desiredNodeWithStatus : latestDesiredNodes) {
+            final var desiredNode = desiredNodeWithStatus.desiredNode();
+            assertThat(desiredNode.settings().get(NODE_PROCESSORS_SETTING.getKey()), is(equalTo(Integer.toString(numProcessors))));
         }
         }
     }
     }
 
 
+    public void testNodeVersionIsValidated() {
+        final var updateDesiredNodesRequest = randomUpdateDesiredNodesRequest(Version.CURRENT.previousMajor(), Settings.EMPTY);
+
+        final IllegalArgumentException exception = expectThrows(
+            IllegalArgumentException.class,
+            () -> updateDesiredNodes(updateDesiredNodesRequest)
+        );
+        assertThat(exception.getMessage(), containsString("Nodes with ids"));
+        assertThat(exception.getMessage(), containsString("contain invalid settings"));
+        assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
+        assertThat(exception.getSuppressed()[0].getMessage(), containsString("Illegal node version"));
+    }
+
     public void testUpdateDesiredNodesTasksAreBatchedCorrectly() throws Exception {
     public void testUpdateDesiredNodesTasksAreBatchedCorrectly() throws Exception {
         final Runnable unblockClusterStateUpdateThread = blockClusterStateUpdateThread();
         final Runnable unblockClusterStateUpdateThread = blockClusterStateUpdateThread();
 
 

+ 15 - 59
server/src/main/java/org/elasticsearch/cluster/desirednodes/DesiredNodesSettingsValidator.java

@@ -11,9 +11,6 @@ package org.elasticsearch.cluster.desirednodes;
 import org.elasticsearch.Build;
 import org.elasticsearch.Build;
 import org.elasticsearch.Version;
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.metadata.DesiredNode;
 import org.elasticsearch.cluster.metadata.DesiredNode;
-import org.elasticsearch.common.settings.ClusterSettings;
-import org.elasticsearch.common.settings.Setting;
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.Nullable;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -22,25 +19,29 @@ import java.util.Locale;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 
 
 import static java.lang.String.format;
 import static java.lang.String.format;
-import static org.elasticsearch.common.util.concurrent.EsExecutors.NODE_PROCESSORS_SETTING;
 
 
 public class DesiredNodesSettingsValidator {
 public class DesiredNodesSettingsValidator {
     private record DesiredNodeValidationError(int position, @Nullable String externalId, RuntimeException exception) {}
     private record DesiredNodeValidationError(int position, @Nullable String externalId, RuntimeException exception) {}
 
 
-    private final ClusterSettings clusterSettings;
-
-    public DesiredNodesSettingsValidator(ClusterSettings clusterSettings) {
-        this.clusterSettings = clusterSettings;
-    }
-
     public void validate(List<DesiredNode> nodes) {
     public void validate(List<DesiredNode> nodes) {
         final List<DesiredNodeValidationError> validationErrors = new ArrayList<>();
         final List<DesiredNodeValidationError> validationErrors = new ArrayList<>();
         for (int i = 0; i < nodes.size(); i++) {
         for (int i = 0; i < nodes.size(); i++) {
             final DesiredNode node = nodes.get(i);
             final DesiredNode node = nodes.get(i);
-            try {
-                validate(node);
-            } catch (IllegalArgumentException e) {
-                validationErrors.add(new DesiredNodeValidationError(i, node.externalId(), e));
+            if (node.version().before(Version.CURRENT)) {
+                validationErrors.add(
+                    new DesiredNodeValidationError(
+                        i,
+                        node.externalId(),
+                        new IllegalArgumentException(
+                            format(
+                                Locale.ROOT,
+                                "Illegal node version [%s]. Only [%s] or newer versions are supported",
+                                node.version(),
+                                Build.current().version()
+                            )
+                        )
+                    )
+                );
             }
             }
         }
         }
 
 
@@ -68,49 +69,4 @@ public class DesiredNodesSettingsValidator {
         }
         }
     }
     }
 
 
-    private void validate(DesiredNode node) {
-        if (node.version().before(Version.CURRENT)) {
-            throw new IllegalArgumentException(
-                format(
-                    Locale.ROOT,
-                    "Illegal node version [%s]. Only [%s] or newer versions are supported",
-                    node.version(),
-                    Build.current().version()
-                )
-            );
-        }
-
-        // Validating settings for future versions can be unsafe:
-        // - If the legal range is upgraded in the newer version
-        // - If a new setting is used as the default value for a previous setting
-        // To avoid considering these as invalid settings,
-        // We just don't validate settings for versions in newer versions.
-        if (node.version().after(Version.CURRENT)) {
-            return;
-        }
-
-        Settings settings = node.settings();
-
-        // node.processors rely on the environment to define its ranges, in this case
-        // we create a new setting just to run the validations using the desired node
-        // number of available processors
-        if (settings.hasValue(NODE_PROCESSORS_SETTING.getKey())) {
-            int minProcessors = node.roundedDownMinProcessors();
-            Integer roundedUpMaxProcessors = node.roundedUpMaxProcessors();
-            int maxProcessors = roundedUpMaxProcessors == null ? minProcessors : roundedUpMaxProcessors;
-            Setting.doubleSetting(
-                NODE_PROCESSORS_SETTING.getKey(),
-                minProcessors,
-                Double.MIN_VALUE,
-                maxProcessors,
-                Setting.Property.NodeScope
-            ).get(settings);
-            final Settings.Builder updatedSettings = Settings.builder().put(settings);
-            updatedSettings.remove(NODE_PROCESSORS_SETTING.getKey());
-            settings = updatedSettings.build();
-        }
-
-        clusterSettings.validate(settings, true);
-    }
-
 }
 }

+ 1 - 3
server/src/main/java/org/elasticsearch/node/Node.java

@@ -1043,9 +1043,7 @@ public class Node implements Closeable {
             clusterService.addListener(pluginShutdownService);
             clusterService.addListener(pluginShutdownService);
 
 
             final RecoveryPlannerService recoveryPlannerService = getRecoveryPlannerService(threadPool, clusterService, repositoryService);
             final RecoveryPlannerService recoveryPlannerService = getRecoveryPlannerService(threadPool, clusterService, repositoryService);
-            final DesiredNodesSettingsValidator desiredNodesSettingsValidator = new DesiredNodesSettingsValidator(
-                clusterService.getClusterSettings()
-            );
+            final DesiredNodesSettingsValidator desiredNodesSettingsValidator = new DesiredNodesSettingsValidator();
 
 
             final MasterHistoryService masterHistoryService = new MasterHistoryService(transportService, threadPool, clusterService);
             final MasterHistoryService masterHistoryService = new MasterHistoryService(transportService, threadPool, clusterService);
             final CoordinationDiagnosticsService coordinationDiagnosticsService = new CoordinationDiagnosticsService(
             final CoordinationDiagnosticsService coordinationDiagnosticsService = new CoordinationDiagnosticsService(

+ 1 - 37
server/src/test/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportUpdateDesiredNodesActionTests.java

@@ -9,7 +9,6 @@
 package org.elasticsearch.action.admin.cluster.desirednodes;
 package org.elasticsearch.action.admin.cluster.desirednodes;
 
 
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.ActionFilters;
-import org.elasticsearch.action.support.PlainActionFuture;
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlockException;
@@ -26,7 +25,6 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.routing.allocation.AllocationService;
 import org.elasticsearch.cluster.routing.allocation.AllocationService;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.cluster.service.ClusterService;
-import org.elasticsearch.tasks.Task;
 import org.elasticsearch.test.MockUtils;
 import org.elasticsearch.test.MockUtils;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 import org.elasticsearch.transport.TransportService;
@@ -42,14 +40,11 @@ import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.notNullValue;
 import static org.hamcrest.Matchers.notNullValue;
 import static org.hamcrest.Matchers.nullValue;
 import static org.hamcrest.Matchers.nullValue;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
 
 
 public class TransportUpdateDesiredNodesActionTests extends DesiredNodesTestCase {
 public class TransportUpdateDesiredNodesActionTests extends DesiredNodesTestCase {
 
 
-    public static final DesiredNodesSettingsValidator NO_OP_SETTINGS_VALIDATOR = new DesiredNodesSettingsValidator(null) {
+    public static final DesiredNodesSettingsValidator NO_OP_SETTINGS_VALIDATOR = new DesiredNodesSettingsValidator() {
         @Override
         @Override
         public void validate(List<DesiredNode> desiredNodes) {}
         public void validate(List<DesiredNode> desiredNodes) {}
     };
     };
@@ -100,37 +95,6 @@ public class TransportUpdateDesiredNodesActionTests extends DesiredNodesTestCase
         assertThat(e, is(nullValue()));
         assertThat(e, is(nullValue()));
     }
     }
 
 
-    public void testSettingsGetValidated() throws Exception {
-        DesiredNodesSettingsValidator validator = new DesiredNodesSettingsValidator(null) {
-            @Override
-            public void validate(List<DesiredNode> desiredNodes) {
-                throw new IllegalArgumentException("Invalid settings");
-            }
-        };
-        ClusterService clusterService = mock(ClusterService.class);
-
-        ThreadPool threadPool = mock(ThreadPool.class);
-        TransportService transportService = MockUtils.setupTransportServiceWithThreadpoolExecutor(threadPool);
-        final TransportUpdateDesiredNodesAction action = new TransportUpdateDesiredNodesAction(
-            transportService,
-            clusterService,
-            threadPool,
-            mock(ActionFilters.class),
-            mock(IndexNameExpressionResolver.class),
-            validator,
-            mock(AllocationService.class)
-        );
-
-        final ClusterState state = ClusterState.builder(new ClusterName(randomAlphaOfLength(10))).build();
-
-        final PlainActionFuture<UpdateDesiredNodesResponse> future = PlainActionFuture.newFuture();
-        action.masterOperation(mock(Task.class), randomUpdateDesiredNodesRequest(), state, future);
-        IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, future::actionGet);
-        assertThat(exception.getMessage(), containsString("Invalid settings"));
-
-        verify(clusterService, never()).submitUnbatchedStateUpdateTask(any(), any());
-    }
-
     public void testUpdateDesiredNodes() {
     public void testUpdateDesiredNodes() {
         final Metadata.Builder metadataBuilder = Metadata.builder();
         final Metadata.Builder metadataBuilder = Metadata.builder();
         boolean containsDesiredNodes = false;
         boolean containsDesiredNodes = false;

+ 1 - 106
server/src/test/java/org/elasticsearch/cluster/desirednodes/DesiredNodesSettingsValidatorTests.java

@@ -10,58 +10,21 @@ package org.elasticsearch.cluster.desirednodes;
 
 
 import org.elasticsearch.Version;
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.metadata.DesiredNode;
 import org.elasticsearch.cluster.metadata.DesiredNode;
-import org.elasticsearch.common.settings.ClusterSettings;
-import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.ESTestCase;
 
 
-import java.util.Collections;
 import java.util.List;
 import java.util.List;
-import java.util.Set;
 
 
 import static org.elasticsearch.cluster.metadata.DesiredNodesTestCase.randomDesiredNode;
 import static org.elasticsearch.cluster.metadata.DesiredNodesTestCase.randomDesiredNode;
-import static org.elasticsearch.common.util.concurrent.EsExecutors.NODE_PROCESSORS_SETTING;
-import static org.elasticsearch.node.Node.NODE_EXTERNAL_ID_SETTING;
-import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.is;
 
 
 public class DesiredNodesSettingsValidatorTests extends ESTestCase {
 public class DesiredNodesSettingsValidatorTests extends ESTestCase {
-
-    public void testSettingsValidation() {
-        final Set<Setting<?>> availableSettings = Set.of(
-            Setting.intSetting("test.invalid_value", 1, Setting.Property.NodeScope),
-            Setting.intSetting("test.invalid_range", 1, 1, 100, Setting.Property.NodeScope),
-            NODE_EXTERNAL_ID_SETTING,
-            NODE_NAME_SETTING
-        );
-        final Settings.Builder settings = Settings.builder();
-
-        if (randomBoolean()) {
-            settings.put("test.invalid_value", randomAlphaOfLength(10));
-        } else {
-            settings.put("test.invalid_range", randomFrom(-1, Integer.MAX_VALUE));
-        }
-
-        final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, availableSettings);
-        final DesiredNodesSettingsValidator validator = new DesiredNodesSettingsValidator(clusterSettings);
-
-        final List<DesiredNode> desiredNodes = randomList(2, 10, () -> randomDesiredNode(Version.CURRENT, settings.build()));
-
-        IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> validator.validate(desiredNodes));
-        assertThat(exception.getMessage(), containsString("Nodes with ids"));
-        assertThat(exception.getMessage(), containsString("contain invalid settings"));
-        assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
-        assertThat(exception.getSuppressed()[0].getMessage(), containsString("Failed to parse value"));
-    }
-
     public void testNodeVersionValidation() {
     public void testNodeVersionValidation() {
         final List<DesiredNode> desiredNodes = List.of(randomDesiredNode(Version.CURRENT.previousMajor(), Settings.EMPTY));
         final List<DesiredNode> desiredNodes = List.of(randomDesiredNode(Version.CURRENT.previousMajor(), Settings.EMPTY));
 
 
-        final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, Collections.emptySet());
-        final DesiredNodesSettingsValidator validator = new DesiredNodesSettingsValidator(clusterSettings);
+        final DesiredNodesSettingsValidator validator = new DesiredNodesSettingsValidator();
 
 
         final IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> validator.validate(desiredNodes));
         final IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> validator.validate(desiredNodes));
         assertThat(exception.getMessage(), containsString("Nodes with ids"));
         assertThat(exception.getMessage(), containsString("Nodes with ids"));
@@ -69,72 +32,4 @@ public class DesiredNodesSettingsValidatorTests extends ESTestCase {
         assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
         assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
         assertThat(exception.getSuppressed()[0].getMessage(), containsString("Illegal node version"));
         assertThat(exception.getSuppressed()[0].getMessage(), containsString("Illegal node version"));
     }
     }
-
-    public void testUnknownSettingsInKnownVersionsAreInvalid() {
-        final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, Collections.emptySet());
-        final DesiredNodesSettingsValidator validator = new DesiredNodesSettingsValidator(clusterSettings);
-        final List<DesiredNode> desiredNodes = randomList(2, 10, () -> randomDesiredNode(Version.CURRENT, Settings.EMPTY));
-
-        IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> validator.validate(desiredNodes));
-        assertThat(exception.getMessage(), containsString("Nodes with ids"));
-        assertThat(exception.getMessage(), containsString("contain invalid settings"));
-        assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
-        assertThat(exception.getSuppressed()[0].getMessage(), containsString("unknown setting"));
-    }
-
-    public void testUnknownSettingsInFutureVersionsAreNotValidated() {
-        final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, Collections.emptySet());
-        final DesiredNodesSettingsValidator validator = new DesiredNodesSettingsValidator(clusterSettings);
-
-        final List<DesiredNode> desiredNodes = randomList(
-            1,
-            10,
-            () -> randomDesiredNode(
-                Version.fromString("99.9.0"),
-                Settings.builder().put(randomAlphaOfLength(10), randomAlphaOfLength(10)).build()
-            )
-        );
-        validator.validate(desiredNodes);
-    }
-
-    public void testNodeProcessorsValidation() {
-        final Set<Setting<?>> availableSettings = Set.of(NODE_PROCESSORS_SETTING, NODE_EXTERNAL_ID_SETTING, NODE_NAME_SETTING);
-
-        final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, availableSettings);
-        final DesiredNodesSettingsValidator validator = new DesiredNodesSettingsValidator(clusterSettings);
-
-        {
-            int desiredNodeProcessors = 128;
-            Settings nodeSettings = Settings.builder()
-                .put(NODE_EXTERNAL_ID_SETTING.getKey(), randomAlphaOfLength(10))
-                .put(NODE_PROCESSORS_SETTING.getKey(), desiredNodeProcessors)
-                .build();
-            final List<DesiredNode> desiredNodes = List.of(
-                new DesiredNode(nodeSettings, desiredNodeProcessors, ByteSizeValue.ofGb(1), ByteSizeValue.ofGb(1), Version.CURRENT)
-            );
-
-            validator.validate(desiredNodes);
-        }
-
-        {
-            int desiredNodeProcessors = 128;
-            Settings nodeSettings = Settings.builder()
-                .put(NODE_EXTERNAL_ID_SETTING.getKey(), randomAlphaOfLength(10))
-                .put(NODE_PROCESSORS_SETTING.getKey(), desiredNodeProcessors + 1.1)
-                .build();
-            final List<DesiredNode> desiredNodes = List.of(
-                new DesiredNode(nodeSettings, desiredNodeProcessors, ByteSizeValue.ofGb(1), ByteSizeValue.ofGb(1), Version.CURRENT)
-            );
-
-            final IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> validator.validate(desiredNodes));
-            assertThat(exception.getMessage(), containsString("Nodes with ids"));
-            assertThat(exception.getMessage(), containsString("contain invalid settings"));
-            assertThat(exception.getSuppressed().length > 0, is(equalTo(true)));
-            assertThat(
-                exception.getSuppressed()[0].getMessage(),
-                containsString("Failed to parse value [129.1] for setting [node.processors] must be <= 128.0")
-            );
-        }
-
-    }
 }
 }