You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by md...@apache.org on 2021/03/24 15:07:32 UTC

[lucene-solr] branch branch_8x updated: SOLR-15249 Properly set ZK ACLs on /security.json

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

mdrob pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new c401dd4  SOLR-15249 Properly set ZK ACLs on /security.json
c401dd4 is described below

commit c401dd4cca2f31db46f6fca3d2bc79694c02489b
Author: Mike Drob <md...@apache.org>
AuthorDate: Mon Mar 15 15:50:45 2021 -0500

    SOLR-15249 Properly set ZK ACLs on /security.json
---
 solr/CHANGES.txt                                   |  2 ++
 .../java/org/apache/solr/cloud/ZkController.java   |  7 +++-
 .../authentication-and-authorization-plugins.adoc  |  2 ++
 .../src/basic-authentication-plugin.adoc           |  9 +----
 .../src/zookeeper-access-control.adoc              | 38 +++++++++++++++-------
 .../common/cloud/SecurityAwareZkACLProvider.java   |  9 ++---
 .../org/apache/solr/common/cloud/SolrZkClient.java | 15 +++++++--
 .../java/org/apache/solr/cloud/ZkTestServer.java   | 37 +++++++++++++++------
 8 files changed, 81 insertions(+), 38 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9d791cb..82fde99 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -68,6 +68,8 @@ Bug Fixes
 
 * SOLR-15216: Fix for Invalid Reference to data.followers in Admin UI. (Dean Pearce)
 
