-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-2641: Upgrade path for ZK authentication #313
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
6a1ca42
afeafab
66b116a
36c3720
a7ae433
78ee23d
2e888de
b94fd4b
bd46f61
8c69f23
00a8169
6b2fd2a
311612f
f76c72a
fb9a52a
8314c7f
76a802d
45e39b6
83e1dc5
02f1ae2
fd799b0
cb0c751
58e17b2
943f30d
fb56da6
cf5ab6a
3c2c8e7
1a8d6b7
f072d15
1e66b3f
2cb1d73
175ff18
ecdfb54
60a0f00
a0273b8
5f49946
5e812b4
8c58e0e
1a2b361
4f85771
1fd97d6
316f831
dbb05fb
36cef5f
b317251
332b125
0674774
d9f3618
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,17 @@ | ||
| #!/bin/bash | ||
| # 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. | ||
|
|
||
| exec $(dirname $0)/kafka-run-class.sh kafka.admin.ZkSecurityMigrator $@ |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,237 @@ | ||
| /** | ||
| * 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 kafka.admin | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Need the license header.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||
|
|
||
| import java.util.concurrent.LinkedBlockingQueue | ||
| import java.util.concurrent.ThreadPoolExecutor | ||
| import java.util.concurrent.TimeUnit | ||
| import joptsimple.OptionParser | ||
| import org.I0Itec.zkclient.exception.ZkException | ||
| import kafka.utils.{Logging, ZkUtils, CommandLineUtils} | ||
| import org.apache.log4j.Level | ||
| import org.apache.kafka.common.security.JaasUtils | ||
| import org.apache.zookeeper.AsyncCallback.{ChildrenCallback, StatCallback} | ||
| import org.apache.zookeeper.data.Stat | ||
| import org.apache.zookeeper.KeeperException | ||
| import org.apache.zookeeper.KeeperException.Code | ||
| import scala.annotation.tailrec | ||
| import scala.collection.JavaConverters._ | ||
| import scala.collection._ | ||
| import scala.collection.mutable.Queue | ||
| import scala.concurrent._ | ||
| import scala.concurrent.duration._ | ||
|
|
||
| /** | ||
| * This tool is to be used when making access to ZooKeeper authenticated or | ||
| * the other way around, when removing authenticated access. The exact steps | ||
| * to migrate a Kafka cluster from unsecure to secure with respect to ZooKeeper | ||
| * access are the following: | ||
| * | ||
| * 1- Perform a rolling upgrade of Kafka servers, setting zookeeper.set.acl to false | ||
| * and passing a valid JAAS login file via the system property | ||
| * java.security.auth.login.config | ||
| * 2- Perform a second rolling upgrade keeping the system property for the login file | ||
| * and now setting zookeeper.set.acl to true | ||
| * 3- Finally run this tool. There is a script under ./bin. Run | ||
| * ./bin/zookeeper-security-migration --help | ||
| * to see the configuration parameters. An example of running it is the following: | ||
| * ./bin/zookeeper-security-migration --zookeeper.acl=secure --zookeeper.connection=localhost:2181 | ||
| * | ||
| * To convert a cluster from secure to unsecure, we need to perform the following | ||
| * steps: | ||
| * 1- Perform a rolling upgrade setting zookeeper.set.acl to false for each server | ||
| * 2- Run this migration tool, setting zookeeper.acl to unsecure | ||
| * 3- Perform another rolling upgrade to remove the system property setting the | ||
| * login file (java.security.auth.login.config). | ||
| */ | ||
|
|
||
| object ZkSecurityMigrator extends Logging { | ||
| val usageMessage = ("ZooKeeper Migration Tool Help. This tool updates the ACLs of " | ||
| + "znodes as part of the process of setting up ZooKeeper " | ||
| + "authentication.") | ||
|
|
||
| def run(args: Array[String]) { | ||
| var jaasFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) | ||
| val parser = new OptionParser() | ||
|
|
||
| val zkAclOpt = parser.accepts("zookeeper.acl", "Indicates whether to make the Kafka znodes in ZooKeeper secure or unsecure." | ||
| + " The options are 'secure' and 'unsecure'").withRequiredArg().ofType(classOf[String]) | ||
| val jaasFileOpt = parser.accepts("jaas.file", "JAAS Config file.").withOptionalArg().ofType(classOf[String]) | ||
| val zkUrlOpt = parser.accepts("zookeeper.connect", "Sets the ZooKeeper connect string (ensemble). This parameter " + | ||
| "takes a comma-separated list of host:port pairs.").withRequiredArg().defaultsTo("localhost:2181"). | ||
| ofType(classOf[String]) | ||
| val zkSessionTimeoutOpt = parser.accepts("zookeeper.session.timeout", "Sets the ZooKeeper session timeout."). | ||
| withRequiredArg().ofType(classOf[java.lang.Integer]).defaultsTo(30000) | ||
| val zkConnectionTimeoutOpt = parser.accepts("zookeeper.connection.timeout", "Sets the ZooKeeper connection timeout."). | ||
| withRequiredArg().ofType(classOf[java.lang.Integer]).defaultsTo(30000) | ||
| val helpOpt = parser.accepts("help", "Print usage information.") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To support going from secured ZK to unsecured ZK, we will need an option to enable/disable the CREATOR_ALL acl. |
||
|
|
||
| val options = parser.parse(args : _*) | ||
| if (options.has(helpOpt)) | ||
| CommandLineUtils.printUsageAndDie(parser, usageMessage) | ||
|
|
||
| if ((jaasFile == null) && !options.has(jaasFileOpt)) { | ||
| val errorMsg = ("No JAAS configuration file has been specified. Please make sure that you have set either " + | ||
| "the system property %s or the option %s".format(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, "--jaas.file")) | ||
| System.out.println("ERROR: %s".format(errorMsg)) | ||
| throw new IllegalArgumentException("Incorrect configuration") | ||
| } | ||
|
|
||
| if (jaasFile == null) { | ||
| jaasFile = options.valueOf(jaasFileOpt) | ||
| System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasFile) | ||
| } | ||
|
|
||
| if (!JaasUtils.isZkSecurityEnabled(jaasFile)) { | ||
| val errorMsg = "Security isn't enabled, most likely the file isn't set properly: %s".format(jaasFile) | ||
| System.out.println("ERROR: %s".format(errorMsg)) | ||
| throw new IllegalArgumentException("Incorrect configuration") | ||
| } | ||
|
|
||
| val zkAcl: Boolean = options.valueOf(zkAclOpt) match { | ||
| case "secure" => | ||
| info("zookeeper.acl option is secure") | ||
| true | ||
| case "unsecure" => | ||
| info("zookeeper.acl option is unsecure") | ||
| false | ||
| case _ => | ||
| CommandLineUtils.printUsageAndDie(parser, usageMessage) | ||
| } | ||
| val zkUrl = options.valueOf(zkUrlOpt) | ||
| val zkSessionTimeout = options.valueOf(zkSessionTimeoutOpt).intValue | ||
| val zkConnectionTimeout = options.valueOf(zkConnectionTimeoutOpt).intValue | ||
| val zkUtils = ZkUtils(zkUrl, zkSessionTimeout, zkConnectionTimeout, zkAcl) | ||
| val migrator = new ZkSecurityMigrator(zkUtils) | ||
| migrator.run() | ||
| } | ||
|
|
||
| def main(args: Array[String]) { | ||
| try { | ||
| run(args) | ||
| } catch { | ||
| case e: Exception => | ||
| e.printStackTrace() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| class ZkSecurityMigrator(zkUtils: ZkUtils) extends Logging { | ||
| private val workQueue = new LinkedBlockingQueue[Runnable] | ||
| private val futures = new Queue[Future[String]] | ||
|
|
||
| private def setAclsRecursively(path: String) = { | ||
| info("Setting ACL for path %s".format(path)) | ||
| val setPromise = Promise[String] | ||
| val childrenPromise = Promise[String] | ||
| futures.synchronized { | ||
| futures += setPromise.future | ||
| futures += childrenPromise.future | ||
| } | ||
| zkUtils.zkConnection.getZookeeper.setACL(path, ZkUtils.DefaultAcls(zkUtils.isSecure), -1, SetACLCallback, setPromise) | ||
| zkUtils.zkConnection.getZookeeper.getChildren(path, false, GetChildrenCallback, childrenPromise) | ||
| } | ||
|
|
||
| private object GetChildrenCallback extends ChildrenCallback { | ||
| def processResult(rc: Int, | ||
| path: String, | ||
| ctx: Object, | ||
| children: java.util.List[String]) { | ||
| val zkHandle = zkUtils.zkConnection.getZookeeper | ||
| val promise = ctx.asInstanceOf[Promise[String]] | ||
| Code.get(rc) match { | ||
| case Code.OK => | ||
| // Set ACL for each child | ||
| for (child <- children.asScala) | ||
| setAclsRecursively(s"$path/$child") | ||
| promise success "done" | ||
| case Code.CONNECTIONLOSS => | ||
| zkHandle.getChildren(path, false, GetChildrenCallback, ctx) | ||
| case Code.NONODE => | ||
| warn("Node is gone, it could be have been legitimately deleted: %s".format(path)) | ||
| promise success "done" | ||
| case Code.SESSIONEXPIRED => | ||
| // Starting a new session isn't really a problem, but it'd complicate | ||
| // the logic of the tool, so we quit and let the user re-run it. | ||
| System.out.println("ZooKeeper session expired while changing ACLs") | ||
| promise failure ZkException.create(KeeperException.create(Code.get(rc))) | ||
| case _ => | ||
| System.out.println("Unexpected return code: %d".format(rc)) | ||
| promise failure ZkException.create(KeeperException.create(Code.get(rc))) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private object SetACLCallback extends StatCallback { | ||
| def processResult(rc: Int, | ||
| path: String, | ||
| ctx: Object, | ||
| stat: Stat) { | ||
| val zkHandle = zkUtils.zkConnection.getZookeeper | ||
| val promise = ctx.asInstanceOf[Promise[String]] | ||
|
|
||
| Code.get(rc) match { | ||
| case Code.OK => | ||
| info("Successfully set ACLs for %s".format(path)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems that we need to print this instead of logging it. |
||
| promise success "done" | ||
| case Code.CONNECTIONLOSS => | ||
| zkHandle.setACL(path, ZkUtils.DefaultAcls(zkUtils.isSecure), -1, SetACLCallback, ctx) | ||
| case Code.NONODE => | ||
| warn("Znode is gone, it could be have been legitimately deleted: %s".format(path)) | ||
| promise success "done" | ||
| case Code.SESSIONEXPIRED => | ||
| // Starting a new session isn't really a problem, but it'd complicate | ||
| // the logic of the tool, so we quit and let the user re-run it. | ||
| System.out.println("ZooKeeper session expired while changing ACLs") | ||
| promise failure ZkException.create(KeeperException.create(Code.get(rc))) | ||
| case _ => | ||
| System.out.println("Unexpected return code: %d".format(rc)) | ||
| promise failure ZkException.create(KeeperException.create(Code.get(rc))) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private def run(): Unit = { | ||
| try { | ||
| for (path <- zkUtils.securePersistentZkPaths) { | ||
| debug("Going to set ACL for %s".format(path)) | ||
| zkUtils.makeSurePersistentPathExists(path) | ||
| setAclsRecursively(path) | ||
| } | ||
|
|
||
| @tailrec | ||
| def recurse(): Unit = { | ||
| val future = futures.synchronized { | ||
| futures.headOption | ||
| } | ||
| future match { | ||
| case Some(a) => | ||
| Await.result(a, 6000 millis) | ||
| futures.synchronized { futures.dequeue } | ||
| recurse | ||
| case None => | ||
| } | ||
| } | ||
| recurse() | ||
|
|
||
| } finally { | ||
| zkUtils.close | ||
| } | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -253,12 +253,17 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr | |
| "" | ||
| } | ||
|
|
||
| val secureAclsEnabled = JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)) && config.zkEnableSecureAcls | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sorry if this was already discussed, but shouldn't we be throwing a configuration error if someone sets
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Agreed.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||
|
|
||
| if(config.zkEnableSecureAcls && !secureAclsEnabled) { | ||
| throw new java.lang.SecurityException("zkEnableSecureAcls is true, but the verification of the JAAS login file failed.") | ||
| } | ||
| if (chroot.length > 1) { | ||
| val zkConnForChrootCreation = config.zkConnect.substring(0, config.zkConnect.indexOf("/")) | ||
| val zkClientForChrootCreation = ZkUtils(zkConnForChrootCreation, | ||
| config.zkSessionTimeoutMs, | ||
| config.zkConnectionTimeoutMs, | ||
| JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM))) | ||
| secureAclsEnabled) | ||
| zkClientForChrootCreation.makeSurePersistentPathExists(chroot) | ||
| info("Created zookeeper path " + chroot) | ||
| zkClientForChrootCreation.zkClient.close() | ||
|
|
@@ -267,7 +272,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr | |
| val zkUtils = ZkUtils(config.zkConnect, | ||
| config.zkSessionTimeoutMs, | ||
| config.zkConnectionTimeoutMs, | ||
| JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM))) | ||
| secureAclsEnabled) | ||
| zkUtils.setupCommonPaths() | ||
| zkUtils | ||
| } | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As we were discussing with @junrao, I think we need to either find a way not to need this or to at least do it in a safe way within our app (ie reload it atomically).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah, I'm not comfortable with this at all, it doesn't sound robust. at the same time, I want to separate bugs in my code from bugs elsewhere, so I'm leaving this here for now, I need to understand what's causing this.