You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2022/05/05 15:07:36 UTC

[GitHub] [solr] gerlowskija commented on a diff in pull request #826: SOLR-15857: Add Secret Manager support for ZK ACL credentials

gerlowskija commented on code in PR #826:
URL: https://github.com/apache/solr/pull/826#discussion_r864870775


##########
solr/CHANGES.txt:
##########
@@ -44,6 +44,12 @@ Improvements
 * SOLR-15045: `DistributedZkUpdateProcessor` now issues commits to local shards and remote shards in parallel,
   halving the latency of synchronous commits (Michael Gibney)
 
+Security

Review Comment:
   [-1] The community usually discusses the sections in this file, and tries to keep them consistent across releases.
   
   Of the existing sections, this would probably fit best under "Improvements".  Could you move it there please?



##########
solr/CHANGES.txt:
##########
@@ -44,6 +44,12 @@ Improvements
 * SOLR-15045: `DistributedZkUpdateProcessor` now issues commits to local shards and remote shards in parallel,
   halving the latency of synchronous commits (Michael Gibney)
 
+Security
+----------------------
+* SOLR-15857: Solr now supports Secret Manager for Zookeeper ACL/credentials retrieval.
+  The default supported SM is AWS SM.

Review Comment:
   [Q] I think I see what you're driving at here, but some readers might be confused to see AWS SM described as the "default", since it's not used ootb.
   
   Or am I mistaken and we are going to use AWS SM ootb?



