You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ja...@apache.org on 2023/11/11 18:50:28 UTC

(solr) branch main updated: SOLR-16975: Remove loading of solr.xml from zookeeper (main branch only) (#1920)

This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/main by this push:
     new 1e684380e87 SOLR-16975: Remove loading of solr.xml from zookeeper (main branch only) (#1920)
1e684380e87 is described below

commit 1e684380e873e7a25220925e0885d6ee1267d878
Author: Jan Høydahl <ja...@apache.org>
AuthorDate: Sat Nov 11 19:50:22 2023 +0100

    SOLR-16975: Remove loading of solr.xml from zookeeper (main branch only) (#1920)
    
    
    ---------
    
    Co-authored-by: Jan Høydahl <ja...@users.noreply.github.com>
---
 dev-tools/scripts/cloud.sh                         |   1 -
 solr/CHANGES.txt                                   |   2 +
 .../core/src/java/org/apache/solr/cloud/ZkCLI.java |   8 +-
 .../java/org/apache/solr/cloud/ZkController.java   |  13 +-
 .../src/java/org/apache/solr/core/NodeConfig.java  |  82 ++-------
 .../java/org/apache/solr/core/SolrXmlConfig.java   |  23 +--
 .../src/java/org/apache/solr/core/ZkContainer.java |   8 +-
 .../org/apache/solr/cloud/SolrXmlInZkTest.java     | 192 ---------------------
 .../src/test/org/apache/solr/cloud/ZkCLITest.java  |   6 +-
 .../security/hadoop/TestZkAclsWithHadoopAuth.java  |   1 -
 .../pages/configuring-solr-xml.adoc                |   6 +-
 .../pages/major-changes-in-solr-10.adoc            |   2 +
 .../apache/solr/common/cloud/ZkStateReader.java    |   3 +
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |  19 +-
 .../solr/cloud/MiniSolrCloudClusterTest.java       |  16 +-
 15 files changed, 68 insertions(+), 314 deletions(-)

diff --git a/dev-tools/scripts/cloud.sh b/dev-tools/scripts/cloud.sh
index 3b5e3bfa13c..01549ddeeb8 100755
--- a/dev-tools/scripts/cloud.sh
+++ b/dev-tools/scripts/cloud.sh
@@ -315,7 +315,6 @@ start(){
     # Need a fresh root in zookeeper...
     "${SOLR}/server/scripts/cloud-scripts/zkcli.sh" -zkhost localhost:${ZK_PORT} -cmd makepath "/solr_${SAFE_DEST}";
     "${SOLR}/server/scripts/cloud-scripts/zkcli.sh" -zkhost localhost:${ZK_PORT} -cmd put "/solr_${SAFE_DEST}" "created by cloud.sh"; # so we can test for existence next time
-    "${SOLR}/server/scripts/cloud-scripts/zkcli.sh" -zkhost localhost:${ZK_PORT} -cmd putfile "/solr_${SAFE_DEST}/solr.xml" "${SOLR}/server/solr/solr.xml";
   fi
 
   ACTUAL_NUM_NODES=$(ls -1 -d ${CLUSTER_WD}/n* | wc -l )
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7c703fb434f..8335ed14fdc 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -43,6 +43,8 @@ Deprecation Removals
 
 * SOLR-16893: Remove bin/solr create_core and create_collection commands in favour of bin/solr create command.  (Eric Pugh)
 
+* SOLR-15959: Remove deprecated feature to load solr.xml from ZK (janhoy)
+
 * SOLR-17042: Remove deprecated `V2RequestSupport` and associated `SolrRequest` methods `setUseV2` and `setUseBinaryV2`. (Jason Gerlowski)
 
 Dependency Upgrades
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java b/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
index a8d0a5f24c7..39c8a95169b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
@@ -225,10 +225,12 @@ public class ZkCLI implements CLIO {
         stdout.println(
             "zkcli.sh -zkhost localhost:9983 -cmd "
                 + PUT_FILE
-                + " /solr.xml /User/myuser/solr/solr.xml");
-        stdout.println("zkcli.sh -zkhost localhost:9983 -cmd " + GET + " /solr.xml");
+                + " /clusterprops.json /User/myuser/solr/clusterprops.json");
+        stdout.println("zkcli.sh -zkhost localhost:9983 -cmd " + GET + " /clusterprops.json");
         stdout.println(
-            "zkcli.sh -zkhost localhost:9983 -cmd " + GET_FILE + " /solr.xml solr.xml.file");
+            "zkcli.sh -zkhost localhost:9983 -cmd "
+                + GET_FILE
+                + " /clusterprops.json clusterprops.json");
         stdout.println("zkcli.sh -zkhost localhost:9983 -cmd " + CLEAR + " /solr");
         stdout.println("zkcli.sh -zkhost localhost:9983 -cmd " + LIST);
         stdout.println("zkcli.sh -zkhost localhost:9983 -cmd " + LS + " /solr/live_nodes");
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 4466ea0b3aa..5fee6fe77e4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -24,6 +24,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.UNSUPPORTED_SOLR_XML;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
 import static org.apache.zookeeper.ZooDefs.Ids.OPEN_ACL_UNSAFE;
 
@@ -376,7 +377,7 @@ public class ZkController implements Closeable {
             .withClosedCheck(cc::isShutDown)
             .withCompressor(compressor)
             .build();
-    // Refuse to start if ZK has a non empty /clusterstate.json
+    // Refuse to start if ZK has a non empty /clusterstate.json or a /solr.xml file
     checkNoOldClusterstate(zkClient);
 
     this.overseerRunningMap = Overseer.getRunningMap(zkClient);
@@ -529,12 +530,20 @@ public class ZkController implements Closeable {
 
   /**
    * Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses
-   * to start and outputs a helpful message regarding collection migration.
+   * to start and outputs a helpful message regarding collection migration. Also aborts if /solr.xml
+   * exists in zookeeper.
    *
    * <p>If /clusterstate.json exists and is empty, it is removed.
    */
   private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
     try {
+      if (zkClient.exists(UNSUPPORTED_SOLR_XML, true)) {
+        String message =
+            "solr.xml found in ZooKeeper. Loading solr.xml from ZooKeeper is no longer supported since Solr 10. "
+                + "Cannot start Solr. The file can be removed with command bin/solr zk rm /solr.xml -z host:port";
+        log.error(message);
+        throw new SolrException(ErrorCode.INVALID_STATE, message);
+      }
       if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
         return;
       }
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index 75e570b2235..0d83279c1c1 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.core;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
@@ -35,11 +34,11 @@ import java.util.function.Predicate;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.solr.client.solrj.impl.SolrZkClientTimeout;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.logging.DeprecationLog;
 import org.apache.solr.logging.LogWatcherConfig;
 import org.apache.solr.search.CacheConfig;
 import org.apache.solr.servlet.SolrDispatchFilter;
@@ -117,10 +116,6 @@ public class NodeConfig {
 
   private final PluginInfo tracerConfig;
 
-  // Track if this config was loaded from zookeeper so that we can skip validating the zookeeper
-  // connection later. If it becomes necessary to track multiple potential sources in the future,
-  // replace this with an Enum
-  private final boolean fromZookeeper;
   private final String defaultZkHost;
 
   private NodeConfig(
@@ -153,7 +148,6 @@ public class NodeConfig {
       MetricsConfig metricsConfig,
       Map<String, CacheConfig> cachesConfig,
       PluginInfo tracerConfig,
-      boolean fromZookeeper,
       String defaultZkHost,
       Set<Path> allowPaths,
       List<String> allowUrls,
@@ -191,7 +185,6 @@ public class NodeConfig {
     this.metricsConfig = metricsConfig;
     this.cachesConfig = cachesConfig == null ? Collections.emptyMap() : cachesConfig;
     this.tracerConfig = tracerConfig;
-    this.fromZookeeper = fromZookeeper;
     this.defaultZkHost = defaultZkHost;
     this.allowPaths = allowPaths;
     this.allowUrls = allowUrls;
@@ -218,23 +211,24 @@ public class NodeConfig {
   }
 
   /**
-   * Get the NodeConfig whether stored on disk, in ZooKeeper, etc. This may also be used by custom
-   * filters to load relevant configuration.
+   * Get the NodeConfig. This may also be used by custom filters to load relevant configuration.
    *
    * @return the NodeConfig
    */
   public static NodeConfig loadNodeConfig(Path solrHome, Properties nodeProperties) {
-    if (StrUtils.isNotNullOrEmpty(System.getProperty("solr.solrxml.location"))) {
-      log.warn(
-          "Solr property solr.solrxml.location is no longer supported. Will automatically load solr.xml from ZooKeeper if it exists");
-    }
     final SolrResourceLoader loader = new SolrResourceLoader(solrHome);
     initModules(loader, null);
     nodeProperties = SolrXmlConfig.wrapAndSetZkHostFromSysPropIfNeeded(nodeProperties);
+
+    // TODO: Only job of this block is to
+    //  delay starting a solr core to satisfy
+    //  ZkFailoverTest test case...
     String zkHost = nodeProperties.getProperty(SolrXmlConfig.ZK_HOST);
     if (StrUtils.isNotNullOrEmpty(zkHost)) {
-      int startUpZkTimeOut = Integer.getInteger("waitForZk", 30);
-      startUpZkTimeOut *= 1000;
+      int startUpZkTimeOut = 1000 * Integer.getInteger("waitForZk", 0);
+      if (startUpZkTimeOut == 0) {
+        startUpZkTimeOut = SolrZkClientTimeout.DEFAULT_ZK_CLIENT_TIMEOUT;
+      }
       try (SolrZkClient zkClient =
           new SolrZkClient.Builder()
               .withUrl(zkHost)
@@ -242,20 +236,11 @@ public class NodeConfig {
               .withConnTimeOut(startUpZkTimeOut, TimeUnit.MILLISECONDS)
               .withSolrClassLoader(loader)
               .build()) {
-        if (zkClient.exists("/solr.xml", true)) {
-          log.info("solr.xml found in ZooKeeper. Loading...");
-          DeprecationLog.log(
-              "solrxml-zookeeper",
-              "Loading solr.xml from zookeeper is deprecated. See reference guide for details.");
-          byte[] data = zkClient.getData("/solr.xml", null, null, true);
-          return SolrXmlConfig.fromInputStream(
-              solrHome, new ByteArrayInputStream(data), nodeProperties, true);
-        }
+        zkClient.exists("/configs", true);
       } catch (Exception e) {
         throw new SolrException(
-            ErrorCode.SERVER_ERROR, "Error occurred while loading solr.xml from zookeeper", e);
+            ErrorCode.SERVER_ERROR, "Error occurred while testing zookeeper connection", e);
       }
-      log.info("Loading solr.xml from SolrHome (not found in ZooKeeper)");
     }
 
     return SolrXmlConfig.fromSolrHome(solrHome, nodeProperties);
@@ -419,37 +404,6 @@ public class NodeConfig {
     return tracerConfig;
   }
 
-  /**
-   * True if this node config was loaded from zookeeper
-   *
-   * @see #getDefaultZkHost
-   */
-  public boolean isFromZookeeper() {
-    return fromZookeeper;
-  }
-
-  /**
-   * This method returns the default "zkHost" value for this node -- either read from the system
-   * properties, or from the "extra" properties configured explicitly on the SolrDispatchFilter; or
-   * null if not specified.
-   *
-   * <p>This is the value that would have been used when attempting to locate the solr.xml in
-   * ZooKeeper (regardless of whether the file was actually loaded from ZK or from local disk)
-   *
-   * <p>(This value should only be used for "accounting" purposes to track where the node config
-   * came from if it <em>was</em> loaded from zk -- ie: to check if the chroot has already been
-   * applied. It may be different from the "zkHost" <em>configured</em> in the "cloud" section of
-   * the solr.xml, which should be used for all zk connections made by this node to participate in
-   * the cluster)
-   *
-   * @see #isFromZookeeper
-   * @see #getCloudConfig()
-   * @see CloudConfig#getZkHost()
-   */
-  public String getDefaultZkHost() {
-    return defaultZkHost;
-  }
-
   /**
    * Extra file paths that will be allowed for core creation, in addition to SOLR_HOME,
    * SOLR_DATA_HOME and coreRootDir
@@ -498,11 +452,6 @@ public class NodeConfig {
     return modules;
   }
 
-  /** Returns the list of hidden system properties. The list values are regex expressions */
-  public Set<String> getHiddenSysProps() {
-    return hiddenSysProps;
-  }
-
   /** Returns whether a given system property is hidden */
   public boolean isSysPropHidden(String sysPropName) {
     return hiddenSysPropPattern.test(sysPropName);
@@ -627,7 +576,6 @@ public class NodeConfig {
     private MetricsConfig metricsConfig;
     private Map<String, CacheConfig> cachesConfig;
     private PluginInfo tracerConfig;
-    private boolean fromZookeeper = false;
     private String defaultZkHost;
     private Set<Path> allowPaths = Collections.emptySet();
     private List<String> allowUrls = Collections.emptyList();
@@ -814,11 +762,6 @@ public class NodeConfig {
       return this;
     }
 
-    public NodeConfigBuilder setFromZookeeper(boolean fromZookeeper) {
-      this.fromZookeeper = fromZookeeper;
-      return this;
-    }
-
     public NodeConfigBuilder setDefaultZkHost(String defaultZkHost) {
       this.defaultZkHost = defaultZkHost;
       return this;
@@ -929,7 +872,6 @@ public class NodeConfig {
           metricsConfig,
           cachesConfig,
           tracerConfig,
-          fromZookeeper,
           defaultZkHost,
           allowPaths,
           allowUrls,
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 36a2c13670c..989fa50a77f 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -95,22 +95,12 @@ public class SolrXmlConfig {
   }
 
   public static NodeConfig fromConfig(
-      Path solrHome,
-      Properties substituteProperties,
-      boolean fromZookeeper,
-      ConfigNode root,
-      SolrResourceLoader loader) {
+      Path solrHome, Properties substituteProperties, ConfigNode root, SolrResourceLoader loader) {
 
     checkForIllegalConfig(root);
 
-    // sanity check: if our config came from zookeeper, then there *MUST* be Node Properties that
-    // tell us what zkHost was used to read it (either via webapp context attribute, or that
-    // SolrDispatchFilter filled in for us from system properties)
-    assert ((!fromZookeeper)
-        || (null != substituteProperties && null != substituteProperties.getProperty(ZK_HOST)));
-
-    // Regardless of where/how we this XmlConfigFile was loaded from, if it contains a zkHost
-    // property, we're going to use that as our "default" and only *directly* check the system
+    // If solr.xml contains a zkHost property, we're going to use that as our "default" and only
+    // *directly* check the system
     // property if it's not specified.
     //
     // (checking the sys prop here is really just for tests that by-pass SolrDispatchFilter. In
@@ -179,7 +169,6 @@ public class SolrXmlConfig {
     configBuilder.setHiddenSysProps(getHiddenSysProps(root.get("metrics")));
     configBuilder.setMetricsConfig(getMetricsConfig(root.get("metrics")));
     configBuilder.setCachesConfig(getCachesConfig(loader, root.get("caches")));
-    configBuilder.setFromZookeeper(fromZookeeper);
     configBuilder.setDefaultZkHost(defaultZkHost);
     configBuilder.setCoreAdminHandlerActions(coreAdminHandlerActions);
     return fillSolrSection(configBuilder, root);
@@ -222,11 +211,6 @@ public class SolrXmlConfig {
 
   public static NodeConfig fromInputStream(
       Path solrHome, InputStream is, Properties substituteProps) {
-    return fromInputStream(solrHome, is, substituteProps, false);
-  }
-
-  public static NodeConfig fromInputStream(
-      Path solrHome, InputStream is, Properties substituteProps, boolean fromZookeeper) {
     SolrResourceLoader loader = new SolrResourceLoader(solrHome);
     if (substituteProps == null) {
       substituteProps = new Properties();
@@ -239,7 +223,6 @@ public class SolrXmlConfig {
         return fromConfig(
             solrHome,
             substituteProps,
-            fromZookeeper,
             new DataConfigNode(new DOMConfigNode(config.getDocument().getDocumentElement())),
             loader);
       }
diff --git a/solr/core/src/java/org/apache/solr/core/ZkContainer.java b/solr/core/src/java/org/apache/solr/core/ZkContainer.java
index f7ea012b249..02e6d4e628b 100644
--- a/solr/core/src/java/org/apache/solr/core/ZkContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/ZkContainer.java
@@ -127,13 +127,7 @@ public class ZkContainer {
         }
         boolean createRoot = Boolean.getBoolean("createZkChroot");
 
-        // We may have already loaded NodeConfig from zookeeper with same connect string, so no need
-        // to recheck chroot
-        boolean alreadyUsedChroot =
-            (cc.getConfig().isFromZookeeper()
-                && zookeeperHost.equals(cc.getConfig().getDefaultZkHost()));
-        if (!alreadyUsedChroot
-            && !ZkController.checkChrootPath(zookeeperHost, zkRunOnly || createRoot)) {
+        if (!ZkController.checkChrootPath(zookeeperHost, zkRunOnly || createRoot)) {
           throw new ZooKeeperException(
               SolrException.ErrorCode.SERVER_ERROR,
               "A chroot was specified in ZkHost but the znode doesn't exist. " + zookeeperHost);
diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java
deleted file mode 100644
index 868cd91e62a..00000000000
--- a/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF 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.apache.solr.cloud;
-
-import java.io.File;
-import java.lang.invoke.MethodHandles;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.core.NodeConfig;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class SolrXmlInZkTest extends SolrTestCaseJ4 {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  protected ZkTestServer zkServer;
-
-  protected Path zkDir;
-
-  private SolrZkClient zkClient;
-
-  private NodeConfig cfg;
-
-  private void setUpZkAndDiskXml(boolean toZk, boolean leaveOnLocal) throws Exception {
-    Path tmpDir = createTempDir();
-    Path solrHome = tmpDir.resolve("home");
-    copyMinConf(new File(solrHome.toFile(), "myCollect"));
-    if (leaveOnLocal) {
-      Files.copy(
-          Path.of(SolrTestCaseJ4.TEST_HOME(), "solr-stress-new.xml"), solrHome.resolve("solr.xml"));
-    }
-
-    ignoreException("No UpdateLog found - cannot sync");
-    ignoreException("No UpdateLog found - cannot recover");
-
-    System.setProperty("zkClientTimeout", "8000");
-
-    zkDir = tmpDir.resolve("zookeeper" + System.nanoTime()).resolve("server1").resolve("data");
-    zkServer = new ZkTestServer(zkDir);
-    zkServer.run();
-    System.setProperty("zkHost", zkServer.getZkAddress());
-    zkServer.buildZooKeeper("solrconfig.xml", "schema.xml");
-
-    zkClient =
-        new SolrZkClient.Builder()
-            .withUrl(zkServer.getZkAddress())
-            .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS)
-            .build();
-
-    if (toZk) {
-      zkClient.makePath("solr.xml", XML_FOR_ZK.getBytes(StandardCharsets.UTF_8), true);
-    }
-
-    zkClient.close();
-
-    if (log.isInfoEnabled()) {
-      log.info("####SETUP_START {}", getTestName());
-    }
-
-    // set some system properties for use by tests
-    Properties props = new Properties();
-    props.setProperty("solr.test.sys.prop1", "propone");
-    props.setProperty("solr.test.sys.prop2", "proptwo");
-
-    cfg = NodeConfig.loadNodeConfig(solrHome, props);
-    if (log.isInfoEnabled()) {
-      log.info("####SETUP_END {}", getTestName());
-    }
-  }
-
-  private void closeZK() throws Exception {
-    if (zkClient != null) {
-      zkClient.close();
-    }
-
-    zkServer.shutdown();
-  }
-
-  @Test
-  public void testXmlOnBoth() throws Exception {
-    try {
-      setUpZkAndDiskXml(true, true);
-      assertEquals(
-          "Should have gotten a new port the xml file sent to ZK, overrides the copy on disk",
-          cfg.getCloudConfig().getSolrHostPort(),
-          9045);
-    } finally {
-      closeZK();
-    }
-  }
-
-  @Test
-  public void testXmlInZkOnly() throws Exception {
-    try {
-      setUpZkAndDiskXml(true, false);
-      assertEquals(
-          "Should have gotten a new port the xml file sent to ZK",
-          cfg.getCloudConfig().getSolrHostPort(),
-          9045);
-    } finally {
-      closeZK();
-    }
-  }
-
-  @Test
-  public void testNotInZkFallbackLocal() throws Exception {
-    try {
-      setUpZkAndDiskXml(false, true);
-      assertEquals(
-          "Should have gotten the default port", cfg.getCloudConfig().getSolrHostPort(), 8983);
-    } finally {
-      closeZK();
-    }
-  }
-
-  @Test
-  public void testNotInZkOrOnDiskFallbackDefault() throws Exception {
-    try {
-      setUpZkAndDiskXml(false, false);
-      assertEquals(
-          "Should have gotten the default port", cfg.getCloudConfig().getSolrHostPort(), 8983);
-    } finally {
-      closeZK();
-    }
-  }
-
-  @Test
-  public void testNotInZkOrOnDiskWhenRequired() throws Exception {
-    try {
-      System.setProperty("solr.solrxml.required", "true");
-      SolrException e =
-          expectThrows(
-              SolrException.class,
-              () -> {
-                System.setProperty("hostPort", "8787");
-                setUpZkAndDiskXml(false, false); // solr.xml not on disk either
-              });
-      assertTrue(
-          "Should be failing to create default solr.xml in code",
-          e.getMessage().contains("solr.xml does not exist"));
-    } finally {
-      closeZK();
-      System.clearProperty("solr.solrxml.required");
-    }
-  }
-
-  @Test
-  public void testOnDiskOnly() throws Exception {
-    try {
-      setUpZkAndDiskXml(false, true);
-      assertEquals(
-          "Should have gotten the default port", cfg.getCloudConfig().getSolrHostPort(), 8983);
-    } finally {
-      closeZK();
-    }
-  }
-
-  // Just a random port, I'm not going to use it but just check that the Solr instance constructed
-  // from the XML file in ZK overrides the default port.
-  private static final String XML_FOR_ZK =
-      "<solr>"
-          + "  <solrcloud>"
-          + "    <str name=\"host\">127.0.0.1</str>"
-          + "    <int name=\"hostPort\">9045</int>"
-          + "  </solrcloud>"
-          + "  <shardHandlerFactory name=\"shardHandlerFactory\" class=\"HttpShardHandlerFactory\">"
-          + "    <int name=\"socketTimeout\">${socketTimeout:120000}</int>"
-          + "    <int name=\"connTimeout\">${connTimeout:15000}</int>"
-          + "  </shardHandlerFactory>"
-          + "</solr>";
-}
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
index c84d180a6b9..32170e545fc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
@@ -211,13 +211,13 @@ public class ZkCLITest extends SolrTestCaseJ4 {
           zkServer.getZkAddress(),
           "-cmd",
           "putfile",
-          "/solr.xml",
+          "/foo.xml",
           SOLR_HOME + File.separator + "solr-stress-new.xml"
         };
     ZkCLI.main(args);
 
     String fromZk =
-        new String(zkClient.getData("/solr.xml", null, null, true), StandardCharsets.UTF_8);
+        new String(zkClient.getData("/foo.xml", null, null, true), StandardCharsets.UTF_8);
     Path locFile = Path.of(SOLR_HOME, "solr-stress-new.xml");
     String fromLoc = Files.readString(locFile);
     assertEquals("Should get back what we put in ZK", fromZk, fromLoc);
@@ -255,7 +255,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
           zkServer.getZkAddress(),
           "-cmd",
           "putfile",
-          "/solr.xml",
+          "/foo.xml",
           SOLR_HOME + File.separator + "not-there.xml"
         };
     NoSuchFileException e = expectThrows(NoSuchFileException.class, () -> ZkCLI.main(args));
diff --git a/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java b/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
index f8b1fa029e3..074d39c51f6 100644
--- a/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
+++ b/solr/modules/hadoop-auth/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
@@ -98,7 +98,6 @@ public class TestZkAclsWithHadoopAuth extends SolrCloudTestCase {
       keeper.addAuthInfo("digest", ("solr:" + SOLR_PASSWD).getBytes(StandardCharsets.UTF_8));
 
       // Test well known paths.
-      checkNonSecurityACLs(keeper, "/solr.xml");
       checkSecurityACLs(keeper, "/security/token");
       checkSecurityACLs(keeper, "/security");
 
diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc b/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
index 65751c24c06..bc398b44e34 100644
--- a/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
+++ b/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
@@ -23,10 +23,8 @@ For details on how to configure `core.properties`, see the section xref:core-dis
 
 == Defining solr.xml
 
-You can find `solr.xml` in your `$SOLR_HOME` directory (usually `server/solr` or `/var/solr/data`) or optionally in ZooKeeper when using SolrCloud. If `$SOLR_HOME/solr.xml` is not found, Solr will use the default `solr.xml` file.
-
-WARNING: Loading `solr.xml` from Zookeeper is deprecated, and will not be supported in a future version.
-Being the node config of Solr, this file must be available at early startup and also be allowed to differ between nodes.
+You can find `solr.xml` in your `$SOLR_HOME` directory (usually `server/solr` or `/var/solr/data`).
+If `$SOLR_HOME/solr.xml` is not found, Solr will use the default `solr.xml` file.
 
 The default `solr.xml` file is found in `$SOLR_TIP/server/solr/solr.xml` and looks like this:
 
diff --git a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-10.adoc b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-10.adoc
index fed3ed14110..d12f894eda9 100644
--- a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-10.adoc
+++ b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-10.adoc
@@ -44,3 +44,5 @@ has been removed. Please use `-Dsolr.hiddenSysProps` or the envVar `SOLR_HIDDEN_
 * The `<hiddenSysProps>` solr.xml element under `<metrics>` has been removed. Instead use the `<hiddenSysProps>` tag under `<solr>`, which accepts a comma-separated string.
 Please see `-Dsolr.redaction.system.pattern`, which allows users to provide a pattern to match sysProps that should be redacted for sensitive information,
 has been removed. Please use `-Dsolr.hiddenSysProps` or the envVar `SOLR_HIDDEN_SYS_PROPS` instead.
+
+* The node configuration file `/solr.xml` can no longer be loaded from Zookeeper. Solr startup will fail if it is present.
\ No newline at end of file
diff --git a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index d6904d5bfcd..fffadfce6de 100644
--- a/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj-zookeeper/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -124,6 +124,9 @@ public class ZkStateReader implements SolrCloseable {
    */
   public static final String UNSUPPORTED_CLUSTER_STATE = "/clusterstate.json";
 
+  // This zookeeper file was allowed until Solr 10
+  public static final String UNSUPPORTED_SOLR_XML = "/solr.xml";
+
   public static final String CLUSTER_PROPS = "/clusterprops.json";
   public static final String COLLECTION_PROPS_ZKNODE = "collectionprops.json";
   public static final String REJOIN_AT_HEAD_PROP = "rejoinAtHead";
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index 6f78c06c018..b62412fd6b5 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -301,8 +301,8 @@ public class MiniSolrCloudCluster {
             .withUrl(zkServer.getZkHost())
             .withTimeout(AbstractZkTestCase.TIMEOUT, TimeUnit.MILLISECONDS)
             .build()) {
-      if (!zkClient.exists("/solr/solr.xml", true)) {
-        zkClient.makePath("/solr/solr.xml", solrXml.getBytes(Charset.defaultCharset()), true);
+      if (!zkClient.exists("/solr/initialized", true)) {
+        zkClient.makePath("/solr/initialized", "yes".getBytes(Charset.defaultCharset()), true);
         if (jettyConfig.sslConfig != null && jettyConfig.sslConfig.isSSLMode()) {
           zkClient.makePath(
               "/solr" + ZkStateReader.CLUSTER_PROPS,
@@ -318,7 +318,7 @@ public class MiniSolrCloudCluster {
 
     List<Callable<JettySolrRunner>> startups = new ArrayList<>(numServers);
     for (int i = 0; i < numServers; ++i) {
-      startups.add(() -> startJettySolrRunner(newNodeName(), jettyConfig));
+      startups.add(() -> startJettySolrRunner(newNodeName(), jettyConfig, solrXml));
     }
 
     final ExecutorService executorLauncher =
@@ -481,7 +481,8 @@ public class MiniSolrCloudCluster {
             .withServlets(extraServlets)
             .withFilters(extraRequestFilters)
             .withSSLConfig(sslConfig)
-            .build());
+            .build(),
+        null);
   }
 
   public JettySolrRunner getJettySolrRunner(int index) {
@@ -493,14 +494,20 @@ public class MiniSolrCloudCluster {
    *
    * @param name the instance name
    * @param config a JettyConfig for the instance's {@link org.apache.solr.embedded.JettySolrRunner}
+   * @param solrXml the string content of the solr.xml file to use, or null to just use default
    * @return a JettySolrRunner
    */
-  public JettySolrRunner startJettySolrRunner(String name, JettyConfig config) throws Exception {
+  public JettySolrRunner startJettySolrRunner(String name, JettyConfig config, String solrXml)
+      throws Exception {
     // tell solr node to look in zookeeper for solr.xml
     final Properties nodeProps = new Properties();
     nodeProps.setProperty("zkHost", zkServer.getZkAddress());
 
     Path runnerPath = createInstancePath(name);
+    if (solrXml == null) {
+      solrXml = DEFAULT_CLOUD_SOLR_XML;
+    }
+    Files.write(runnerPath.resolve("solr.xml"), solrXml.getBytes(StandardCharsets.UTF_8));
     JettyConfig newConfig = JettyConfig.builder(config).build();
     JettySolrRunner jetty =
         !trackJettyMetrics
@@ -520,7 +527,7 @@ public class MiniSolrCloudCluster {
    * @return a JettySolrRunner
    */
   public JettySolrRunner startJettySolrRunner() throws Exception {
-    return startJettySolrRunner(newNodeName(), jettyConfig);
+    return startJettySolrRunner(newNodeName(), jettyConfig, null);
   }
 
   /**
diff --git a/solr/test-framework/src/test/org/apache/solr/cloud/MiniSolrCloudClusterTest.java b/solr/test-framework/src/test/org/apache/solr/cloud/MiniSolrCloudClusterTest.java
index 49b68a134fb..6bf75b0b457 100644
--- a/solr/test-framework/src/test/org/apache/solr/cloud/MiniSolrCloudClusterTest.java
+++ b/solr/test-framework/src/test/org/apache/solr/cloud/MiniSolrCloudClusterTest.java
@@ -20,6 +20,8 @@ package org.apache.solr.cloud;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.Properties;
@@ -56,10 +58,10 @@ public class MiniSolrCloudClusterTest extends SolrTestCaseJ4 {
       cluster =
           new MiniSolrCloudCluster(3, createTempDir(), JettyConfig.builder().build()) {
             @Override
-            public JettySolrRunner startJettySolrRunner(String name, JettyConfig config)
-                throws Exception {
+            public JettySolrRunner startJettySolrRunner(
+                String name, JettyConfig config, String solrXml) throws Exception {
               if (jettyIndex.incrementAndGet() != 2)
-                return super.startJettySolrRunner(name, config);
+                return super.startJettySolrRunner(name, config, solrXml);
               throw new IOException("Fake exception on startup!");
             }
           };
@@ -199,14 +201,18 @@ public class MiniSolrCloudClusterTest extends SolrTestCaseJ4 {
       final MiniSolrCloudCluster x =
           new MiniSolrCloudCluster(1, createTempDir(), JettyConfig.builder().build()) {
             @Override
-            public JettySolrRunner startJettySolrRunner(String name, JettyConfig config)
-                throws Exception {
+            public JettySolrRunner startJettySolrRunner(
+                String name, JettyConfig config, String solrXml) throws Exception {
               System.setProperty("zkHost", getZkServer().getZkAddress());
 
               final Properties nodeProps = new Properties();
               nodeProps.setProperty("test-from-sysprop", "yup");
 
               Path runnerPath = createTempDir(name);
+              if (solrXml == null) {
+                solrXml = DEFAULT_CLOUD_SOLR_XML;
+              }
+              Files.write(runnerPath.resolve("solr.xml"), solrXml.getBytes(StandardCharsets.UTF_8));
               JettyConfig newConfig = JettyConfig.builder(config).build();
               JettySolrRunner jetty =
                   new JettySolrRunner(runnerPath.toString(), nodeProps, newConfig);