Skip to content

Commit

Permalink
SOLR-17098: Only use ZK ACLs for default ZK Host
Browse files Browse the repository at this point in the history
(cherry picked from commit e2bf1f434aad873fbb24c21d46ac00e888806d98)
  • Loading branch information
HoustonPutman committed Jan 10, 2024
1 parent 6c8f24e commit 61c956c
Show file tree
Hide file tree
Showing 12 changed files with 161 additions and 11 deletions.
3 changes: 3 additions & 0 deletions solr/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,9 @@ Bug Fixes

* SOLR-16777: Schema Designer now correctly manages trust of the ConfigSets it is managing. (Ishan Chattopadhyaya, Skay, Houston Putman)

* SOLR-17098: ZK Credentials and ACLs are no longer sent to all ZK Servers when using Streaming Expressions.
They will only be used when sent to the default ZK Host. (Houston Putman, Jan Høydahl, David Smiley, Gus Heck, Qing Xu)

Optimizations
---------------------
* SOLR-16555: SolrIndexSearcher - FilterCache intersections/andNot should not clone bitsets repeatedly (Kevin Risden, David Smiley)
Expand Down
1 change: 1 addition & 0 deletions solr/core/src/java/org/apache/solr/core/CoreContainer.java
Original file line number Diff line number Diff line change
Expand Up @@ -726,6 +726,7 @@ public void load() {

zkSys.initZooKeeper(this, cfg.getCloudConfig());
if (isZooKeeperAware()) {
solrClientCache.setDefaultZKHost(getZkController().getZkServerAddress());
pkiAuthenticationSecurityBuilder = new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName(),
(PublicKeyHandler) containerHandlers.get(PublicKeyHandler.PATH));
// use deprecated API for back-compat, remove in 9.0
Expand Down
1 change: 1 addition & 0 deletions solr/solr-ref-guide/src/stream-decorator-reference.adoc
Original file line number Diff line number Diff line change
Expand Up @@ -1055,6 +1055,7 @@ The worker nodes can be from the same collection as the data, or they can be a d
* `StreamExpression`: Expression to send to the worker collection.
* `workers`: Number of workers in the worker collection to send the expression to.
* `zkHost`: (Optional) The ZooKeeper connect string where the worker collection resides.
Zookeeper Credentials and ACLs will only be included if the same ZkHost is used as the Solr instance that you are connecting to (the `chroot` can be different).
* `sort`: The sort criteria for ordering tuples returned by the worker nodes.

=== parallel Syntax
Expand Down
3 changes: 3 additions & 0 deletions solr/solr-ref-guide/src/stream-source-reference.adoc
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ This expression allows you to specify a request hander using the `qt` parameter.
* `fl`: (Mandatory) The list of fields to return.
* `sort`: (Mandatory) The sort criteria.
* `zkHost`: Only needs to be defined if the collection being searched is found in a different zkHost than the local stream handler.
Zookeeper Credentials and ACLs will only be included if the same ZkHost is used as the Solr instance that you are connecting to (the `chroot` can be different).
* `qt`: Specifies the query type, or request handler, to use. Set this to `/export` to work with large result sets. The default is `/select`.
* `rows`: (Mandatory with the `/select` handler) The rows parameter specifies how many rows to return. This parameter is only needed with the `/select` handler (which is the default) since the `/export` handler always returns all rows.
* `partitionKeys`: Comma delimited list of keys to partition the search results by. To be used with the parallel function for parallelizing operations across worker nodes. See the <<stream-decorator-reference.adoc#parallel,parallel>> function for details.
Expand Down Expand Up @@ -461,6 +462,7 @@ stream decorator to perform parallel relational algebra. When used in parallel m
* `fl`: (Mandatory) The list of fields to return.
* `sort`: (Mandatory) The sort criteria.
* `zkHost`: Only needs to be defined if the collection being searched is found in a different zkHost than the local stream handler.
Zookeeper Credentials and ACLs will only be included if the same ZkHost is used as the Solr instance that you are connecting to (the `chroot` can be different).
* `partitionKeys`: Comma delimited list of keys to partition the search results by. To be used with the parallel function for parallelizing operations across worker nodes. See the <<stream-decorator-reference.adoc#parallel,parallel>> function for details.

=== shuffle Syntax
Expand Down Expand Up @@ -592,6 +594,7 @@ The topic function should be considered in beta until https://issues.apache.org/
* `fl`: (Mandatory) The field list returned by the topic function.
* `initialCheckpoint`: (Optional) Sets the initial Solr `\_version_` number to start reading from the queue. If not set, it defaults to the highest version in the index. Setting to 0 will process all records that match query in the index.
* `zkHost`: (Optional) Only needs to be defined if the collection being searched is found in a different zkHost than the local stream handler.
Zookeeper Credentials and ACLs will only be included if the same ZkHost is used as the Solr instance that you are connecting to (the `chroot` can be different).

=== topic Syntax

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ protected CloudHttp2SolrClient(Builder builder) {
throw new IllegalArgumentException("Both zkHost(s) & solrUrl(s) have been specified. Only specify one.");
}
if (builder.zkHosts != null) {
this.stateProvider = new ZkClientClusterStateProvider(builder.zkHosts, builder.zkChroot);
this.stateProvider = new ZkClientClusterStateProvider(builder.zkHosts, builder.zkChroot, builder.canUseZkACLs);
} else if (builder.solrUrls != null && !builder.solrUrls.isEmpty()) {
try {
this.stateProvider = new Http2ClusterStateProvider(builder.solrUrls, builder.httpClient);
Expand Down Expand Up @@ -135,6 +135,7 @@ public static class Builder {
protected boolean parallelUpdates = true;
protected ClusterStateProvider stateProvider;
protected Http2SolrClient.Builder internalClientBuilder;
private boolean canUseZkACLs = true;

/**
* Provide a series of Solr URLs to be used when configuring {@link CloudHttp2SolrClient} instances.
Expand Down Expand Up @@ -182,6 +183,12 @@ public Builder(List<String> zkHosts, Optional<String> zkChroot) {
if (zkChroot.isPresent()) this.zkChroot = zkChroot.get();
}

/** Whether or not to use the default ZK ACLs when building a ZK Client. */
public Builder canUseZkACLs(boolean canUseZkACLs) {
this.canUseZkACLs = canUseZkACLs;
return this;
}

/**
* Tells {@link CloudHttp2SolrClient.Builder} that created clients should send direct updates to shard leaders only.
*
Expand Down Expand Up @@ -245,7 +252,7 @@ public Builder withInternalClientBuilder(Http2SolrClient.Builder internalClientB
public CloudHttp2SolrClient build() {
if (stateProvider == null) {
if (!zkHosts.isEmpty()) {
stateProvider = new ZkClientClusterStateProvider(zkHosts, zkChroot);
stateProvider = new ZkClientClusterStateProvider(zkHosts, zkChroot, canUseZkACLs);
}
else if (!this.solrUrls.isEmpty()) {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ protected CloudSolrClient(Builder builder) {
throw new IllegalArgumentException("Both zkHost(s) & solrUrl(s) have been specified. Only specify one.");
}
if (builder.zkHosts != null) {
this.stateProvider = new ZkClientClusterStateProvider(builder.zkHosts, builder.zkChroot);
this.stateProvider = new ZkClientClusterStateProvider(builder.zkHosts, builder.zkChroot, builder.canUseZkACLs);
} else if (builder.solrUrls != null && !builder.solrUrls.isEmpty()) {
try {
this.stateProvider = new HttpClusterStateProvider(builder.solrUrls, builder.httpClient);
Expand Down Expand Up @@ -228,6 +228,7 @@ public static class Builder extends SolrClientBuilder<Builder> {
protected boolean directUpdatesToLeadersOnly = false;
protected boolean parallelUpdates = true;
protected ClusterStateProvider stateProvider;
private boolean canUseZkACLs = true;

/**
* @deprecated use other constructors instead. This constructor will be changing visibility in an upcoming release.
Expand Down Expand Up @@ -375,6 +376,12 @@ public Builder withZkChroot(String zkChroot) {
this.zkChroot = zkChroot;
return this;
}

/** Whether or not to use the default ZK ACLs when building a ZK Client. */
public Builder canUseZkACLs(boolean canUseZkACLs) {
this.canUseZkACLs = canUseZkACLs;
return this;
}

/**
* Provides a {@link LBHttpSolrClient} for the builder to use when creating clients.
Expand Down Expand Up @@ -457,7 +464,7 @@ public Builder withClusterStateProvider(ClusterStateProvider stateProvider) {
public CloudSolrClient build() {
if (stateProvider == null) {
if (!zkHosts.isEmpty()) {
stateProvider = new ZkClientClusterStateProvider(zkHosts, zkChroot);
stateProvider = new ZkClientClusterStateProvider(zkHosts, zkChroot, canUseZkACLs);
}
else if (!this.solrUrls.isEmpty()) {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider {
volatile ZkStateReader zkStateReader;
private boolean closeZkStateReader = true;
String zkHost;
private final boolean canUseZkACLs;
int zkConnectTimeout = 15000;
int zkClientTimeout = 45000;

Expand All @@ -53,14 +54,22 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider {
public ZkClientClusterStateProvider(ZkStateReader zkStateReader) {
this.zkStateReader = zkStateReader;
this.closeZkStateReader = false;
this.canUseZkACLs = true;
}

public ZkClientClusterStateProvider(Collection<String> zkHosts, String chroot) {
zkHost = buildZkHostString(zkHosts,chroot);
this(zkHosts, chroot, true);
}

public ZkClientClusterStateProvider(
Collection<String> zkHosts, String chroot, boolean canUseZkACLs) {
zkHost = buildZkHostString(zkHosts, chroot);
this.canUseZkACLs = canUseZkACLs;
}

public ZkClientClusterStateProvider(String zkHost){
this.zkHost = zkHost;
this.canUseZkACLs = true;
}

@Override
Expand Down Expand Up @@ -173,7 +182,7 @@ public ZkStateReader getZkStateReader() {
if (zkStateReader == null) {
ZkStateReader zk = null;
try {
zk = new ZkStateReader(zkHost, zkClientTimeout, zkConnectTimeout);
zk = new ZkStateReader(zkHost, zkClientTimeout, zkConnectTimeout, canUseZkACLs);
zk.createClusterStateWatchersAndUpdate();
log.info("Cluster at {} ready", zkHost);
zkStateReader = zk;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;

import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrClient;
Expand All @@ -49,6 +50,7 @@ public class SolrClientCache implements Serializable {
//Timeouts cans be increased by setting the system properties defined below.
private static final int conTimeout = Math.max(Integer.parseInt(System.getProperty(HttpClientUtil.PROP_CONNECTION_TIMEOUT,"60000")), 60000);
private static final int socketTimeout = Math.max(Integer.parseInt(System.getProperty(HttpClientUtil.PROP_SO_TIMEOUT,"60000")), 60000);
private final AtomicReference<String> defaultZkHost = new AtomicReference<>();


public SolrClientCache() {
Expand All @@ -59,6 +61,17 @@ public SolrClientCache(HttpClient httpClient) {
this.httpClient = httpClient;
}

public void setDefaultZKHost(String zkHost) {
if (zkHost != null) {
zkHost = zkHost.split("/")[0];
if (!zkHost.isEmpty()) {
defaultZkHost.set(zkHost);
} else {
defaultZkHost.set(null);
}
}
}

public synchronized CloudSolrClient getCloudSolrClient(String zkHost) {

//Timeouts should never be lower then 60000 but they can be set higher
Expand All @@ -75,9 +88,16 @@ public synchronized CloudSolrClient getCloudSolrClient(String zkHost) {
if (solrClients.containsKey(zkHost)) {
client = (CloudSolrClient) solrClients.get(zkHost);
} else {
// Can only use ZK ACLs if there is a default ZK Host, and the given ZK host contains that
// default.
// Basically the ZK ACLs are assumed to be only used for the default ZK host,
// thus we should only provide the ACLs to that Zookeeper instance.
String zkHostNoChroot = zkHost.split("/")[0];
boolean canUseACLs =
Optional.ofNullable(defaultZkHost.get()).map(zkHostNoChroot::equals).orElse(false);
final List<String> hosts = new ArrayList<String>();
hosts.add(zkHost);
CloudSolrClient.Builder builder = new CloudSolrClient.Builder(hosts, Optional.empty()).withSocketTimeout(socketTimeout).withConnectionTimeout(conTimeout);
CloudSolrClient.Builder builder = new CloudSolrClient.Builder(hosts, Optional.empty()).withSocketTimeout(socketTimeout).withConnectionTimeout(conTimeout).canUseZkACLs(canUseACLs);
if (httpClient != null) {
builder = builder.withHttpClient(httpClient);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,13 @@ public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConne
}

public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConnectTimeout,
ZkClientConnectionStrategy strat, final OnReconnect onReconnect, BeforeReconnect beforeReconnect, ZkACLProvider zkACLProvider, IsClosed higherLevelIsClosed) {
ZkClientConnectionStrategy strat, final OnReconnect onReconnect, BeforeReconnect beforeReconnect, ZkACLProvider zkACLProvider, IsClosed higherLevelIsClosed) {
this(zkServerAddress, zkClientTimeout, clientConnectTimeout, strat, onReconnect, beforeReconnect, zkACLProvider, higherLevelIsClosed, true);
}


public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConnectTimeout,
ZkClientConnectionStrategy strat, final OnReconnect onReconnect, BeforeReconnect beforeReconnect, ZkACLProvider zkACLProvider, IsClosed higherLevelIsClosed, boolean useDefaultCredsAndACLs) {
this.zkServerAddress = zkServerAddress;
this.higherLevelIsClosed = higherLevelIsClosed;
if (strat == null) {
Expand All @@ -134,7 +140,7 @@ public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConne
this.zkClientConnectionStrategy = strat;

if (!strat.hasZkCredentialsToAddAutomatically()) {
ZkCredentialsProvider zkCredentialsToAddAutomatically = createZkCredentialsToAddAutomatically();
ZkCredentialsProvider zkCredentialsToAddAutomatically = useDefaultCredsAndACLs ? createZkCredentialsToAddAutomatically() : new DefaultZkCredentialsProvider();
strat.setZkCredentialsToAddAutomatically(zkCredentialsToAddAutomatically);
}

Expand Down Expand Up @@ -196,7 +202,7 @@ public boolean isClosed() {
}
assert ObjectReleaseTracker.track(this);
if (zkACLProvider == null) {
this.zkACLProvider = createZkACLProvider();
this.zkACLProvider = useDefaultCredsAndACLs ? createZkACLProvider() : new DefaultZkACLProvider();
} else {
this.zkACLProvider = zkACLProvider;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,16 @@ public ZkStateReader(SolrZkClient zkClient, Runnable securityNodeListener) {


public ZkStateReader(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout) {
this(zkServerAddress, zkClientTimeout, zkClientConnectTimeout, true);
}

public ZkStateReader(
String zkServerAddress,
int zkClientTimeout,
int zkClientConnectTimeout,
boolean canUseZkACLs) {
this.zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout, zkClientConnectTimeout,
new DefaultConnectionStrategy(),
// on reconnect, reload cloud info
new OnReconnect() {
@Override
Expand All @@ -370,7 +379,7 @@ public void command() {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "Interrupted", e);
}
}
});
}, null, null, null, canUseZkACLs);
this.configManager = new ZkConfigManager(zkClient);
this.closeClient = true;
this.securityNodeListener = null;
Expand Down
3 changes: 3 additions & 0 deletions solr/solrj/src/test-files/solrj/solr/solr.xml
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,9 @@
<int name="leaderVoteWait">0</int>
<int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:45000}</int>
<int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:340000}</int>
<str name="zkCredentialsProvider">${zkCredentialsProvider:org.apache.solr.common.cloud.DefaultZkCredentialsProvider}</str>
<str name="zkACLProvider">${zkACLProvider:org.apache.solr.common.cloud.DefaultZkACLProvider}</str>
<str name="zkCredentialsInjector">${zkCredentialsInjector:org.apache.solr.common.cloud.DefaultZkCredentialsInjector}</str>
</solrcloud>

</solr>
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* 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.client.solrj.io;

import java.util.Map;

import com.google.common.collect.ImmutableMap;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider;
import org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;

public class SolrClientCacheTest extends SolrCloudTestCase {

private static final Map<String, String> sysProps =
ImmutableMap.of(
SolrZkClient.ZK_CRED_PROVIDER_CLASS_NAME_VM_PARAM_NAME,
VMParamsSingleSetCredentialsDigestZkCredentialsProvider.class.getName(),
SolrZkClient.ZK_ACL_PROVIDER_CLASS_NAME_VM_PARAM_NAME,
VMParamsAllAndReadonlyDigestZkACLProvider.class.getName(),
VMParamsSingleSetCredentialsDigestZkCredentialsProvider.DEFAULT_DIGEST_USERNAME_VM_PARAM_NAME, "admin-user",
VMParamsSingleSetCredentialsDigestZkCredentialsProvider.DEFAULT_DIGEST_PASSWORD_VM_PARAM_NAME, "pass",
VMParamsAllAndReadonlyDigestZkACLProvider.DEFAULT_DIGEST_READONLY_USERNAME_VM_PARAM_NAME, "read-user",
VMParamsAllAndReadonlyDigestZkACLProvider.DEFAULT_DIGEST_READONLY_PASSWORD_VM_PARAM_NAME, "pass");

@BeforeClass
public static void before() throws Exception {
sysProps.forEach(System::setProperty);
configureCluster(1)
.addConfig("config", getFile("solrj/solr/configsets/streaming/conf").toPath())
.configure();
}

@AfterClass
public static void after() {
sysProps.keySet().forEach(System::clearProperty);
}

@Test
public void testZkACLsNotUsedWithDifferentZkHost() {
SolrClientCache cache = new SolrClientCache();
try {
// This ZK Host is fake, thus the ZK ACLs should not be used
cache.setDefaultZKHost("test:2181");
expectThrows(
SolrException.class, () -> cache.getCloudSolrClient(zkClient().getZkServerAddress()));
} finally {
cache.close();
}
}

@Test
public void testZkACLsUsedWithDifferentChroot() {
SolrClientCache cache = new SolrClientCache();
try {
// The same ZK Host is used, so the ZK ACLs should still be applied
cache.setDefaultZKHost(zkClient().getZkServerAddress() + "/random/chroot");
cache.getCloudSolrClient(zkClient().getZkServerAddress());
} finally {
cache.close();
}
}
}

0 comments on commit 61c956c

Please sign in to comment.