Repository: lucene-solr
Updated Branches:
refs/heads/branch_6_6 927d10660 -> 4704efb1f
Create znode upfront and fix chroot handling in delegation token feature
(cherry picked from commit 065edbc)
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/4704efb1
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/4704efb1
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/4704efb1
Branch: refs/heads/branch_6_6
Commit: 4704efb1f92588ad106a39dea5f20bf8e80b133b
Parents: 927d106
Author: Shalin Shekhar Mangar <shalin@apache.org>
Authored: Wed Aug 16 18:50:20 2017 +0530
Committer: Shalin Shekhar Mangar <shalin@apache.org>
Committed: Wed Aug 16 18:55:04 2017 +0530
----------------------------------------------------------------------
.../security/DelegationTokenKerberosFilter.java | 42 ++++-
.../apache/solr/security/HadoopAuthFilter.java | 42 ++++-
.../apache/solr/security/HadoopAuthPlugin.java | 3 +-
.../hadoop/TestZkAclsWithHadoopAuth.java | 165 +++++++++++++++++++
.../apache/solr/cloud/MiniSolrCloudCluster.java | 2 +
5 files changed, 243 insertions(+), 11 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4704efb1/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
index 007e0bd..ce3544c 100644
--- a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
@@ -46,6 +46,8 @@ import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkACLProvider;
import org.apache.solr.common.cloud.ZkCredentialsProvider;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.ACL;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -65,8 +67,12 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider")))
{
SolrZkClient zkClient =
(SolrZkClient)conf.getServletContext().getAttribute(KerberosPlugin.DELEGATION_TOKEN_ZK_CLIENT);
- conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
- getCuratorClient(zkClient));
+ try {
+ conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
+ getCuratorClient(zkClient));
+ } catch (InterruptedException | KeeperException e) {
+ throw new ServletException(e);
+ }
}
super.init(conf);
}
@@ -147,7 +153,7 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
newAuthHandler.setAuthHandler(authHandler);
}
- protected CuratorFramework getCuratorClient(SolrZkClient zkClient) {
+ protected CuratorFramework getCuratorClient(SolrZkClient zkClient) throws InterruptedException,
KeeperException {
// should we try to build a RetryPolicy off of the ZkController?
RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
if (zkClient == null) {
@@ -161,6 +167,17 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient);
final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561.
+ // Create /security znode upfront. Without this, the curator framework creates this directory
path
+ // without the appropriate ACL configuration. This issue is possibly related to HADOOP-11973
+ try {
+ zkClient.makePath(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, CreateMode.PERSISTENT,
true);
+
+ } catch (KeeperException ex) {
+ if (ex.code() != KeeperException.Code.NODEEXISTS) {
+ throw ex;
+ }
+ }
+
curatorFramework = CuratorFrameworkFactory.builder()
.namespace(zkNamespace)
.connectString(zkConnectionString)
@@ -178,12 +195,15 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
* Convert Solr Zk Credentials/ACLs to Curator versions
*/
protected static class SolrZkToCuratorCredentialsACLs {
+ private final String zkChroot;
private final ACLProvider aclProvider;
private final List<AuthInfo> authInfos;
public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) {
this.aclProvider = createACLProvider(zkClient);
this.authInfos = createAuthInfo(zkClient);
+ String zkHost = zkClient.getZkServerAddress();
+ this.zkChroot = zkHost.contains("/")? zkHost.substring(zkHost.indexOf("/")): null;
}
public ACLProvider getACLProvider() { return aclProvider; }
@@ -199,8 +219,20 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
@Override
public List<ACL> getAclForPath(String path) {
- List<ACL> acls = zkACLProvider.getACLsToAdd(path);
- return acls;
+ List<ACL> acls = null;
+
+ // The logic in SecurityAwareZkACLProvider does not work when
+ // the Solr zkPath is chrooted (e.g. /solr instead of /). This
+ // due to the fact that the getACLsToAdd(..) callback provides
+ // an absolute path (instead of relative path to the chroot) and
+ // the string comparison in SecurityAwareZkACLProvider fails.
+ if (zkACLProvider instanceof SecurityAwareZkACLProvider && zkChroot !=
null) {
+ acls = zkACLProvider.getACLsToAdd(path.replace(zkChroot, ""));
+ } else {
+ acls = zkACLProvider.getACLsToAdd(path);
+ }
+
+ return acls;
}
};
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4704efb1/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
index fb35e72..205becc 100644
--- a/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
@@ -43,6 +43,8 @@ import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkACLProvider;
import org.apache.solr.common.cloud.ZkCredentialsProvider;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.ACL;
/**
@@ -62,8 +64,12 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter
{
if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider")))
{
SolrZkClient zkClient =
(SolrZkClient)conf.getServletContext().getAttribute(DELEGATION_TOKEN_ZK_CLIENT);
- conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
- getCuratorClient(zkClient));
+ try {
+ conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
+ getCuratorClient(zkClient));
+ } catch (KeeperException | InterruptedException e) {
+ throw new ServletException(e);
+ }
}
super.init(conf);
}
@@ -125,7 +131,7 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter
{
newAuthHandler.setAuthHandler(authHandler);
}
- protected CuratorFramework getCuratorClient(SolrZkClient zkClient) {
+ protected CuratorFramework getCuratorClient(SolrZkClient zkClient) throws KeeperException,
InterruptedException {
// should we try to build a RetryPolicy off of the ZkController?
RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
if (zkClient == null) {
@@ -139,6 +145,17 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter
{
SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient);
final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561.
+ // Create /security znode upfront. Without this, the curator framework creates this directory
path
+ // without the appropriate ACL configuration. This issue is possibly related to HADOOP-11973
+ try {
+ zkClient.makePath(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, CreateMode.PERSISTENT,
true);
+
+ } catch (KeeperException ex) {
+ if (ex.code() != KeeperException.Code.NODEEXISTS) {
+ throw ex;
+ }
+ }
+
curatorFramework = CuratorFrameworkFactory.builder()
.namespace(zkNamespace)
.connectString(zkConnectionString)
@@ -156,12 +173,15 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter
{
* Convert Solr Zk Credentials/ACLs to Curator versions
*/
protected static class SolrZkToCuratorCredentialsACLs {
+ private final String zkChroot;
private final ACLProvider aclProvider;
private final List<AuthInfo> authInfos;
public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) {
this.aclProvider = createACLProvider(zkClient);
this.authInfos = createAuthInfo(zkClient);
+ String zkHost = zkClient.getZkServerAddress();
+ this.zkChroot = zkHost.contains("/")? zkHost.substring(zkHost.indexOf("/")): null;
}
public ACLProvider getACLProvider() { return aclProvider; }
@@ -177,8 +197,20 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter
{
@Override
public List<ACL> getAclForPath(String path) {
- List<ACL> acls = zkACLProvider.getACLsToAdd(path);
- return acls;
+ List<ACL> acls = null;
+
+ // The logic in SecurityAwareZkACLProvider does not work when
+ // the Solr zkPath is chrooted (e.g. /solr instead of /). This
+ // due to the fact that the getACLsToAdd(..) callback provides
+ // an absolute path (instead of relative path to the chroot) and
+ // the string comparison in SecurityAwareZkACLProvider fails.
+ if (zkACLProvider instanceof SecurityAwareZkACLProvider && zkChroot !=
null) {
+ acls = zkACLProvider.getACLsToAdd(path.replace(zkChroot, ""));
+ } else {
+ acls = zkACLProvider.getACLsToAdd(path);
+ }
+
+ return acls;
}
};
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4704efb1/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
index 68a8585..341f4a8 100644
--- a/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
@@ -144,7 +144,8 @@ public class HadoopAuthPlugin extends AuthenticationPlugin {
authFilter.init(conf);
} catch (ServletException e) {
- throw new SolrException(ErrorCode.SERVER_ERROR, "Error initializing kerberos authentication
plugin: "+e);
+ log.error("Error initializing " + getClass().getSimpleName(), e);
+ throw new SolrException(ErrorCode.SERVER_ERROR, "Error initializing " + getClass().getName()
+ ": "+e);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4704efb1/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
b/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
new file mode 100644
index 0000000..3bc1c0b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
@@ -0,0 +1,165 @@
+/*
+ * 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.security.hadoop;
+
+import static org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider.DEFAULT_DIGEST_READONLY_PASSWORD_VM_PARAM_NAME;
+import static org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider.DEFAULT_DIGEST_READONLY_USERNAME_VM_PARAM_NAME;
+import static org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider.DEFAULT_DIGEST_PASSWORD_VM_PARAM_NAME;
+import static org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider.DEFAULT_DIGEST_USERNAME_VM_PARAM_NAME;
+
+import java.nio.charset.StandardCharsets;
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.Constants;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider;
+import org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestZkAclsWithHadoopAuth extends SolrCloudTestCase {
+ protected static final int NUM_SERVERS = 1;
+ protected static final int NUM_SHARDS = 1;
+ protected static final int REPLICATION_FACTOR = 1;
+ private static final String SOLR_PASSWD = "solr";
+ private static final String FOO_PASSWD = "foo";
+ private static final Id SOLR_ZK_ID = new Id("digest", digest ("solr", SOLR_PASSWD));
+ private static final Id FOO_ZK_ID = new Id("digest", digest ("foo", FOO_PASSWD));
+
+ @BeforeClass
+ public static void setupClass() throws Exception {
+ assumeFalse("Hadoop does not work on Windows", Constants.WINDOWS);
+ assumeFalse("FIXME: SOLR-8182: This test fails under Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+
+ System.setProperty(SolrZkClient.ZK_ACL_PROVIDER_CLASS_NAME_VM_PARAM_NAME,
+ VMParamsAllAndReadonlyDigestZkACLProvider.class.getName());
+ System.setProperty(SolrZkClient.ZK_CRED_PROVIDER_CLASS_NAME_VM_PARAM_NAME,
+ VMParamsSingleSetCredentialsDigestZkCredentialsProvider.class.getName());
+ System.setProperty(DEFAULT_DIGEST_USERNAME_VM_PARAM_NAME, "solr");
+ System.setProperty(DEFAULT_DIGEST_PASSWORD_VM_PARAM_NAME, SOLR_PASSWD);
+ System.setProperty(DEFAULT_DIGEST_READONLY_USERNAME_VM_PARAM_NAME, "foo");
+ System.setProperty(DEFAULT_DIGEST_READONLY_PASSWORD_VM_PARAM_NAME, FOO_PASSWD);
+
+ configureCluster(NUM_SERVERS)// nodes
+ .withSolrXml(MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML)
+ .withSecurityJson(TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegation.json"))
+ .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+ .configure();
+ }
+
+ @AfterClass
+ public static void tearDownClass() {
+ System.clearProperty(SolrZkClient.ZK_ACL_PROVIDER_CLASS_NAME_VM_PARAM_NAME);
+ System.clearProperty(SolrZkClient.ZK_CRED_PROVIDER_CLASS_NAME_VM_PARAM_NAME);
+ System.clearProperty(DEFAULT_DIGEST_USERNAME_VM_PARAM_NAME);
+ System.clearProperty(DEFAULT_DIGEST_PASSWORD_VM_PARAM_NAME);
+ System.clearProperty(DEFAULT_DIGEST_READONLY_USERNAME_VM_PARAM_NAME);
+ System.clearProperty(DEFAULT_DIGEST_READONLY_PASSWORD_VM_PARAM_NAME);
+ }
+
+ @Test
+ public void testZkAcls() throws Exception {
+ ZooKeeper keeper = null;
+ try {
+ keeper = new ZooKeeper(cluster.getZkServer().getZkAddress(), (int) TimeUnit.MINUTES.toMillis(1),
new Watcher() {
+ @Override
+ public void process(WatchedEvent arg0) {
+ // Do nothing
+ }
+ });
+
+ keeper.addAuthInfo("digest", ("solr:"+SOLR_PASSWD).getBytes(StandardCharsets.UTF_8));
+
+ // Test well known paths.
+ checkNonSecurityACLs(keeper, "/solr.xml");
+ checkSecurityACLs(keeper, "/security/token");
+ checkSecurityACLs(keeper, "/security");
+
+ // Now test all ZK tree.
+ String zkHost = cluster.getSolrClient().getZkHost();
+ String zkChroot = zkHost.contains("/")? zkHost.substring(zkHost.indexOf("/")): null;
+ walkZkTree(keeper, zkChroot, "/");
+
+ } finally {
+ if (keeper != null) {
+ keeper.close();
+ }
+ }
+ }
+
+ private void walkZkTree (ZooKeeper keeper, String zkChroot, String path) throws Exception
{
+ if (isSecurityZNode(zkChroot, path)) {
+ checkSecurityACLs(keeper, path);
+ } else {
+ checkNonSecurityACLs(keeper, path);
+ }
+
+ List<String> children = keeper.getChildren(path, false);
+ for (String child : children) {
+ String subpath = path.endsWith("/") ? path + child : path + "/" + child;
+ walkZkTree(keeper, zkChroot, subpath);
+ }
+ }
+
+ private boolean isSecurityZNode(String zkChroot, String path) {
+ String temp = path;
+ if (zkChroot != null) {
+ temp = path.replace(zkChroot, "");
+ }
+ return !ZkStateReader.SOLR_SECURITY_CONF_PATH.equals(path) &&
+ temp.startsWith(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH);
+ }
+
+ private void checkSecurityACLs(ZooKeeper keeper, String path) throws Exception {
+ List<ACL> acls = keeper.getACL(path, new Stat());
+ String message = String.format(Locale.ROOT, "Path %s ACLs found %s", path, acls);
+ assertEquals(message, 1, acls.size());
+ assertTrue(message, acls.contains(new ACL(ZooDefs.Perms.ALL, SOLR_ZK_ID)));
+ }
+
+ private void checkNonSecurityACLs(ZooKeeper keeper, String path) throws Exception {
+ List<ACL> acls = keeper.getACL(path, new Stat());
+ String message = String.format(Locale.ROOT, "Path %s ACLs found %s", path, acls);
+ assertEquals(message, 2, acls.size());
+ assertTrue(message, acls.contains(new ACL(ZooDefs.Perms.ALL, SOLR_ZK_ID)));
+ assertTrue(message, acls.contains(new ACL(ZooDefs.Perms.READ, FOO_ZK_ID)));
+ }
+
+ private static String digest (String userName, String passwd) {
+ try {
+ return DigestAuthenticationProvider.generateDigest(userName+":"+passwd);
+ } catch (NoSuchAlgorithmException ex) {
+ throw new RuntimeException(ex);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4704efb1/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index dc90049..76e0d75 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -95,6 +95,8 @@ public class MiniSolrCloudCluster {
" <int name=\"leaderVoteWait\">10000</int>\n" +
" <int name=\"distribUpdateConnTimeout\">${distribUpdateConnTimeout:45000}</int>\n"
+
" <int name=\"distribUpdateSoTimeout\">${distribUpdateSoTimeout:340000}</int>\n"
+
+ " <str name=\"zkCredentialsProvider\">${zkCredentialsProvider:org.apache.solr.common.cloud.DefaultZkCredentialsProvider}</str>
\n" +
+ " <str name=\"zkACLProvider\">${zkACLProvider:org.apache.solr.common.cloud.DefaultZkACLProvider}</str>
\n" +
" </solrcloud>\n" +
" \n" +
"</solr>\n";
|