##########
solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-access-control.adoc:
##########
@@ -69,64 +97,559 @@ Solr nodes, clients, and tools (e.g., ZkCLI) always use a java class called {sol
 The implementation of the solution described here is all about changing `SolrZkClient`.
 If you use `SolrZkClient` in your application, the descriptions below will be true for your application too.
 
-=== Controlling Credentials
 
-You control which credentials provider will be used by configuring the `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}/solrj/org/apache/solr/common/cloud/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface.
+* Controlling credentials and ACLs is done in 3 steps: Set a `ZkCredentialsInjector` that reads the credentials from
+some source and then inject them into a `ZkCredentialsProvider` that Solr uses to connect to Zookeeper. ZkACLProvider
+uses the same credentials to set the ACLs.
+
+
+We will describe these 3 steps in details before giving some ready to use examples.
+
+
+. Set the `ZkCredentialsInjector`.
+. Set the `ZkCredentialsProvider`.
+. Set the `ZkACLProvider`.
+
+
+=== Set a Credentials Injector
+
+* A credentials injector gets the credentials from an external source and injects them into Solr.
+
+
+** You control which credentials will be injected by configuring `zkCredentialsInjector` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsInjector`such that it will take on the value
+of the same-named `zkCredentialsInjector` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkCredentialsInjector` implementation.
+
+==== Out of the Box Credentials Injector Implementations
+
+
+*  Solr comes with the following `ZkCredentialsInjectors`:
+
+** `org.apache.solr.common.cloud.acl.DefaultZkCredentialsInjector`: Its `getCredentials()` method returns a list of length zero,

Review Comment:
   [0] nitpick: method names and return values might be a little low-level for docs aimed at users like this page is.  Could we convey the same information without getting into those details?
   
   e.g.
   
   > `DefaultZkCredentialsInjector`, the default injector if no sys prop or Solr.xml value is provided, uses no credentials.



##########
solr/modules/aws-secret-provider/src/test/org/apache/solr/secret/zk/AWSSecretCredentialsProviderTest.java:
##########
@@ -0,0 +1,425 @@
+/*
+ * 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.secret.zk;
+
+import static org.apache.solr.common.cloud.acl.SecretCredentialInjector.*;
+import static org.apache.solr.secret.zk.AWSSecretCredentialsProvider.SecretMultiCredentials;
+
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.*;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.cloud.AbstractZkTestCase;
+import org.apache.solr.cloud.ZkTestServer;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.acl.DigestZkACLProvider;
+import org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider;
+import org.apache.solr.common.cloud.acl.SecretCredentialInjector;
+import org.apache.solr.common.cloud.acl.SecurityAwareZkACLProvider;
+import org.apache.solr.common.cloud.acl.ZkCredentialsInjector;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException.NoAuthException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AWSSecretCredentialsProviderTest extends SolrTestCaseJ4 {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final Charset DATA_ENCODING = StandardCharsets.UTF_8;
+
+  private static final String ALL_USERNAME = "connectAndAllACLUsername";
+  private static final String ALL_PASSWORD = "connectAndAllACLPassword";
+  private static final String READONLY_USERNAME = "readonlyACLUsername";
+  private static final String READONLY_PASSWORD = "readonlyACLPassword";
+
+  public static final String SECRET_NAME = "zkSecretCredentialSecretName";
+
+  protected ZkTestServer zkServer;
+
+  protected Path zkDir;
+
+  @BeforeClass
+  public static void beforeClass() {
+    System.setProperty("solrcloud.skip.autorecovery", "true");
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    System.clearProperty("solrcloud.skip.autorecovery");
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    assumeWorkingMockito();
+    if (log.isInfoEnabled()) {
+      log.info("####SETUP_START {}", getTestName());
+    }
+    createTempDir();
+
+    zkDir = createTempDir().resolve("zookeeper/server1/data");
+    log.info("ZooKeeper dataDir:{}", zkDir);
+    setSecuritySystemProperties();
+    zkServer = new ZkTestServer(zkDir);
+    zkServer.run(false);
+
+    System.setProperty("zkHost", zkServer.getZkAddress());
+
+    setSecretDigestZkSystemProps();
+    System.setProperty(
+        SolrZkClient.ZK_CREDENTIALS_INJECTOR_CLASS_NAME_VM_PARAM_NAME,
+        AllAndReadonlyCredentialZkCredentialsInjector.class.getName());
+
+    SolrZkClient zkClient =
+        new SolrZkClient(
+            zkServer.getZkHost(), AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT);
+    zkClient.makePath("/solr", false, true);
+    zkClient.close();
+
+    zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
+    zkClient.create(
+        "/protectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+    zkClient.makePath(
+        "/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+
+    zkClient.create(
+        SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH,
+        "content".getBytes(DATA_ENCODING),
+        CreateMode.PERSISTENT,
+        false);
+    zkClient.close();
+
+    clearSecuritySystemProperties();
+
+    zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
+    // Currently, no credentials on ZK connection, because those same VM-params are used for adding
+    // ACLs, and here we want
+    // no (or completely open) ACLs added. Therefore, hack your way into being authorized for
+    // creating anyway
+    zkClient
+        .getZooKeeper()
+        .addAuthInfo(
+            "digest", (ALL_USERNAME + ":" + ALL_PASSWORD).getBytes(StandardCharsets.UTF_8));
+    zkClient.create(
+        "/unprotectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+    zkClient.makePath(
+        "/unprotectedMakePathNode",
+        "content".getBytes(DATA_ENCODING),
+        CreateMode.PERSISTENT,
+        false);
+    zkClient.close();
+
+    setSecretDigestZkSystemProps();
+    if (log.isInfoEnabled()) {
+      log.info("####SETUP_END {}", getTestName());
+    }
+  }
+
+  private void setSecretDigestZkSystemProps() {
+    System.setProperty(
+        SolrZkClient.ZK_CRED_PROVIDER_CLASS_NAME_VM_PARAM_NAME,
+        DigestZkCredentialsProvider.class.getName());
+    System.setProperty(
+        SolrZkClient.ZK_ACL_PROVIDER_CLASS_NAME_VM_PARAM_NAME, DigestZkACLProvider.class.getName());
+    System.setProperty(
+        SolrZkClient.ZK_CREDENTIALS_INJECTOR_CLASS_NAME_VM_PARAM_NAME,
+        SecretCredentialInjector.class.getName());
+    System.setProperty(SECRET_CREDENTIAL_PROVIDER_SECRET_NAME_VM_PARAM, SECRET_NAME);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    zkServer.shutdown();
+    clearSecuritySystemProperties();
+    super.tearDown();
+  }
+
+  @Test
+  public void whenPassedSecretCredentials_thenReturnSameZkCredentials() {
+    final List<ZkCredential> zkCredentials =
+        new ArrayList<>() {
+          {
+            add(new ZkCredential(ALL_USERNAME, ALL_PASSWORD, ZkCredential.Perms.all));
+            add(new ZkCredential(READONLY_USERNAME, READONLY_PASSWORD, ZkCredential.Perms.read));
+          }
+        };
+
+    final SecretMultiCredentials secretMultiCredentials = new SecretMultiCredentials(zkCredentials);
+    AWSSecretCredentialsProvider awsSecretCredentialsProvider =
+        Mockito.spy(new AWSSecretCredentialsProvider());
+    Mockito.doReturn(secretMultiCredentials)
+        .when(awsSecretCredentialsProvider)
+        .createSecretMultiCredential(SECRET_NAME);
+    assertEquals(
+        secretMultiCredentials.getZkCredentials(),
+        awsSecretCredentialsProvider.getZkCredentials(SECRET_NAME));
+  }
+
+  @Test
+  public void testSecretJsonFormat() {
+    final AWSSecretCredentialsProvider awsSecretCredentialsProvider =
+        Mockito.spy(new AWSSecretCredentialsProvider());
+
+    final String secretEmpty = "";
+    Mockito.doReturn(secretEmpty).when(awsSecretCredentialsProvider).getSecretValue(SECRET_NAME);
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> awsSecretCredentialsProvider.createSecretMultiCredential(SECRET_NAME));
+
+    final String secretInvalidJson = "invalid json";
+    Mockito.doReturn(secretInvalidJson)
+        .when(awsSecretCredentialsProvider)
+        .getSecretValue(SECRET_NAME);
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> awsSecretCredentialsProvider.createSecretMultiCredential(SECRET_NAME));
+
+    final String secretWrongFormat =
+        "{"
+            + "   \"zkCredential\": ["
+            + "      {\"username\": \"admin-user\", \"password\": \"ADMIN-PASSWORD\", \"perms\": \"all\"},"
+            + "      {\"username\": \"readonly-user\", \"password\": \"READONLY-PASSWORD\", \"perms\": \"read\"}"
+            + "   ]"
+            + "}";
+    Mockito.doReturn(secretWrongFormat)
+        .when(awsSecretCredentialsProvider)
+        .getSecretValue(SECRET_NAME);
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> awsSecretCredentialsProvider.createSecretMultiCredential(SECRET_NAME));
+
+    final String secretRightFormat =
+        "{"
+            + "   \"zkCredentials\": ["
+            + "      {\"username\": \"admin-user\", \"password\": \"ADMIN-PASSWORD\", \"perms\": \"all\"},"
+            + "      {\"username\": \"readonly-user\", \"password\": \"READONLY-PASSWORD\", \"perms\": \"read\"}"
+            + "   ]"
+            + "}";
+    Mockito.doReturn(secretRightFormat)
+        .when(awsSecretCredentialsProvider)
+        .getSecretValue(SECRET_NAME);
+    awsSecretCredentialsProvider.createSecretMultiCredential(SECRET_NAME);
+  }
+
+  @Test
+  public void testAllCredentials() throws Exception {
+    System.setProperty(
+        SECRET_CREDENTIAL_PROVIDER_CLASS_VM_PARAM,
+        AllAndReadonlyAWSSecretCredentialsProvider.class.getName());
+
+    try (SolrZkClient zkClient =
+        new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT)) {
+      doTest(zkClient, true, true, true, true, true, true, true, true, true, true);
+    }
+  }
+
+  @Test
+  public void testNoCredentials() throws Exception {
+    System.setProperty(
+        SECRET_CREDENTIAL_PROVIDER_CLASS_VM_PARAM, NoAWSSecretCredentialsProvider.class.getName());
+
+    try (SolrZkClient zkClient =
+        new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT)) {
+      doTest(zkClient, false, false, false, false, false, false, false, false, false, false);
+    }
+  }
+
+  @Test
+  public void testWrongCredentials0() throws Exception {

Review Comment:
   [0] Nitpick, is the '0' in the test name intentional?



##########
solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-access-control.adoc:
##########
@@ -69,64 +97,559 @@ Solr nodes, clients, and tools (e.g., ZkCLI) always use a java class called {sol
 The implementation of the solution described here is all about changing `SolrZkClient`.
 If you use `SolrZkClient` in your application, the descriptions below will be true for your application too.
 
-=== Controlling Credentials
 
-You control which credentials provider will be used by configuring the `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}/solrj/org/apache/solr/common/cloud/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface.
+* Controlling credentials and ACLs is done in 3 steps: Set a `ZkCredentialsInjector` that reads the credentials from
+some source and then inject them into a `ZkCredentialsProvider` that Solr uses to connect to Zookeeper. ZkACLProvider
+uses the same credentials to set the ACLs.
+
+
+We will describe these 3 steps in details before giving some ready to use examples.
+
+
+. Set the `ZkCredentialsInjector`.
+. Set the `ZkCredentialsProvider`.
+. Set the `ZkACLProvider`.
+
+
+=== Set a Credentials Injector
+
+* A credentials injector gets the credentials from an external source and injects them into Solr.
+
+
+** You control which credentials will be injected by configuring `zkCredentialsInjector` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsInjector`such that it will take on the value
+of the same-named `zkCredentialsInjector` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkCredentialsInjector` implementation.
+
+==== Out of the Box Credentials Injector Implementations
+
+
+*  Solr comes with the following `ZkCredentialsInjectors`:
+
+** `org.apache.solr.common.cloud.acl.DefaultZkCredentialsInjector`: Its `getCredentials()` method returns a list of length zero,
+or "no credentials used". This is the default.
+
+** `org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector`: The username and password are defined by system
+properties name:`zkDigestUsername` and `zkDigestPassword`. This set of credentials will be added to the list of credentials
+returned by `getCredentials()` if both username and password are provided.
+
+*** If the one set of credentials above is not added to the list, this implementation will fall back to default
+behavior and use the (empty) credentials list from `DefaultZkCredentialsInjector`.
+
+*** Alternatively, you can set the `zkDigestCredentialsFile` system property to load `zkDigestUsername` and
+`zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a
+Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties.
+
+*** Usage (See full example later in the page):
+
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector
+-DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD
+-DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+
+# Or using a Java property file containing the credentials:
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector
+-DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+----
+
+
+
+** `org.apache.solr.common.cloud.acl.SecretCredentialInjector`: Used to get credentials from a Secret Manager. It retrieves the
+credentials using a {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`] passed
+through Systems Properties via `zkSecretCredentialsProvider` property name.
+
+*** The out the box implementation of `SecretCredentialsProvider` is {solr-javadocs}aws-secret-provider/org/apache/solr/secret/zk/AWSSecretCredentialsProvider.html[`AWSSecretCredentialsProvider`]  which pulls the credentials
+from  https://aws.amazon.com/secrets-manager/[AWS Secret Manager]
+
+
+*** Usage (See full example later in the page):
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+-DzkSecretCredentialsProvider=org.apache.solr.secret.zk.AWSSecretCredentialsProvider
+-DzkSecretCredentialSecretName=zkCredentialsSecret
+-DzkCredentialsAWSSecretRegion=us-west-2"
+----
+
+It expects a secret value containing the credentials in the following Json format (the secret name is `zkCredentialsSecret`):
+----
+{
+  "zkCredentials": [
+                      {"username": "admin-user", "password": "ADMIN-PASSWORD", "perms": "all"},
+                      {"username": "readonly-user", "password": "READONLY-PASSWORD", "perms": "read"}
+                  ]
+}
+----
+
+** You can have your own Secret Manager Implementation, implementing
+{solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`]  and pass it
+through `zkSecretCredentialsProvider` System Property name.
+
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+-DzkSecretCredentialsProvider=fully.qualified.class.CustomSecretCredentialsProvider
+-DzkSecretCredentialSecretName=zkCredentialsSecret
+----
+
+** Alternatively, you can create your own credentials injector by
+implementing {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] and pass it through System Properties using `zkCredentialsInjector` name:
+
+----
+-DzkCredentialsInjector=fully.qualified.class.CustomInjectorClassName
+----
+
+
+After the credentials are injected they are then used in the `ZkCredentialsProvider`.
+
+
+
+=== Set a Credential Provider
 
-`solr.xml` defines the `zkCredentialsProvider` such that it will take on the value of the same-named `zkCredentialsProvider` system property if it is defined (in `solr.in.sh/.cmd` - see <<ZooKeeper ACLs in Solr Scripts,below>>), or if not, default to the `DefaultZkCredentialsProvider` implementation.
 
-==== Out of the Box Credential Implementations
+
+
+`ZkCredentialsProvider` gets the credentials from the `ZkCredentialsInjector` and uses them to connect to Zookeeper.
+
+
+** You control which credentials will be used by configuring `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsProvider`such that it will take on the value

Review Comment:
   [0] nitpick - I think there should be spaces around `zkCredentialsProvider` on this line?



##########
solr/modules/aws-secret-provider/README.md:
##########
@@ -0,0 +1,80 @@
+Apache Solr - AWS Secret Provider
+===========================
+
+An implementation of `SecretCredentialsProvider` that pulls Zookeeper credentials from an AWS Secret Manager.

Review Comment:
   [0] One one hand, I understand the rationale for this to be packaged as a contrib/module: it makes it easy to keep on/off the class path in particular Solr installations.
   
   But OTOH, contribs typically extend something in "Solr Core": it feels a little weird to create a contrib to hold an extension point for a SolrJ API like `SecretCredentialsProvider`.  Especially now that there seems to be consensus around modularizing and breaking up SolrJ - in theory this could be an "optional" SolrJ dependency.
   
   I can't think of any specific problem with the existing contrib packaging.  Maybe it's the best option.  Especially if it doesn't fail anyone else's smell test.  But I just figured I'd point it out in case any other reviewers thought it was odd.
   
   Feel free to ignore this comment if no one else chimes in 



##########
solr/solrj/src/java/org/apache/solr/common/cloud/VMParamsSingleSetCredentialsDigestZkCredentialsProvider.java:
##########
@@ -16,64 +16,66 @@
  */
 package org.apache.solr.common.cloud;
 
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
-import java.util.Properties;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.StringUtils;
+import org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider;
+import org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector;
+import org.apache.solr.common.cloud.acl.ZkCredentialsInjector;
 
+/**
+ * Deprecated in favor of a combination of {@link DigestZkCredentialsProvider} and {@link
+ * VMParamsZkCredentialsInjector}.
+ *
+ * <pre>
+ * Current implementation delegates to {@link DigestZkCredentialsProvider} with an injected {@link VMParamsZkCredentialsInjector}
+ * </pre>
+ */
+@Deprecated
 public class VMParamsSingleSetCredentialsDigestZkCredentialsProvider
     extends DefaultZkCredentialsProvider {
 
-  public static final String DEFAULT_DIGEST_FILE_VM_PARAM_NAME = "zkDigestCredentialsFile";
-  public static final String DEFAULT_DIGEST_USERNAME_VM_PARAM_NAME = "zkDigestUsername";
-  public static final String DEFAULT_DIGEST_PASSWORD_VM_PARAM_NAME = "zkDigestPassword";
-
-  static Properties readCredentialsFile(String pathToFile) throws SolrException {
-    Properties props = new Properties();
-    try (Reader reader =
-        new InputStreamReader(new FileInputStream(pathToFile), StandardCharsets.UTF_8)) {
-      props.load(reader);
-    } catch (IOException ioExc) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, ioExc);
-    }
-    return props;
-  }
+  public static final String DEFAULT_DIGEST_USERNAME_VM_PARAM_NAME =

Review Comment:
   [0] nitpick: A few of these deprecated SolrJ classes have public constants that clients might be using.  It might help these clients work around the deprecation if the class-level javadocs have an `@see` pointing at the class that the replacement constants live in (in this case that'd be `@see VMParamsZkCredentialsInjector`).
   
   



##########
solr/modules/aws-secret-provider/README.md:
##########
@@ -0,0 +1,80 @@
+Apache Solr - AWS Secret Provider
+===========================
+
+An implementation of `SecretCredentialsProvider` that pulls Zookeeper credentials from an AWS Secret Manager.
+
+This plugin uses the [default AWS credentials provider chain](https://docs.aws.amazon.com/sdk-for-java/v2/developer-guide/credentials.html), so ensure that your credentials are set appropriately (e.g., via env var, or in `~/.aws/credentials`, etc.).
+
+## Usage
+
+- To enable this feature copy the jar files in `modules/aws-secret-provider/lib` to `SOLR_INSTALL/server/solr-webapp/webapp/WEB-INF/lib/` and add follow the below steps before restarting Solr.
+
+- Create a secret in AWS SM (for example named *zkCredentialsSecret*) containing the Zookeeper credentials in the following Json format:
+
+
+
+```json
+    {
+        "zkCredentials": [
+            {"username": "admin-user", "password": "ADMIN-PASSWORD", "perms": "all"},
+            {"username": "readonly-user", "password": "READONLY-PASSWORD", "perms": "read"}
+        ]
+    }
+```
+
+
+-  Pass the secret name and region trough `solr.in.sh:`

Review Comment:
   [0] nitpick: trough -> through



##########
solr/modules/aws-secret-provider/README.md:
##########
@@ -0,0 +1,80 @@
+Apache Solr - AWS Secret Provider
+===========================
+
+An implementation of `SecretCredentialsProvider` that pulls Zookeeper credentials from an AWS Secret Manager.
+
+This plugin uses the [default AWS credentials provider chain](https://docs.aws.amazon.com/sdk-for-java/v2/developer-guide/credentials.html), so ensure that your credentials are set appropriately (e.g., via env var, or in `~/.aws/credentials`, etc.).
+
+## Usage
+
+- To enable this feature copy the jar files in `modules/aws-secret-provider/lib` to `SOLR_INSTALL/server/solr-webapp/webapp/WEB-INF/lib/` and add follow the below steps before restarting Solr.

Review Comment:
   [Q/-1] I'm a little rusty on working with modules, but I'm pretty sure the standard way to enable/disable them isn't to copy the JARs.  If memory serves, we typically enable modules either (1) using a `<lib>` or `<sharedLib>` directive in Solr.xml, or (2) using the awesome [SOLR_MODULES env-var support](https://github.com/apache/solr/blob/main/solr/solr-ref-guide/modules/configuration-guide/pages/solr-modules.adoc) coming in 9.0.
   
   Is there something specific to this module that requires users to copy around jars?



##########
solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-access-control.adoc:
##########
@@ -56,9 +56,37 @@ ACLs describe who is allowed to read, update, delete, create, etc.
 Each piece of information (znode/content) in ZooKeeper has its own set of ACLs, and inheritance or sharing is not possible.
 The default behavior in Solr is to add one ACL on all the content it creates - one ACL that gives anyone the permission to do anything (in ZooKeeper terms this is called "the open-unsafe ACL").
 
+
+
+== Solr to Zookeeper ACLs Workflow
+
+* Solr to Zookeeper credentials and ACLs are controlled trough 3 interfaces: {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`],  {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsProvider.html[`ZkCredentialsProvider`] and {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkACLProvider.html[`ZkACLProvider`].
+
+* The workflow is as follow: Credentials source →   `ZkCredentialsInjector` →  `ZkCredentialsProvider/ZkACLProvider` → Zookeeper.
+
+`ZkCredentialsInjector` gets the credentials from an external source which in turn get injected into `ZkCredentialsProvider`
+and `ZkACLProvider`. The "external source" here can be System Properties, a file, a Secret Manager, or any other local or remote source.
+
+* Those credentials are then passed to Solr trough System Properties using the following properties names:
+`zkCredentialsInjector`, `zkACLProvider` and `zkCredentialsProvider`. See below sections for details.
+
+* Two sets of roles are supported:
+** `ALL` user: A user that is allowed to do everything (corresponding to all of `CREATE`, `READ`, `WRITE`, `DELETE`, and `ADMIN`).
+** `READ` user: A ready-only user that is only allowed to perform read operations.
+
+
+* We always protect access to content by limiting to two users - an admin-user and a readonly-user - AND we always connect with

Review Comment:
   [0] nitpick: It might be better to avoid "first-person" in the ref-guide, just because it's ambiguous.
   
   I think I understand what you're trying to say.  But some readers might wonder (for example) who the "we" is in "we always connect with credentials corresponding to this same admin-user".
   
   Is "we" the Solr server itself? Command line tools like zkcli.sh and bin/solr?  All ZK clients (including or excluding the Solr server)? etc.
   
   > Each zkACLProvider implementation limits access to two users: an admin-user and a readonly-user.  Solr itself always connects to ZooKeeper using the admin credentials.  Read-only credentials should be used by any other clients (e.g. SolrJ) who don't need write-access to ZooKeeper. 
   
   That said I see there's a lot of "we" in this page even before your PR, so feel free to ignore this comment unless you agree.



##########
solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-access-control.adoc:
##########
@@ -69,64 +97,559 @@ Solr nodes, clients, and tools (e.g., ZkCLI) always use a java class called {sol
 The implementation of the solution described here is all about changing `SolrZkClient`.
 If you use `SolrZkClient` in your application, the descriptions below will be true for your application too.
 
-=== Controlling Credentials
 
-You control which credentials provider will be used by configuring the `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}/solrj/org/apache/solr/common/cloud/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface.
+* Controlling credentials and ACLs is done in 3 steps: Set a `ZkCredentialsInjector` that reads the credentials from
+some source and then inject them into a `ZkCredentialsProvider` that Solr uses to connect to Zookeeper. ZkACLProvider
+uses the same credentials to set the ACLs.
+
+
+We will describe these 3 steps in details before giving some ready to use examples.
+
+
+. Set the `ZkCredentialsInjector`.
+. Set the `ZkCredentialsProvider`.
+. Set the `ZkACLProvider`.
+
+
+=== Set a Credentials Injector
+
+* A credentials injector gets the credentials from an external source and injects them into Solr.
+
+
+** You control which credentials will be injected by configuring `zkCredentialsInjector` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsInjector`such that it will take on the value
+of the same-named `zkCredentialsInjector` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkCredentialsInjector` implementation.
+
+==== Out of the Box Credentials Injector Implementations
+
+
+*  Solr comes with the following `ZkCredentialsInjectors`:
+
+** `org.apache.solr.common.cloud.acl.DefaultZkCredentialsInjector`: Its `getCredentials()` method returns a list of length zero,
+or "no credentials used". This is the default.
+
+** `org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector`: The username and password are defined by system
+properties name:`zkDigestUsername` and `zkDigestPassword`. This set of credentials will be added to the list of credentials
+returned by `getCredentials()` if both username and password are provided.
+
+*** If the one set of credentials above is not added to the list, this implementation will fall back to default
+behavior and use the (empty) credentials list from `DefaultZkCredentialsInjector`.
+
+*** Alternatively, you can set the `zkDigestCredentialsFile` system property to load `zkDigestUsername` and
+`zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a
+Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties.
+
+*** Usage (See full example later in the page):
+
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector
+-DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD
+-DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+
+# Or using a Java property file containing the credentials:
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector
+-DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+----
+
+
+
+** `org.apache.solr.common.cloud.acl.SecretCredentialInjector`: Used to get credentials from a Secret Manager. It retrieves the
+credentials using a {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`] passed
+through Systems Properties via `zkSecretCredentialsProvider` property name.
+
+*** The out the box implementation of `SecretCredentialsProvider` is {solr-javadocs}aws-secret-provider/org/apache/solr/secret/zk/AWSSecretCredentialsProvider.html[`AWSSecretCredentialsProvider`]  which pulls the credentials
+from  https://aws.amazon.com/secrets-manager/[AWS Secret Manager]
+
+
+*** Usage (See full example later in the page):
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+-DzkSecretCredentialsProvider=org.apache.solr.secret.zk.AWSSecretCredentialsProvider
+-DzkSecretCredentialSecretName=zkCredentialsSecret
+-DzkCredentialsAWSSecretRegion=us-west-2"
+----
+
+It expects a secret value containing the credentials in the following Json format (the secret name is `zkCredentialsSecret`):
+----
+{
+  "zkCredentials": [
+                      {"username": "admin-user", "password": "ADMIN-PASSWORD", "perms": "all"},
+                      {"username": "readonly-user", "password": "READONLY-PASSWORD", "perms": "read"}
+                  ]
+}
+----
+
+** You can have your own Secret Manager Implementation, implementing
+{solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`]  and pass it
+through `zkSecretCredentialsProvider` System Property name.
+
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+-DzkSecretCredentialsProvider=fully.qualified.class.CustomSecretCredentialsProvider
+-DzkSecretCredentialSecretName=zkCredentialsSecret
+----
+
+** Alternatively, you can create your own credentials injector by
+implementing {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] and pass it through System Properties using `zkCredentialsInjector` name:
+
+----
+-DzkCredentialsInjector=fully.qualified.class.CustomInjectorClassName
+----
+
+
+After the credentials are injected they are then used in the `ZkCredentialsProvider`.
+
+
+
+=== Set a Credential Provider
 
-`solr.xml` defines the `zkCredentialsProvider` such that it will take on the value of the same-named `zkCredentialsProvider` system property if it is defined (in `solr.in.sh/.cmd` - see <<ZooKeeper ACLs in Solr Scripts,below>>), or if not, default to the `DefaultZkCredentialsProvider` implementation.
 
-==== Out of the Box Credential Implementations
+
+
+`ZkCredentialsProvider` gets the credentials from the `ZkCredentialsInjector` and uses them to connect to Zookeeper.
+
+
+** You control which credentials will be used by configuring `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsProvider`such that it will take on the value
+of the same-named `zkCredentialsProvider` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkCredentialsProvider` implementation.
+
+
+==== Out of the Box credentials Implementations
 
 You can always make you own implementation, but Solr comes with two implementations:
 
-* `org.apache.solr.common.cloud.DefaultZkCredentialsProvider`: Its `getCredentials()` returns a list of length zero, or "no credentials used".
-This is the default.
-* `org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider`: This lets you define your credentials using system properties.
-It supports at most one set of credentials.
-** The schema is "digest".
-The username and password are defined by system properties `zkDigestUsername` and `zkDigestPassword`.
-This set of credentials will be added to the list of credentials returned by `getCredentials()` if both username and password are provided.
-** If the one set of credentials above is not added to the list, this implementation will fall back to default behavior and use the (empty) credentials list from `DefaultZkCredentialsProvider`.
-** Alternatively, you can set the `zkDigestCredentialsFile` system property to load the `zkDigestUsername` and `zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties.
+* No credentials:
 
-=== Controlling ACLs
+`org.apache.solr.common.cloud.acl.DefaultZkCredentialsProvider`: Its `getCredentials()` returns a list of length
+zero, or "no credentials used". This is the default.
 
-You control which ACLs will be added by configuring `zkACLProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}/solrj/org/apache/solr/common/cloud/ZkACLProvider.html[`ZkACLProvider`] interface.
 
-`solr.xml` defines the `zkACLProvider` such that it will take on the value of the same-named `zkACLProvider` system property if it is defined (in `solr.in.sh/.cmd` - see <<ZooKeeper ACLs in Solr Scripts,below>>), or if not, default to the `DefaultZkACLProvider` implementation.
+* `digest` scheme based credentialsProvider:
 
-==== Out of the Box ACL Implementations
+`org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider`: The used scheme is `digest` and it gets the `ALL` user
+credentials  (perms=all) from the specified `ZkCredentialsInjector`.
+
+If a `ZkCredentialsInjector` with an `ALL` user ( having both username and password provided) is not defined, it will fall
+back to default behavior and use the (empty) credentials list from `DefaultZkCredentialsProvider`.
+
+
+=== Set an ACL Provider
 
+
+** You control which ACLs will be added by configuring `zkACLProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkACLProvider.html[`ZkACLProvider`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkACLProvider`such that it will take on the value
+of the same-named `zkACLProvider` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkACLProvider` implementation.
+
+
+==== Out of the Box ACL Implementations
 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.
-The `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.
-*** This ACL will not be added to the list of ACLs unless both username and password are provided.
-** A user that is only allowed to perform read operations.
-*** The permission is `READ` and the schema is `digest`.
-*** The username and password are defined by system properties `zkDigestReadonlyUsername` and `zkDigestReadonlyPassword`, respectively.
-*** This ACL will not be added to the list of ACLs unless both username and password are provided.
-** Alternatively, you can set the `zkDigestCredentialsFile` system property to load the `zkDigestUsername` and `zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties for the `ALL` user, as well as the `zkDigestReadonlyUsername` and `zkDigestReadonlyPassword` properties for the `READONLY` user.
-* `org.apache.solr.common.cloud.SaslZkACLProvider`: Requires SASL authentication.
-Gives all permissions for the user specified in system property `solr.authorization.superuser` (default: `solr`) when using SASL, and gives read permissions for anyone else.
-Designed for a setup where configurations have already been set up and will not be modified, or where configuration changes are controlled via Solr APIs.
-This provider will be useful for administration in a Kerberos environment.
-In such an environment, the administrator wants Solr to authenticate to ZooKeeper using SASL, since this is only way to authenticate with ZooKeeper via Kerberos.
-
-If none of the above ACLs is added to the list, the (empty) ACL list of `DefaultZkACLProvider` will be used by default.
-
-Notice the overlap in system property names with credentials provider `VMParamsSingleSetCredentialsDigestZkCredentialsProvider` (described above).
-This is to let the two providers collaborate in a nice and perhaps common way: we always protect access to content by limiting to two users - an admin-user and a readonly-user - AND we always connect with credentials corresponding to this same admin-user, basically so that we can do anything to the content/znodes we create ourselves.
-
-You can give the readonly credentials to "clients" of your SolrCloud cluster - e.g., to be used by SolrJ clients.
-They will be able to read whatever is necessary to run a functioning SolrJ client, but they will not be able to modify any content in ZooKeeper.
+* `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.acl.DigestZkACLProvider`: This lets you define your ACLs using the defined `ZkCredentialsInjector`. 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 `all` and `read` users are injected through the `ZkCredentialsInjector` described earlier in the page.
+
+* `org.apache.solr.common.cloud.SaslZkACLProvider`: Requires SASL authentication. Gives all permissions for the user specified in system
+property `solr.authorization.superuser` (default:`solr`) when using SASL, and gives read permissions for anyone else. Designed for
+a setup where configurations have already been set up and will not be modified, or where configuration changes are
+controlled via Solr APIs. This provider will be useful for administration in a kerberos environment. In such
+an environment, the administrator wants Solr to authenticate to ZooKeeper using SASL, since this is only way to
+authenticate with ZooKeeper via Kerberos.
+
+* If none of the above ACLs is added to the list, the (empty) ACL list of `DefaultZkACLProvider` will be used by default.
+
+
+
+=== Examples
+
+
+Below examples are for `digest` scheme.
+
+* xref:#through-system-properties[System Properties]
+* xref:#through-a-file[Through a File]
+* xref:#through-aws-secret-manager[AWS Secret Manager]
+* xref:#through-a-custom-secret-manager[A Custom Secret Manager]
+* xref:#through-a-custom-credentials-injector[Custom Credentials Injector]
+
+
+==== Through System Properties
+:sectanchors:
+
+* ZK credentials are passed through System Properties via `DzkDigestUsername`, `DzkDigestPassword`, `DzkDigestReadonlyUsername`
+and `DzkDigestReadonlyPassword` properties names.
+
+[.dynamic-tabs]
+--
+[example.tab-pane#system-props-nix]
+====
+[.tab-label]**nix*
+
+.solr.in.sh
+[source,bash]
+----
+
+# Settings for ZK ACL
+SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+  -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \
+  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD"
+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.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+  -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \
+  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD"
+----
+====
+
+[example.tab-pane#system-props-windows]
+====
+[.tab-label]*Windows*
+
+.solr.in.cmd
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector ^
+ -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^
+ -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS%
+----
+
+.zkcli.bat
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector ^
+ -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^
+ -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+----
+====
+--
+
+
+
+==== Through a File
+
+* Create a Java property files, for example named `zookeepercredentials.properties` containing the credentials in the following format:
+----
+zkDigestUsername=admin-user
+zkDigestPassword=CHANGEME-ADMIN-PASSWORD
+zkDigestReadonlyUsername=readonly-user
+zkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+----
+
+* Pass the file path via System Properties:
+
+
+[.dynamic-tabs]
+--
+[example.tab-pane#file-system-props-nix]
+====
+[.tab-label]**nix*
+
+.solr.in.sh
+[source,bash]
+----
+
+# Settings for ZK ACL
+SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+  -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties"
+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.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+  -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties"
+----
+====
+
+[example.tab-pane#file-system-props-windows]
+====
+[.tab-label]*Windows*
+
+.solr.in.cmd
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector ^
+ -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS%
+----
+
+.zkcli.bat
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector ^
+ -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+----
+====
+--
+
+
+
+==== Through AWS Secret Manager
+
+* To enable this feature, copy the jar files in `modules/aws-secret-provider/lib` to `SOLR_INSTALL/server/solr-webapp/webapp/WEB-INF/lib/` and add follow the below steps before restarting Solr.

Review Comment:
   [Q] See my comment elsewhere on whether manual jar-copying is really preferred here, or whether a more hands-off approach (e.g. SOLR_MODULES) can be used.



##########
solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-access-control.adoc:
##########
@@ -56,9 +56,37 @@ ACLs describe who is allowed to read, update, delete, create, etc.
 Each piece of information (znode/content) in ZooKeeper has its own set of ACLs, and inheritance or sharing is not possible.
 The default behavior in Solr is to add one ACL on all the content it creates - one ACL that gives anyone the permission to do anything (in ZooKeeper terms this is called "the open-unsafe ACL").
 
+
+
+== Solr to Zookeeper ACLs Workflow
+
+* Solr to Zookeeper credentials and ACLs are controlled trough 3 interfaces: {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`],  {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsProvider.html[`ZkCredentialsProvider`] and {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkACLProvider.html[`ZkACLProvider`].
+
+* The workflow is as follow: Credentials source →   `ZkCredentialsInjector` →  `ZkCredentialsProvider/ZkACLProvider` → Zookeeper.
+
+`ZkCredentialsInjector` gets the credentials from an external source which in turn get injected into `ZkCredentialsProvider`
+and `ZkACLProvider`. The "external source" here can be System Properties, a file, a Secret Manager, or any other local or remote source.
+
+* Those credentials are then passed to Solr trough System Properties using the following properties names:

Review Comment:
   > Those credentials are then passed to Solr through System Properties [...] `zkCredentialsInjector`, `zkACLProvider` and `zkCredentialsProvider`
   
   [0] nitpick: I think I get what you're trying to say here, but I could see some readers taking this to mean that the value for those sysprops are username/password literals (instead of classnames, which is what we actually expect there AFAICT).
   
   Maybe consider rewording to make it more explicit that the expected values for these sysprops are classnames, not username/password literals? One suggestion/example is included below, but feel free to use whatever wording you like.  (Or leave it as-is if you think it's clear enough.)
   
   > Solr looks up the ZooKeeper credential and ACL information using the classes pointed to by the `zkCredentialsInjector`, `zkACLProvider`, and `zkCredentialsProvider` system properties



##########
solr/solrj/src/java/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.common.cloud.acl;
+
+import java.util.List;
+
+public interface ZkCredentialsInjector {
+
+  List<ZkCredential> getZkCredentials();
+
+  class ZkCredential {
+
+    public enum Perms {
+      all,

Review Comment:
   [0] nitpick: I think enum values generally use ALL_CAPS, unless there's a particular reason to avoid it?



##########
solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-access-control.adoc:
##########
@@ -69,64 +97,559 @@ Solr nodes, clients, and tools (e.g., ZkCLI) always use a java class called {sol
 The implementation of the solution described here is all about changing `SolrZkClient`.
 If you use `SolrZkClient` in your application, the descriptions below will be true for your application too.
 
-=== Controlling Credentials
 
-You control which credentials provider will be used by configuring the `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}/solrj/org/apache/solr/common/cloud/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface.
+* Controlling credentials and ACLs is done in 3 steps: Set a `ZkCredentialsInjector` that reads the credentials from
+some source and then inject them into a `ZkCredentialsProvider` that Solr uses to connect to Zookeeper. ZkACLProvider
+uses the same credentials to set the ACLs.
+
+
+We will describe these 3 steps in details before giving some ready to use examples.
+
+
+. Set the `ZkCredentialsInjector`.
+. Set the `ZkCredentialsProvider`.
+. Set the `ZkACLProvider`.
+
+
+=== Set a Credentials Injector
+
+* A credentials injector gets the credentials from an external source and injects them into Solr.
+
+
+** You control which credentials will be injected by configuring `zkCredentialsInjector` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsInjector`such that it will take on the value
+of the same-named `zkCredentialsInjector` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkCredentialsInjector` implementation.
+
+==== Out of the Box Credentials Injector Implementations
+
+
+*  Solr comes with the following `ZkCredentialsInjectors`:
+
+** `org.apache.solr.common.cloud.acl.DefaultZkCredentialsInjector`: Its `getCredentials()` method returns a list of length zero,
+or "no credentials used". This is the default.
+
+** `org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector`: The username and password are defined by system
+properties name:`zkDigestUsername` and `zkDigestPassword`. This set of credentials will be added to the list of credentials
+returned by `getCredentials()` if both username and password are provided.
+
+*** If the one set of credentials above is not added to the list, this implementation will fall back to default
+behavior and use the (empty) credentials list from `DefaultZkCredentialsInjector`.
+
+*** Alternatively, you can set the `zkDigestCredentialsFile` system property to load `zkDigestUsername` and
+`zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a
+Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties.
+
+*** Usage (See full example later in the page):
+
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector
+-DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD
+-DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+
+# Or using a Java property file containing the credentials:
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector
+-DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+----
+
+
+
+** `org.apache.solr.common.cloud.acl.SecretCredentialInjector`: Used to get credentials from a Secret Manager. It retrieves the
+credentials using a {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`] passed
+through Systems Properties via `zkSecretCredentialsProvider` property name.
+
+*** The out the box implementation of `SecretCredentialsProvider` is {solr-javadocs}aws-secret-provider/org/apache/solr/secret/zk/AWSSecretCredentialsProvider.html[`AWSSecretCredentialsProvider`]  which pulls the credentials
+from  https://aws.amazon.com/secrets-manager/[AWS Secret Manager]
+
+
+*** Usage (See full example later in the page):
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+-DzkSecretCredentialsProvider=org.apache.solr.secret.zk.AWSSecretCredentialsProvider
+-DzkSecretCredentialSecretName=zkCredentialsSecret
+-DzkCredentialsAWSSecretRegion=us-west-2"
+----
+
+It expects a secret value containing the credentials in the following Json format (the secret name is `zkCredentialsSecret`):
+----
+{
+  "zkCredentials": [
+                      {"username": "admin-user", "password": "ADMIN-PASSWORD", "perms": "all"},
+                      {"username": "readonly-user", "password": "READONLY-PASSWORD", "perms": "read"}
+                  ]
+}
+----
+
+** You can have your own Secret Manager Implementation, implementing
+{solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`]  and pass it
+through `zkSecretCredentialsProvider` System Property name.
+
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+-DzkSecretCredentialsProvider=fully.qualified.class.CustomSecretCredentialsProvider
+-DzkSecretCredentialSecretName=zkCredentialsSecret
+----
+
+** Alternatively, you can create your own credentials injector by
+implementing {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] and pass it through System Properties using `zkCredentialsInjector` name:
+
+----
+-DzkCredentialsInjector=fully.qualified.class.CustomInjectorClassName
+----
+
+
+After the credentials are injected they are then used in the `ZkCredentialsProvider`.
+
+
+
+=== Set a Credential Provider
 
-`solr.xml` defines the `zkCredentialsProvider` such that it will take on the value of the same-named `zkCredentialsProvider` system property if it is defined (in `solr.in.sh/.cmd` - see <<ZooKeeper ACLs in Solr Scripts,below>>), or if not, default to the `DefaultZkCredentialsProvider` implementation.
 
-==== Out of the Box Credential Implementations
+
+
+`ZkCredentialsProvider` gets the credentials from the `ZkCredentialsInjector` and uses them to connect to Zookeeper.
+
+
+** You control which credentials will be used by configuring `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsProvider`such that it will take on the value
+of the same-named `zkCredentialsProvider` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkCredentialsProvider` implementation.
+
+
+==== Out of the Box credentials Implementations
 
 You can always make you own implementation, but Solr comes with two implementations:
 
-* `org.apache.solr.common.cloud.DefaultZkCredentialsProvider`: Its `getCredentials()` returns a list of length zero, or "no credentials used".
-This is the default.
-* `org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider`: This lets you define your credentials using system properties.
-It supports at most one set of credentials.
-** The schema is "digest".
-The username and password are defined by system properties `zkDigestUsername` and `zkDigestPassword`.
-This set of credentials will be added to the list of credentials returned by `getCredentials()` if both username and password are provided.
-** If the one set of credentials above is not added to the list, this implementation will fall back to default behavior and use the (empty) credentials list from `DefaultZkCredentialsProvider`.
-** Alternatively, you can set the `zkDigestCredentialsFile` system property to load the `zkDigestUsername` and `zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties.
+* No credentials:
 
-=== Controlling ACLs
+`org.apache.solr.common.cloud.acl.DefaultZkCredentialsProvider`: Its `getCredentials()` returns a list of length
+zero, or "no credentials used". This is the default.
 
-You control which ACLs will be added by configuring `zkACLProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}/solrj/org/apache/solr/common/cloud/ZkACLProvider.html[`ZkACLProvider`] interface.
 
-`solr.xml` defines the `zkACLProvider` such that it will take on the value of the same-named `zkACLProvider` system property if it is defined (in `solr.in.sh/.cmd` - see <<ZooKeeper ACLs in Solr Scripts,below>>), or if not, default to the `DefaultZkACLProvider` implementation.
+* `digest` scheme based credentialsProvider:
 
-==== Out of the Box ACL Implementations
+`org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider`: The used scheme is `digest` and it gets the `ALL` user
+credentials  (perms=all) from the specified `ZkCredentialsInjector`.
+
+If a `ZkCredentialsInjector` with an `ALL` user ( having both username and password provided) is not defined, it will fall
+back to default behavior and use the (empty) credentials list from `DefaultZkCredentialsProvider`.
+
+
+=== Set an ACL Provider
 
+
+** You control which ACLs will be added by configuring `zkACLProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkACLProvider.html[`ZkACLProvider`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkACLProvider`such that it will take on the value
+of the same-named `zkACLProvider` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkACLProvider` implementation.
+
+
+==== Out of the Box ACL Implementations
 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.
-The `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.
-*** This ACL will not be added to the list of ACLs unless both username and password are provided.
-** A user that is only allowed to perform read operations.
-*** The permission is `READ` and the schema is `digest`.
-*** The username and password are defined by system properties `zkDigestReadonlyUsername` and `zkDigestReadonlyPassword`, respectively.
-*** This ACL will not be added to the list of ACLs unless both username and password are provided.
-** Alternatively, you can set the `zkDigestCredentialsFile` system property to load the `zkDigestUsername` and `zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties for the `ALL` user, as well as the `zkDigestReadonlyUsername` and `zkDigestReadonlyPassword` properties for the `READONLY` user.
-* `org.apache.solr.common.cloud.SaslZkACLProvider`: Requires SASL authentication.
-Gives all permissions for the user specified in system property `solr.authorization.superuser` (default: `solr`) when using SASL, and gives read permissions for anyone else.
-Designed for a setup where configurations have already been set up and will not be modified, or where configuration changes are controlled via Solr APIs.
-This provider will be useful for administration in a Kerberos environment.
-In such an environment, the administrator wants Solr to authenticate to ZooKeeper using SASL, since this is only way to authenticate with ZooKeeper via Kerberos.
-
-If none of the above ACLs is added to the list, the (empty) ACL list of `DefaultZkACLProvider` will be used by default.
-
-Notice the overlap in system property names with credentials provider `VMParamsSingleSetCredentialsDigestZkCredentialsProvider` (described above).
-This is to let the two providers collaborate in a nice and perhaps common way: we always protect access to content by limiting to two users - an admin-user and a readonly-user - AND we always connect with credentials corresponding to this same admin-user, basically so that we can do anything to the content/znodes we create ourselves.
-
-You can give the readonly credentials to "clients" of your SolrCloud cluster - e.g., to be used by SolrJ clients.
-They will be able to read whatever is necessary to run a functioning SolrJ client, but they will not be able to modify any content in ZooKeeper.
+* `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.acl.DigestZkACLProvider`: This lets you define your ACLs using the defined `ZkCredentialsInjector`. 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 `all` and `read` users are injected through the `ZkCredentialsInjector` described earlier in the page.
+
+* `org.apache.solr.common.cloud.SaslZkACLProvider`: Requires SASL authentication. Gives all permissions for the user specified in system
+property `solr.authorization.superuser` (default:`solr`) when using SASL, and gives read permissions for anyone else. Designed for
+a setup where configurations have already been set up and will not be modified, or where configuration changes are
+controlled via Solr APIs. This provider will be useful for administration in a kerberos environment. In such
+an environment, the administrator wants Solr to authenticate to ZooKeeper using SASL, since this is only way to
+authenticate with ZooKeeper via Kerberos.
+
+* If none of the above ACLs is added to the list, the (empty) ACL list of `DefaultZkACLProvider` will be used by default.
+
+
+
+=== Examples
+
+
+Below examples are for `digest` scheme.
+
+* xref:#through-system-properties[System Properties]
+* xref:#through-a-file[Through a File]
+* xref:#through-aws-secret-manager[AWS Secret Manager]
+* xref:#through-a-custom-secret-manager[A Custom Secret Manager]
+* xref:#through-a-custom-credentials-injector[Custom Credentials Injector]
+
+
+==== Through System Properties
+:sectanchors:
+
+* ZK credentials are passed through System Properties via `DzkDigestUsername`, `DzkDigestPassword`, `DzkDigestReadonlyUsername`
+and `DzkDigestReadonlyPassword` properties names.
+
+[.dynamic-tabs]
+--
+[example.tab-pane#system-props-nix]
+====
+[.tab-label]**nix*
+

Review Comment:
   [+1] Awesome idea to give both nix and Windows examples here.
   
   It's a shame the Solr.in.sh/solr.in.cmd distinction requires this, but it's the perfect use of dynamic-tab.
   
   And I love your thorough use of examples here!



##########
solr/solrj/src/java/org/apache/solr/common/cloud/acl/VMParamsZkCredentialsInjector.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.common.cloud.acl;
+
+import static org.apache.solr.common.cloud.acl.ZkCredentialsInjector.ZkCredential.Perms;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import org.apache.solr.common.SolrException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Reads credentials from System Properties and injects them into {@link
+ * DigestZkCredentialsProvider} &amp; {@link DigestZkACLProvider} Usage:
+ *
+ * <pre>
+ *   -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+ *   -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \
+ *   -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+ * </pre>
+ *
+ * Or from a Java property file:
+ *
+ * <pre>
+ *   -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+ *   -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+ * </pre>
+ *
+ * Example of a Java property file:
+ *
+ * <pre>
+ * zkDigestUsername=admin-user
+ * zkDigestPassword=CHANGEME-ADMIN-PASSWORD
+ * zkDigestReadonlyUsername=readonly-user
+ * zkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+ * </pre>
+ */

Review Comment:
   [+1] Love the thorough Javadocs!



##########
solr/solr-ref-guide/modules/deployment-guide/pages/zookeeper-access-control.adoc:
##########
@@ -69,64 +97,559 @@ Solr nodes, clients, and tools (e.g., ZkCLI) always use a java class called {sol
 The implementation of the solution described here is all about changing `SolrZkClient`.
 If you use `SolrZkClient` in your application, the descriptions below will be true for your application too.
 
-=== Controlling Credentials
 
-You control which credentials provider will be used by configuring the `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}/solrj/org/apache/solr/common/cloud/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface.
+* Controlling credentials and ACLs is done in 3 steps: Set a `ZkCredentialsInjector` that reads the credentials from
+some source and then inject them into a `ZkCredentialsProvider` that Solr uses to connect to Zookeeper. ZkACLProvider
+uses the same credentials to set the ACLs.
+
+
+We will describe these 3 steps in details before giving some ready to use examples.
+
+
+. Set the `ZkCredentialsInjector`.
+. Set the `ZkCredentialsProvider`.
+. Set the `ZkACLProvider`.
+
+
+=== Set a Credentials Injector
+
+* A credentials injector gets the credentials from an external source and injects them into Solr.
+
+
+** You control which credentials will be injected by configuring `zkCredentialsInjector` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsInjector`such that it will take on the value
+of the same-named `zkCredentialsInjector` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkCredentialsInjector` implementation.
+
+==== Out of the Box Credentials Injector Implementations
+
+
+*  Solr comes with the following `ZkCredentialsInjectors`:
+
+** `org.apache.solr.common.cloud.acl.DefaultZkCredentialsInjector`: Its `getCredentials()` method returns a list of length zero,
+or "no credentials used". This is the default.
+
+** `org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector`: The username and password are defined by system
+properties name:`zkDigestUsername` and `zkDigestPassword`. This set of credentials will be added to the list of credentials
+returned by `getCredentials()` if both username and password are provided.
+
+*** If the one set of credentials above is not added to the list, this implementation will fall back to default
+behavior and use the (empty) credentials list from `DefaultZkCredentialsInjector`.
+
+*** Alternatively, you can set the `zkDigestCredentialsFile` system property to load `zkDigestUsername` and
+`zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a
+Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties.
+
+*** Usage (See full example later in the page):
+
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector
+-DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD
+-DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+
+# Or using a Java property file containing the credentials:
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector
+-DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+----
+
+
+
+** `org.apache.solr.common.cloud.acl.SecretCredentialInjector`: Used to get credentials from a Secret Manager. It retrieves the
+credentials using a {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`] passed
+through Systems Properties via `zkSecretCredentialsProvider` property name.
+
+*** The out the box implementation of `SecretCredentialsProvider` is {solr-javadocs}aws-secret-provider/org/apache/solr/secret/zk/AWSSecretCredentialsProvider.html[`AWSSecretCredentialsProvider`]  which pulls the credentials
+from  https://aws.amazon.com/secrets-manager/[AWS Secret Manager]
+
+
+*** Usage (See full example later in the page):
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+-DzkSecretCredentialsProvider=org.apache.solr.secret.zk.AWSSecretCredentialsProvider
+-DzkSecretCredentialSecretName=zkCredentialsSecret
+-DzkCredentialsAWSSecretRegion=us-west-2"
+----
+
+It expects a secret value containing the credentials in the following Json format (the secret name is `zkCredentialsSecret`):
+----
+{
+  "zkCredentials": [
+                      {"username": "admin-user", "password": "ADMIN-PASSWORD", "perms": "all"},
+                      {"username": "readonly-user", "password": "READONLY-PASSWORD", "perms": "read"}
+                  ]
+}
+----
+
+** You can have your own Secret Manager Implementation, implementing
+{solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`]  and pass it
+through `zkSecretCredentialsProvider` System Property name.
+
+----
+-DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+-DzkSecretCredentialsProvider=fully.qualified.class.CustomSecretCredentialsProvider
+-DzkSecretCredentialSecretName=zkCredentialsSecret
+----
+
+** Alternatively, you can create your own credentials injector by
+implementing {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsInjector.html[`ZkCredentialsInjector`] and pass it through System Properties using `zkCredentialsInjector` name:
+
+----
+-DzkCredentialsInjector=fully.qualified.class.CustomInjectorClassName
+----
+
+
+After the credentials are injected they are then used in the `ZkCredentialsProvider`.
+
+
+
+=== Set a Credential Provider
 
-`solr.xml` defines the `zkCredentialsProvider` such that it will take on the value of the same-named `zkCredentialsProvider` system property if it is defined (in `solr.in.sh/.cmd` - see <<ZooKeeper ACLs in Solr Scripts,below>>), or if not, default to the `DefaultZkCredentialsProvider` implementation.
 
-==== Out of the Box Credential Implementations
+
+
+`ZkCredentialsProvider` gets the credentials from the `ZkCredentialsInjector` and uses them to connect to Zookeeper.
+
+
+** You control which credentials will be used by configuring `zkCredentialsProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkCredentialsProvider.html[`ZkCredentialsProvider`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkCredentialsProvider`such that it will take on the value
+of the same-named `zkCredentialsProvider` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkCredentialsProvider` implementation.
+
+
+==== Out of the Box credentials Implementations
 
 You can always make you own implementation, but Solr comes with two implementations:
 
-* `org.apache.solr.common.cloud.DefaultZkCredentialsProvider`: Its `getCredentials()` returns a list of length zero, or "no credentials used".
-This is the default.
-* `org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider`: This lets you define your credentials using system properties.
-It supports at most one set of credentials.
-** The schema is "digest".
-The username and password are defined by system properties `zkDigestUsername` and `zkDigestPassword`.
-This set of credentials will be added to the list of credentials returned by `getCredentials()` if both username and password are provided.
-** If the one set of credentials above is not added to the list, this implementation will fall back to default behavior and use the (empty) credentials list from `DefaultZkCredentialsProvider`.
-** Alternatively, you can set the `zkDigestCredentialsFile` system property to load the `zkDigestUsername` and `zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties.
+* No credentials:
 
-=== Controlling ACLs
+`org.apache.solr.common.cloud.acl.DefaultZkCredentialsProvider`: Its `getCredentials()` returns a list of length
+zero, or "no credentials used". This is the default.
 
-You control which ACLs will be added by configuring `zkACLProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}/solrj/org/apache/solr/common/cloud/ZkACLProvider.html[`ZkACLProvider`] interface.
 
-`solr.xml` defines the `zkACLProvider` such that it will take on the value of the same-named `zkACLProvider` system property if it is defined (in `solr.in.sh/.cmd` - see <<ZooKeeper ACLs in Solr Scripts,below>>), or if not, default to the `DefaultZkACLProvider` implementation.
+* `digest` scheme based credentialsProvider:
 
-==== Out of the Box ACL Implementations
+`org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider`: The used scheme is `digest` and it gets the `ALL` user
+credentials  (perms=all) from the specified `ZkCredentialsInjector`.
+
+If a `ZkCredentialsInjector` with an `ALL` user ( having both username and password provided) is not defined, it will fall
+back to default behavior and use the (empty) credentials list from `DefaultZkCredentialsProvider`.
+
+
+=== Set an ACL Provider
 
+
+** You control which ACLs will be added by configuring `zkACLProvider` property in the `<solrcloud>` section of xref:configuration-guide:configuring-solr-xml.adoc[`solr.xml`] to the name of a class (on the classpath) implementing the {solr-javadocs}solrj/org/apache/solr/common/cloud/acl/ZkACLProvider.html[`ZkACLProvider`] interface. +
+`server/solr/solr.xml` file in the Solr distribution defines the`zkACLProvider`such that it will take on the value
+of the same-named `zkACLProvider` system property if it is defined (e.g., by uncommenting
+the `SOLR_ZK_CREDS_AND_ACLS` environment variable definition in `solr.in.sh/.cmd`- see below), or if not, default
+to the `DefaultZkACLProvider` implementation.
+
+
+==== Out of the Box ACL Implementations
 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.
-The `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.
-*** This ACL will not be added to the list of ACLs unless both username and password are provided.
-** A user that is only allowed to perform read operations.
-*** The permission is `READ` and the schema is `digest`.
-*** The username and password are defined by system properties `zkDigestReadonlyUsername` and `zkDigestReadonlyPassword`, respectively.
-*** This ACL will not be added to the list of ACLs unless both username and password are provided.
-** Alternatively, you can set the `zkDigestCredentialsFile` system property to load the `zkDigestUsername` and `zkDigestPassword` from a file instead of exposing the credentials as system properties. The provided file must be a Java properties file and contain both the `zkDigestUsername` and `zkDigestPassword` properties for the `ALL` user, as well as the `zkDigestReadonlyUsername` and `zkDigestReadonlyPassword` properties for the `READONLY` user.
-* `org.apache.solr.common.cloud.SaslZkACLProvider`: Requires SASL authentication.
-Gives all permissions for the user specified in system property `solr.authorization.superuser` (default: `solr`) when using SASL, and gives read permissions for anyone else.
-Designed for a setup where configurations have already been set up and will not be modified, or where configuration changes are controlled via Solr APIs.
-This provider will be useful for administration in a Kerberos environment.
-In such an environment, the administrator wants Solr to authenticate to ZooKeeper using SASL, since this is only way to authenticate with ZooKeeper via Kerberos.
-
-If none of the above ACLs is added to the list, the (empty) ACL list of `DefaultZkACLProvider` will be used by default.
-
-Notice the overlap in system property names with credentials provider `VMParamsSingleSetCredentialsDigestZkCredentialsProvider` (described above).
-This is to let the two providers collaborate in a nice and perhaps common way: we always protect access to content by limiting to two users - an admin-user and a readonly-user - AND we always connect with credentials corresponding to this same admin-user, basically so that we can do anything to the content/znodes we create ourselves.
-
-You can give the readonly credentials to "clients" of your SolrCloud cluster - e.g., to be used by SolrJ clients.
-They will be able to read whatever is necessary to run a functioning SolrJ client, but they will not be able to modify any content in ZooKeeper.
+* `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.acl.DigestZkACLProvider`: This lets you define your ACLs using the defined `ZkCredentialsInjector`. 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 `all` and `read` users are injected through the `ZkCredentialsInjector` described earlier in the page.
+
+* `org.apache.solr.common.cloud.SaslZkACLProvider`: Requires SASL authentication. Gives all permissions for the user specified in system
+property `solr.authorization.superuser` (default:`solr`) when using SASL, and gives read permissions for anyone else. Designed for
+a setup where configurations have already been set up and will not be modified, or where configuration changes are
+controlled via Solr APIs. This provider will be useful for administration in a kerberos environment. In such
+an environment, the administrator wants Solr to authenticate to ZooKeeper using SASL, since this is only way to
+authenticate with ZooKeeper via Kerberos.
+
+* If none of the above ACLs is added to the list, the (empty) ACL list of `DefaultZkACLProvider` will be used by default.
+
+
+
+=== Examples
+
+
+Below examples are for `digest` scheme.
+
+* xref:#through-system-properties[System Properties]
+* xref:#through-a-file[Through a File]
+* xref:#through-aws-secret-manager[AWS Secret Manager]
+* xref:#through-a-custom-secret-manager[A Custom Secret Manager]
+* xref:#through-a-custom-credentials-injector[Custom Credentials Injector]
+
+
+==== Through System Properties
+:sectanchors:
+
+* ZK credentials are passed through System Properties via `DzkDigestUsername`, `DzkDigestPassword`, `DzkDigestReadonlyUsername`
+and `DzkDigestReadonlyPassword` properties names.
+
+[.dynamic-tabs]
+--
+[example.tab-pane#system-props-nix]
+====
+[.tab-label]**nix*
+
+.solr.in.sh
+[source,bash]
+----
+
+# Settings for ZK ACL
+SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+  -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \
+  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD"
+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.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+  -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \
+  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD"
+----
+====
+
+[example.tab-pane#system-props-windows]
+====
+[.tab-label]*Windows*
+
+.solr.in.cmd
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector ^
+ -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^
+ -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS%
+----
+
+.zkcli.bat
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector ^
+ -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^
+ -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+----
+====
+--
+
+
+
+==== Through a File
+
+* Create a Java property files, for example named `zookeepercredentials.properties` containing the credentials in the following format:
+----
+zkDigestUsername=admin-user
+zkDigestPassword=CHANGEME-ADMIN-PASSWORD
+zkDigestReadonlyUsername=readonly-user
+zkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
+----
+
+* Pass the file path via System Properties:
+
+
+[.dynamic-tabs]
+--
+[example.tab-pane#file-system-props-nix]
+====
+[.tab-label]**nix*
+
+.solr.in.sh
+[source,bash]
+----
+
+# Settings for ZK ACL
+SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+  -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties"
+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.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector \
+  -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties"
+----
+====
+
+[example.tab-pane#file-system-props-windows]
+====
+[.tab-label]*Windows*
+
+.solr.in.cmd
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector ^
+ -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS%
+----
+
+.zkcli.bat
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.VMParamsZkCredentialsInjector ^
+ -DzkDigestCredentialsFile=SOLR_HOME_DIR/server/etc/zookeepercredentials.properties
+----
+====
+--
+
+
+
+==== Through AWS Secret Manager
+
+* To enable this feature, copy the jar files in `modules/aws-secret-provider/lib` to `SOLR_INSTALL/server/solr-webapp/webapp/WEB-INF/lib/` and add follow the below steps before restarting Solr.
+
+* If reusing an existing 'solr.xml' make sure to add the following line to '<solrcloud>' block:
+----
+    <str name="zkCredentialsInjector">${zkCredentialsInjector:org.apache.solr.common.cloud.acl.DefaultZkCredentialsInjector}</str>
+----
+
+* Create a secret in AWS SM (for example named `zkCredentialsSecret`) containing the Zookeeper credentials in the following Json format:
+
+----
+{
+  "zkCredentials": [
+                      {"username": "admin-user", "password": "ADMIN-PASSWORD", "perms": "all"},
+                      {"username": "readonly-user", "password": "READONLY-PASSWORD", "perms": "read"}
+                  ]
+}
+----
+
+* Pass the secret name and region using:
+
+
+[.dynamic-tabs]
+--
+[example.tab-pane#aws-sm-nix]
+====
+[.tab-label]**nix*
+
+.solr.in.sh
+[source,bash]
+----
+
+# Settings for ZK ACL
+SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+  -DzkSecretCredentialsProvider=org.apache.solr.secret.zk.AWSSecretCredentialsProvider
+  -DzkSecretCredentialSecretName=zkCredentialsSecret
+  -DzkCredentialsAWSSecretRegion=us-west-2"
+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.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector
+  -DzkSecretCredentialsProvider=org.apache.solr.secret.zk.AWSSecretCredentialsProvider
+  -DzkSecretCredentialSecretName=zkCredentialsSecret
+  -DzkCredentialsAWSSecretRegion=us-west-2"
+----
+====
+
+[example.tab-pane#aws-sm-windows]
+====
+[.tab-label]*Windows*
+
+.solr.in.cmd
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector ^
+ -DzkSecretCredentialsProvider=org.apache.solr.secret.zk.AWSSecretCredentialsProvider ^
+ -DzkSecretCredentialSecretName=zkCredentialsSecret ^
+ -DzkCredentialsAWSSecretRegion=us-west-2
+set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS%
+----
+
+.zkcli.bat
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector ^
+ -DzkSecretCredentialsProvider=org.apache.solr.secret.zk.AWSSecretCredentialsProvider ^
+ -DzkSecretCredentialSecretName=zkCredentialsSecret ^
+ -DzkCredentialsAWSSecretRegion=us-west-2
+----
+====
+--
+
+
+
+==== Through a Custom Secret Manager
+
+* Create your own Secret Manager Implementation by implementing
+{solr-javadocs}solrj/org/apache/solr/common/cloud/acl/SecretCredentialsProvider.html[`SecretCredentialsProvider`]  and pass it
+through `zkSecretCredentialsProvider` using:
+
+[.dynamic-tabs]
+--
+[example.tab-pane#custom-sm-nix]
+====
+[.tab-label]**nix*
+
+.solr.in.sh
+[source,bash]
+----
+
+# Settings for ZK ACL
+SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector \
+  -DzkSecretCredentialsProvider=fully.qualified.class.CustomSecretCredentialsProvider \
+  -DzkSecretCredentialSecretName=zkCredentialsSecret"
+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.acl.DigestZkACLProvider \
+  -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider \
+  -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector \
+  -DzkSecretCredentialsProvider=fully.qualified.class.CustomSecretCredentialsProvider \
+  -DzkSecretCredentialSecretName=zkCredentialsSecret
+----
+====
+
+[example.tab-pane#custom-sm-windows]
+====
+[.tab-label]*Windows*
+
+.solr.in.cmd
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector ^
+ -DzkSecretCredentialsProvider=fully.qualified.class.CustomSecretCredentialsProvider ^
+ -DzkSecretCredentialSecretName=zkCredentialsSecret
+set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS%
+----
+
+.zkcli.bat
+[source,powershell]
+----
+REM Settings for ZK ACL
+set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.acl.DigestZkACLProvider ^
+ -DzkCredentialsProvider=org.apache.solr.common.cloud.acl.DigestZkCredentialsProvider ^
+ -DzkCredentialsInjector=org.apache.solr.common.cloud.acl.SecretCredentialInjector ^
+ -DzkSecretCredentialsProvider=fully.qualified.class.CustomSecretCredentialsProvider ^
+ -DzkSecretCredentialSecretName=zkCredentialsSecret
+----
+====
+--
+
+
+
+
+==== Through a Custom Credentials Injector
+
+
+* Altertanitivally you can create your own credential injector by

Review Comment:
   [0] nitpick: spelling on "Alternatively"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org