+* SOLR-15249: Properly set ZK ACLs on /security.json (Mike Drob)
+
 Other Changes
 ---------------------
 * SOLR-15118: Deprecate CollectionAdminRequest.getV2Request(). (Jason Gerlowski)
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 02ae1bc..475eb41 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -134,6 +134,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.zookeeper.ZooDefs.Ids.OPEN_ACL_UNSAFE;
 
 /**
  * Handle ZooKeeper interactions.
@@ -861,8 +862,12 @@ public class ZkController implements Closeable {
     cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, zkClient);
     byte[] emptyJson = "{}".getBytes(StandardCharsets.UTF_8);
     cmdExecutor.ensureExists(ZkStateReader.CLUSTER_STATE, emptyJson, CreateMode.PERSISTENT, zkClient);
-    cmdExecutor.ensureExists(ZkStateReader.SOLR_SECURITY_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
     cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
+    cmdExecutor.ensureExists(ZkStateReader.SOLR_SECURITY_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
+    if (zkClient.getACL(ZkStateReader.SOLR_SECURITY_CONF_PATH, null, true).equals(OPEN_ACL_UNSAFE)) {
+      log.warn("Contents of zookeeper /security.json are world-readable;" +
+          " consider setting up ACLs as described in https://solr.apache.org/guide/zookeeper-access-control.html");
+    }
     bootstrapDefaultConfigSet(zkClient);
   }
 
diff --git a/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc b/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
index 8ca87c4..fe0aa89 100644
--- a/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
+++ b/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
@@ -88,6 +88,8 @@ Then use the `bin/solr zk` command to upload the file:
 >bin/solr zk cp ./security.json zk:security.json -z localhost:2181
 ----
 
+NOTE: If you have defined `ZK_HOST` in `solr.in.sh`/`solr.in.cmd` (see <<setting-up-an-external-zookeeper-ensemble#updating-solr-include-files,instructions>>) you can omit `-z <zk host string>` from the above command.
+
 [WARNING]
 ====
 Whenever you use any security plugins and store `security.json` in ZooKeeper, we highly recommend that you implement access control in your ZooKeeper nodes. Information about how to enable this is available in the section <<zookeeper-access-control.adoc#,ZooKeeper Access Control>>.
diff --git a/solr/solr-ref-guide/src/basic-authentication-plugin.adoc b/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
index 178872d..f908a5b 100644
--- a/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
+++ b/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
@@ -64,14 +64,7 @@ If `blockUnknown` does not appear in the `security.json` file, it will default t
 
 If `realm` is not defined, it will default to `solr`.
 
-If you are using SolrCloud, you must upload `security.json` to ZooKeeper. You can use this example command, ensuring that the ZooKeeper port is correct:
-
-[source,bash]
-----
-$ bin/solr zk cp file:path_to_local_security.json zk:/security.json -z localhost:9983
-----
-
-NOTE: If you have defined `ZK_HOST` in `solr.in.sh`/`solr.in.cmd` (see <<setting-up-an-external-zookeeper-ensemble#updating-solr-include-files,instructions>>) you can omit `-z <zk host string>` from the above command.
+If you are using SolrCloud, you must upload `security.json` to ZooKeeper. An example command and more information about securing your setup can be found at <<authentication-and-authorization-plugins#in-solrcloud-mode,Authentication and Authorization Plugins In SolrCloud Mode>>.
 
 === Caveats
 
diff --git a/solr/solr-ref-guide/src/zookeeper-access-control.adoc b/solr/solr-ref-guide/src/zookeeper-access-control.adoc
index 55ada59..6382cae 100644
--- a/solr/solr-ref-guide/src/zookeeper-access-control.adoc
+++ b/solr/solr-ref-guide/src/zookeeper-access-control.adoc
@@ -73,7 +73,7 @@ You control which ACLs will be added by configuring `zkACLProvider` property in
 You can always make you own implementation, but Solr comes with:
 
 * `org.apache.solr.common.cloud.DefaultZkACLProvider`: It returns a list of length one for all `zNodePath`-s. The single ACL entry in the list is "open-unsafe". This is the default.
-* `org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider`: This lets you define your ACLs using system properties. Its `getACLsToAdd()` implementation does not use `zNodePath` for anything, so all znodes will get the same set of ACLs. It supports adding one or both of these options:
+* `org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider`: This lets you define your ACLs using system properties. Its `getACLsToAdd()` implementation will apply only admin ACLs to pre-defined sensitive paths as defined by `SecurityAwareZkACLProvider` (`/security.json` and `/security/*`) and both admin and user ACLs to the rest of the contents. The two sets of roles will be defined as:
 ** A user that is allowed to do everything.
 *** The permission is `ALL` (corresponding to all of `CREATE`, `READ`, `WRITE`, `DELETE`, and `ADMIN`), and the schema is "digest".
 *** The username and password are defined by system properties `zkDigestUsername` and `zkDigestPassword`, respectively.
@@ -97,8 +97,17 @@ There are two scripts that impact ZooKeeper ACLs:
 * For *nix systems: `bin/solr` & `server/scripts/cloud-scripts/zkcli.sh`
 * For Windows systems: `bin/solr.cmd` & `server/scripts/cloud-scripts/zkcli.bat`
 
+[IMPORTANT]
+Both the solr.in.* and the zkcli.* files will need to be updated with the same password for everything to work. The contents may appear redundant, but the scripts will not consult each other during operations.
+
 These Solr scripts can enable use of ZooKeeper ACLs by setting the appropriate system properties: uncomment the following and replace the passwords with ones you choose to enable the above-described VM parameters ACL and credentials providers in the following files:
 
+[.dynamic-tabs]
+--
+[example.tab-pane#nix]
+====
+[.tab-label]**nix*
+
 .solr.in.sh
 [source,bash]
 ----
@@ -110,6 +119,21 @@ These Solr scripts can enable use of ZooKeeper ACLs by setting the appropriate s
 #SOLR_OPTS="$SOLR_OPTS $SOLR_ZK_CREDS_AND_ACLS"
 ----
 
+.zkcli.sh
+[source,bash]
+----
+# Settings for ZK ACL
+#SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider \
+#  -DzkCredentialsProvider=org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider \
+#  -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \
+#  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD"
+----
+====
+
+[example.tab-pane#windows]
+====
+[.tab-label]*Windows*
+
 .solr.in.cmd
 [source,powershell]
 ----
@@ -121,16 +145,6 @@ REM  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEM
 REM set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS%
 ----
 
-.zkcli.sh
-[source,bash]
-----
-# Settings for ZK ACL
-#SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider \
-#  -DzkCredentialsProvider=org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider \
-#  -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \
-#  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD"
-----
-
 .zkcli.bat
 [source,powershell]
 ----
@@ -140,6 +154,8 @@ REM  -DzkCredentialsProvider=org.apache.solr.common.cloud.VMParamsSingleSetCrede
 REM  -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^
 REM  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
 ----
+====
+--
 
 == Changing ACL Schemes
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java
index 1c74d94..2fe2da9 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java
@@ -22,7 +22,7 @@ import org.apache.zookeeper.data.ACL;
 
 /**
  * {@link ZkACLProvider} capable of returning a different set of
- * {@link ACL}s for security-related znodes (default: subtree under /security)
+ * {@link ACL}s for security-related znodes (default: subtree under /security and security.json)
  * vs non-security-related znodes.
  */
 public abstract class SecurityAwareZkACLProvider implements ZkACLProvider {
@@ -42,11 +42,8 @@ public abstract class SecurityAwareZkACLProvider implements ZkACLProvider {
   }
 
   protected boolean isSecurityZNodePath(String zNodePath) {
-    if (zNodePath != null
-        && (zNodePath.equals(SECURITY_ZNODE_PATH) || zNodePath.startsWith(SECURITY_ZNODE_PATH + "/"))) {
-      return true;
-    }
-    return false;
+    return zNodePath != null
+        && (zNodePath.equals(ZkStateReader.SOLR_SECURITY_CONF_PATH) || zNodePath.equals(SECURITY_ZNODE_PATH) || zNodePath.startsWith(SECURITY_ZNODE_PATH + "/"));
   }
 
   /**
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index 6943fc5..42ec0f2 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -798,11 +798,23 @@ public class SolrZkClient implements Closeable {
   }
 
   /**
+   * @return the ACLs on a single node in ZooKeeper.
+   */
+  public List<ACL> getACL(String path, Stat stat, boolean retryOnConnLoss) throws KeeperException, InterruptedException {
+    if (retryOnConnLoss) {
+      return zkCmdExecutor.retryOperation(() -> keeper.getACL(path, stat));
+    } else {
+      return keeper.getACL(path, stat);
+    }
+  }
+
+  /**
    * Set the ACL on a single node in ZooKeeper. This will replace all existing ACL on that node.
    *
    * @param path path to set ACL on e.g. /solr/conf/solrconfig.xml
    * @param acls a list of {@link ACL}s to be applied
    * @param retryOnConnLoss true if the command should be retried on connection loss
+   * @return the stat of the node
    */
   public Stat setACL(String path, List<ACL> acls, boolean retryOnConnLoss) throws InterruptedException, KeeperException  {
     if (retryOnConnLoss) {
@@ -821,9 +833,8 @@ public class SolrZkClient implements Closeable {
       try {
         setACL(path, getZkACLProvider().getACLsToAdd(path), true);
         log.debug("Updated ACL on {}", path);
-      } catch (NoNodeException e) {
+      } catch (NoNodeException ignored) {
         // If a node was deleted, don't bother trying to set ACLs on it.
-        return;
       }
     });
   }
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
index 952f330..138e260 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
@@ -70,6 +70,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 public class ZkTestServer {
 
@@ -389,11 +390,17 @@ public class ZkTestServer {
           zkDb.close();
         }
 
-        if (cnxnFactory != null && cnxnFactory.getLocalPort() != 0) {
-          waitForServerDown(getZkHost(), 30000);
+        if (cnxnFactory != null) {
+          try {
+            int port = cnxnFactory.getLocalPort();
+            if (port > 0) {
+              waitForServerDown(getZkHost(), 30000);
+            }
+          } catch (NullPointerException ignored) {
+            // server never successfully started
+          }
         }
       } finally {
-
         ObjectReleaseTracker.release(this);
       }
     }
@@ -533,10 +540,12 @@ public class ZkTestServer {
 
   public void run(boolean solrFormat) throws InterruptedException, IOException {
     log.info("STARTING ZK TEST SERVER");
+    AtomicReference<Throwable> zooError = new AtomicReference<>();
     try {
       if (zooThread != null) {
         throw new IllegalStateException("ZK TEST SERVER IS ALREADY RUNNING");
       }
+      Thread parentThread = Thread.currentThread();
       // we don't call super.distribSetUp
       zooThread = new Thread("ZkTestServer Run Thread") {
 
@@ -570,7 +579,8 @@ public class ZkTestServer {
           try {
             zkServer.runFromConfig(config);
           } catch (Throwable t) {
-            log.error("zkServer error", t);
+            zooError.set(t);
+            parentThread.interrupt();
           }
         }
       };
@@ -582,15 +592,15 @@ public class ZkTestServer {
       int port = -1;
       try {
         port = getPort();
-      } catch (IllegalStateException e) {
-
+      } catch (IllegalStateException ignored) {
+        // Possibly fix this API to return null instead of throwing
       }
       while (port < 1) {
         Thread.sleep(100);
         try {
           port = getPort();
-        } catch (IllegalStateException e) {
-
+        } catch (IllegalStateException ignored) {
+          // Possibly fix this API to return null instead of throwing
         }
         if (cnt == 500) {
           throw new RuntimeException("Could not get the port for ZooKeeper server");
@@ -603,8 +613,15 @@ public class ZkTestServer {
 
       init(solrFormat);
     } catch (Exception e) {
-      log.error("Error trying to run ZK Test Server", e);
-      throw new RuntimeException(e);
+      RuntimeException toThrow = new RuntimeException("Could not get ZK port");
+      Throwable t = zooError.get();
+      if (t != null) {
+        toThrow.initCause(t);
+        toThrow.addSuppressed(e);
+      } else {
+        toThrow.initCause(e);
+      }
+      throw toThrow;
     }
   }