You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by th...@apache.org on 2021/06/17 14:47:47 UTC
[solr] branch main updated: SOLR-15451: SolrSchema (for SQL) should
use PKI principal for request to /admin/luke (#168)
This is an automated email from the ASF dual-hosted git repository.
thelabdude pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/solr.git
The following commit(s) were added to refs/heads/main by this push:
new b24d73e SOLR-15451: SolrSchema (for SQL) should use PKI principal for request to /admin/luke (#168)
b24d73e is described below
commit b24d73ec1042de92f8f048a5d5263e41414019ab
Author: Timothy Potter <th...@gmail.com>
AuthorDate: Thu Jun 17 08:47:38 2021 -0600
SOLR-15451: SolrSchema (for SQL) should use PKI principal for request to /admin/luke (#168)
---
solr/CHANGES.txt | 2 +
.../org/apache/solr/handler/sql/SolrSchema.java | 5 +
.../org/apache/solr/request/SolrRequestInfo.java | 13 +++
.../solr/security/PKIAuthenticationPlugin.java | 18 +++-
.../solr/handler/sql/SQLWithAuthzEnabledTest.java | 120 +++++++++++++++++++++
.../solr-ref-guide/src/parallel-sql-interface.adoc | 9 ++
.../solr/client/solrj/io/stream/SolrStream.java | 28 ++++-
7 files changed, 192 insertions(+), 3 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c7a611c..cd23b4a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -323,6 +323,8 @@ Other Changes
* SOLR-15471: Rename lang id whitelist parameter to allowlist (Mike Drob)
+* SOLR-15451: SolrSchema (for Parallel SQL) should use PKI principal for internal request to /admin/luke to get table metadata (Timothy Potter)
+
Bug Fixes
---------------------
* SOLR-14546: Fix for a relatively hard to hit issue in OverseerTaskProcessor that could lead to out of order execution
diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
index 3bf5bd4..caa3076 100644
--- a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
+++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
@@ -40,6 +40,7 @@ import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ZkStateReader;
import com.google.common.collect.ImmutableMap;
+import org.apache.solr.security.PKIAuthenticationPlugin;
class SolrSchema extends AbstractSchema implements Closeable {
final Properties properties;
@@ -93,6 +94,8 @@ class SolrSchema extends AbstractSchema implements Closeable {
private Map<String, LukeResponse.FieldInfo> getFieldInfo(String collection) {
String zk = this.properties.getProperty("zk");
CloudSolrClient cloudSolrClient = solrClientCache.getCloudSolrClient(zk);
+ // Send the Luke request using the server identity vs. the logged in user
+ PKIAuthenticationPlugin.withServerIdentity(true);
try {
LukeRequest lukeRequest = new LukeRequest();
lukeRequest.setNumTerms(0);
@@ -100,6 +103,8 @@ class SolrSchema extends AbstractSchema implements Closeable {
return lukeResponse.getFieldInfo();
} catch (SolrServerException | IOException e) {
throw new RuntimeException(e);
+ } finally {
+ PKIAuthenticationPlugin.withServerIdentity(false);
}
}
diff --git a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
index aa10d10..3771a65 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
@@ -52,6 +52,7 @@ public class SolrRequestInfo {
protected ResponseBuilder rb;
protected List<Closeable> closeHooks;
protected SolrDispatchFilter.Action action;
+ protected boolean useServerToken = false;
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -200,6 +201,18 @@ public class SolrRequestInfo {
this.action = action;
}
+ /**
+ * Used when making remote requests to other Solr nodes from the thread associated with this request,
+ * true means the server token header should be used instead of the Principal associated with the request.
+ */
+ public boolean useServerToken() {
+ return useServerToken;
+ }
+
+ public void setUseServerToken(boolean use) {
+ this.useServerToken = use;
+ }
+
public static ExecutorUtil.InheritableThreadLocalProvider getInheritableThreadLocalProvider() {
return new ExecutorUtil.InheritableThreadLocalProvider() {
@Override
diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index e506309..6a66f5a 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -58,6 +58,22 @@ import org.slf4j.LoggerFactory;
import static java.nio.charset.StandardCharsets.UTF_8;
public class PKIAuthenticationPlugin extends AuthenticationPlugin implements HttpClientBuilderPlugin {
+
+ /**
+ * Mark the current thread as a server thread and set a flag in SolrRequestInfo to indicate you want
+ * to send a request as the server identity instead of as the authenticated user.
+ *
+ * @param enabled If true, enable the current thread to make requests with the server identity.
+ * @see SolrRequestInfo#setUseServerToken(boolean)
+ */
+ public static void withServerIdentity(final boolean enabled) {
+ SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
+ if (requestInfo != null) {
+ requestInfo.setUseServerToken(enabled);
+ }
+ ExecutorUtil.setServerThreadFlag(enabled ? enabled : null);
+ }
+
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final Map<String, PublicKey> keyCache = new ConcurrentHashMap<>();
private final PublicKeyHandler publicKeyHandler;
@@ -282,7 +298,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
private Optional<String> generateToken() {
SolrRequestInfo reqInfo = getRequestInfo();
String usr;
- if (reqInfo != null) {
+ if (reqInfo != null && !reqInfo.useServerToken()) {
Principal principal = reqInfo.getUserPrincipal();
if (principal == null) {
log.debug("generateToken: principal is null");
diff --git a/solr/core/src/test/org/apache/solr/handler/sql/SQLWithAuthzEnabledTest.java b/solr/core/src/test/org/apache/solr/handler/sql/SQLWithAuthzEnabledTest.java
new file mode 100644
index 0000000..3205988
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/sql/SQLWithAuthzEnabledTest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.handler.sql;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.SolrStream;
+import org.apache.solr.client.solrj.io.stream.TupleStream;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.security.BasicAuthPlugin;
+import org.apache.solr.security.RuleBasedAuthorizationPlugin;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.security.Sha256AuthenticationProvider.getSaltedHashedValue;
+
+public class SQLWithAuthzEnabledTest extends SolrCloudTestCase {
+
+ private static final String ADMIN_USER = "solr";
+ private static final String SQL_USER = "sql";
+ private static final String SAD_USER = "sad";
+ private static final String PASS = "SolrRocks!!";
+ private static final String collectionName = "testSQLWithAuthz";
+
+ @BeforeClass
+ public static void setupClusterWithSecurityEnabled() throws Exception {
+ final String SECURITY_JSON = Utils.toJSONString
+ (Map.of("authorization",
+ Map.of("class", RuleBasedAuthorizationPlugin.class.getName(),
+ "user-role", Map.of(SQL_USER, "sql", ADMIN_USER, "admin", SAD_USER, "sad"),
+ "permissions", Arrays.asList(
+ Map.of("name", "sql", "role", "sql", "path", "/sql", "collection", "*"),
+ Map.of("name", "export", "role", "sql", "path", "/export", "collection", "*"),
+ Map.of("name", "all", "role", "admin"))
+ ),
+ "authentication",
+ Map.of("class", BasicAuthPlugin.class.getName(),
+ "blockUnknown", true,
+ "credentials", Map.of(
+ SAD_USER, getSaltedHashedValue(PASS),
+ SQL_USER, getSaltedHashedValue(PASS),
+ ADMIN_USER, getSaltedHashedValue(PASS)))));
+
+ configureCluster(2)
+ .addConfig("conf", configset("sql"))
+ .withSecurityJson(SECURITY_JSON)
+ .configure();
+ }
+
+ private <T extends SolrRequest<? extends SolrResponse>> T doAsAdmin(T req) {
+ req.setBasicAuthCredentials(ADMIN_USER, PASS);
+ return req;
+ }
+
+ @Test
+ public void testSqlAuthz() throws Exception {
+ doAsAdmin(CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2, 0, 0)).process(cluster.getSolrClient());
+ waitForState("Expected collection to be created with 1 shards and 2 replicas", collectionName, clusterShape(1, 2));
+
+ doAsAdmin(new UpdateRequest()
+ .add("id", "1")
+ .add("id", "2")
+ .add("id", "3")
+ .add("id", "4")
+ .add("id", "5")
+ .add("id", "6")
+ .add("id", "7")
+ .add("id", "8")).commit(cluster.getSolrClient(), collectionName);
+
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ params.set(CommonParams.QT, "/sql");
+ params.set("stmt", "select id from " + collectionName);
+ String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/" + collectionName;
+ SolrStream solrStream = new SolrStream(baseUrl, params);
+ solrStream.setCredentials(SAD_USER, PASS);
+
+ // sad user is not authorized to access /sql endpoints
+ expectThrows(IOException.class, () -> countTuples(solrStream));
+
+ // sql user has access
+ SolrStream solrStream2 = new SolrStream(baseUrl, params);
+ solrStream2.setCredentials(SQL_USER, PASS);
+ assertEquals(8, countTuples(solrStream2));
+ }
+
+ private int countTuples(TupleStream tupleStream) throws IOException {
+ int count = 0;
+ try (tupleStream) {
+ tupleStream.open();
+ for (Tuple t = tupleStream.read(); !t.EOF; t = tupleStream.read())
+ count++;
+ }
+ return count;
+ }
+}
diff --git a/solr/solr-ref-guide/src/parallel-sql-interface.adoc b/solr/solr-ref-guide/src/parallel-sql-interface.adoc
index 34f7fe6..3571221 100644
--- a/solr/solr-ref-guide/src/parallel-sql-interface.adoc
+++ b/solr/solr-ref-guide/src/parallel-sql-interface.adoc
@@ -66,6 +66,15 @@ The `/sql` handler is the front end of the Parallel SQL interface. All SQL queri
By default, the `/sql` request handler is configured as an implicit handler, meaning that it is always enabled in every Solr installation and no further configuration is required.
+==== Authorization for SQL Requests
+
+If your Solr cluster is configured to use the <<rule-based-authorization-plugin.adoc#,Rule-based Authorization Plugin>>,
+then you need to grant `GET` and `POST` permission on the `/sql`, `/select`, and `/export` endpoints for all collections you intend to execute SQL queries against.
+The `/select` endpoint is used for `LIMIT` queries, whereas the `/export` handler is used for queries without a `LIMIT`, so in most cases, you'll want to grant access to both.
+If you're using a worker collection for the `/sql` handler, then you only need to grant access to the `/sql` endpoint for the worker collection and not the collections in the data tier.
+Behind the scenes, the SQL handler also sends requests using the internal Solr server identity to the `/admin/luke` endpoint to get schema metadata for a collection.
+Consequently, you do not need to grant explicit permission to the `/admin/luke` endpoint for users to execute SQL queries.
+
[IMPORTANT]
====
As described below in the section <<Best Practices>>, you may want to set up a separate collection for parallelized SQL queries. If you have high cardinality fields and a large amount of data, please be sure to review that section and consider using a separate collection.
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java
index e6210ee..27fc622 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java
@@ -125,9 +125,11 @@ public class SolrStream extends TupleStream {
try {
SolrParams requestParams = loadParams(params);
if (!distrib) {
- ((ModifiableSolrParams) requestParams).add("distrib","false");
+ ((ModifiableSolrParams) requestParams).add("distrib", "false");
}
tupleStreamParser = constructParser(requestParams);
+ } catch (IOException ioe) {
+ throw ioe;
} catch (Exception e) {
throw new IOException("params " + params, e);
}
@@ -303,7 +305,16 @@ public class SolrStream extends TupleStream {
NamedList<Object> genericResponse = client.request(query);
InputStream stream = (InputStream) genericResponse.get("stream");
- this.closeableHttpResponse = (CloseableHttpResponse)genericResponse.get("closeableResponse");
+ CloseableHttpResponse httpResponse = (CloseableHttpResponse)genericResponse.get("closeableResponse");
+
+ final int statusCode = httpResponse.getStatusLine().getStatusCode();
+ if (statusCode != 200) {
+ String errMsg = consumeStreamAsErrorMessage(stream);
+ httpResponse.close();
+ throw new IOException("Query to '" + query.getPath() + "?" + query.getParams() + "' failed due to: (" + statusCode + ") " + errMsg);
+ }
+
+ this.closeableHttpResponse = httpResponse;
if (CommonParams.JAVABIN.equals(wt)) {
return new JavabinTupleStreamParser(stream, true);
} else {
@@ -311,4 +322,17 @@ public class SolrStream extends TupleStream {
return new JSONTupleStream(reader);
}
}
+
+ private String consumeStreamAsErrorMessage(InputStream stream) throws IOException {
+ StringBuilder errMsg = new StringBuilder();
+ int r;
+ char[] ach = new char[1024];
+ if (stream != null) {
+ try (InputStreamReader reader = new InputStreamReader(stream, StandardCharsets.UTF_8)) {
+ while ((r = reader.read(ach)) != -1)
+ errMsg.append(ach, 0, r);
+ }
+ }
+ return errMsg.toString();
+ }
}