From 39290304fd980ed6b1d8ee86213c402340ea1f0c Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Fri, 27 Oct 2017 13:20:27 +0530 Subject: [PATCH 1/8] KAFKA-5645: Use async ZookeeperClient in SimpleAclAuthorizer --- .../ZkNodeChangeNotificationListener.scala | 108 +++++----- .../main/scala/kafka/security/auth/Acl.scala | 11 +- .../security/auth/SimpleAclAuthorizer.scala | 122 ++++++----- .../kafka/server/DynamicConfigManager.scala | 16 +- .../main/scala/kafka/server/KafkaServer.scala | 4 +- .../main/scala/kafka/zk/KafkaZkClient.scala | 191 ++++++++++++++++++ core/src/main/scala/kafka/zk/ZkData.scala | 41 ++++ .../kafka/zookeeper/ZooKeeperClient.scala | 20 +- ...ZkNodeChangeNotificationListenerTest.scala | 21 +- .../unit/kafka/security/auth/AclTest.scala | 8 +- .../auth/SimpleAclAuthorizerTest.scala | 6 +- .../server/DynamicConfigChangeTest.scala | 2 +- .../unit/kafka/zk/KafkaZkClientTest.scala | 19 +- .../unit/kafka/zk/ZooKeeperTestHarness.scala | 11 + .../kafka/zookeeper/ZooKeeperClientTest.scala | 2 + 15 files changed, 404 insertions(+), 178 deletions(-) diff --git a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala index 0e34c5a9b39dc..11cd449369793 100644 --- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala @@ -16,12 +16,12 @@ */ package kafka.common +import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.atomic.AtomicBoolean -import kafka.utils.{Logging, ZkUtils} -import org.apache.zookeeper.Watcher.Event.KeeperState -import org.I0Itec.zkclient.exception.ZkInterruptedException -import org.I0Itec.zkclient.{IZkChildListener, IZkStateListener} +import kafka.utils.{Logging, ShutdownableThread} +import kafka.zk.KafkaZkClient +import kafka.zookeeper.{StateChangeHandler, ZNodeChildChangeHandler} import org.apache.kafka.common.utils.Time /** @@ -38,60 +38,53 @@ trait NotificationHandler { * notificationHandler's processNotification() method with the child's data as argument. As part of processing these changes it also * purges any children with currentTime - createTime > changeExpirationMs. * - * The caller/user of this class should ensure that they use zkUtils.subscribeStateChanges and call processAllNotifications - * method of this class from ZkStateChangeListener's handleNewSession() method. This is necessary to ensure that if zk session - * is terminated and reestablished any missed notification will be processed immediately. - * @param zkUtils + * @param zkClient * @param seqNodeRoot * @param seqNodePrefix * @param notificationHandler * @param changeExpirationMs * @param time */ -class ZkNodeChangeNotificationListener(private val zkUtils: ZkUtils, +class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, private val seqNodeRoot: String, private val seqNodePrefix: String, private val notificationHandler: NotificationHandler, private val changeExpirationMs: Long = 15 * 60 * 1000, private val time: Time = Time.SYSTEM) extends Logging { private var lastExecutedChange = -1L + private val queue = new LinkedBlockingQueue[ChangeNotification] + private val thread = new ChangeEventProcessThread("change-event-process-thread") private val isClosed = new AtomicBoolean(false) - /** - * create seqNodeRoot and begin watching for any new children nodes. - */ def init() { - zkUtils.makeSurePersistentPathExists(seqNodeRoot) - zkUtils.subscribeChildChanges(seqNodeRoot, NodeChangeListener) - zkUtils.subscribeStateChanges(ZkStateChangeListener) - processAllNotifications() + zkClient.registerStatusChangeHandler(ZkStateChangeListener) + zkClient.registerZNodeChildChangeHandler(ChangeNotificationHandler) + addChangeNotification() + thread.start() } def close() = { isClosed.set(true) + zkClient.unregisterStateChangeHandler(ZkStateChangeListener.name) + zkClient.unregisterZNodeChildChangeHandler(ChangeNotificationHandler.path) + queue.clear() + thread.shutdown() } /** - * Process all changes - */ - def processAllNotifications() { - val changes = zkUtils.getChildren(seqNodeRoot) - processNotifications(changes.sorted) - } - - /** - * Process the given list of notifications + * Process notifications */ - private def processNotifications(notifications: Seq[String]) { - if (notifications.nonEmpty) { - info(s"Processing notification(s) to $seqNodeRoot") - try { + private def processNotifications() { + try { + val notifications = zkClient.getChildren(seqNodeRoot).sorted + if (notifications.nonEmpty) { + info(s"Processing notification(s) to $seqNodeRoot") val now = time.milliseconds for (notification <- notifications) { val changeId = changeNumber(notification) if (changeId > lastExecutedChange) { val changeZnode = seqNodeRoot + "/" + notification - val data = zkUtils.readDataMaybeNull(changeZnode)._1.orNull + val data = zkClient.getDataAndStat(changeZnode)._1.orNull if (data != null) { notificationHandler.processNotification(data) } else { @@ -101,14 +94,22 @@ class ZkNodeChangeNotificationListener(private val zkUtils: ZkUtils, } } purgeObsoleteNotifications(now, notifications) - } catch { - case e: ZkInterruptedException => - if (!isClosed.get) - throw e } + } catch { + case e: InterruptedException => if (!isClosed.get) error(s"Error while processing notification change for path = $seqNodeRoot", e) + case e: Exception => error(s"Error while processing notification change for path = $seqNodeRoot", e) } } + private def addChangeNotification(): Unit = { + if (queue.peek() == null) + queue.put(new ChangeNotification) + } + + class ChangeNotification { + def process(): Unit = processNotifications + } + /** * Purges expired notifications. * @@ -118,11 +119,11 @@ class ZkNodeChangeNotificationListener(private val zkUtils: ZkUtils, private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { for (notification <- notifications.sorted) { val notificationNode = seqNodeRoot + "/" + notification - val (data, stat) = zkUtils.readDataMaybeNull(notificationNode) + val (data, stat) = zkClient.getDataAndStat(notificationNode) if (data.isDefined) { if (now - stat.getCtime > changeExpirationMs) { debug(s"Purging change notification $notificationNode") - zkUtils.deletePath(notificationNode) + zkClient.deletePath(notificationNode) } } } @@ -131,35 +132,18 @@ class ZkNodeChangeNotificationListener(private val zkUtils: ZkUtils, /* get the change number from a change notification znode */ private def changeNumber(name: String): Long = name.substring(seqNodePrefix.length).toLong - /** - * A listener that gets invoked when a node is created to notify changes. - */ - object NodeChangeListener extends IZkChildListener { - override def handleChildChange(path: String, notifications: java.util.List[String]) { - try { - import scala.collection.JavaConverters._ - if (notifications != null) - processNotifications(notifications.asScala.sorted) - } catch { - case e: Exception => error(s"Error processing notification change for path = $path and notification= $notifications :", e) - } - } + class ChangeEventProcessThread(name: String) extends ShutdownableThread(name = name) { + override def doWork(): Unit = queue.take().process } - object ZkStateChangeListener extends IZkStateListener { - - override def handleNewSession() { - processAllNotifications - } - - override def handleSessionEstablishmentError(error: Throwable) { - fatal("Could not establish session with ZooKeeper", error) - } - - override def handleStateChanged(state: KeeperState) { - debug(s"New ZooKeeper state: ${state}") - } + object ChangeNotificationHandler extends ZNodeChildChangeHandler { + override val path: String = seqNodeRoot + override def handleChildChange(): Unit = addChangeNotification } + object ZkStateChangeListener extends StateChangeHandler { + override val name: String = seqNodeRoot + override def afterInitializingSession(): Unit = addChangeNotification + } } diff --git a/core/src/main/scala/kafka/security/auth/Acl.scala b/core/src/main/scala/kafka/security/auth/Acl.scala index 1fbcfb14f5e9c..4e2cba4032f4e 100644 --- a/core/src/main/scala/kafka/security/auth/Acl.scala +++ b/core/src/main/scala/kafka/security/auth/Acl.scala @@ -19,6 +19,7 @@ package kafka.security.auth import kafka.utils.Json import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.apache.kafka.common.utils.SecurityUtils object Acl { val WildCardPrincipal: KafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*") @@ -34,7 +35,7 @@ object Acl { /** * - * @param aclJson + * @param bytes of acls json string * *

{ @@ -52,15 +53,15 @@ object Acl { * * @return */ - def fromJson(aclJson: String): Set[Acl] = { - if (aclJson == null || aclJson.isEmpty) + def fromBytes(bytes: Array[Byte]): Set[Acl] = { + if (bytes == null || bytes.isEmpty) return collection.immutable.Set.empty[Acl] - Json.parseFull(aclJson).map(_.asJsonObject).map { js => + Json.parseBytes(bytes).map(_.asJsonObject).map { js => //the acl json version. require(js(VersionKey).to[Int] == CurrentVersion) js(AclsKey).asJsonArray.iterator.map(_.asJsonObject).map { itemJs => - val principal = KafkaPrincipal.fromString(itemJs(PrincipalKey).to[String]) + val principal = SecurityUtils.parseKafkaPrincipal(itemJs(PrincipalKey).to[String]) val permissionType = PermissionType.fromString(itemJs(PermissionTypeKey).to[String]) val host = itemJs(HostsKey).to[String] val operation = Operation.fromString(itemJs(OperationKey).to[String]) diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index 3c9496485b9c6..a269d75b12ef1 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -18,18 +18,21 @@ package kafka.security.auth import java.util import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} +import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} import kafka.network.RequestChannel.Session import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls import kafka.server.KafkaConfig import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ -import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} +import kafka.zk.{AclChangeNotificationSequenceZNode, AclChangeNotificationZNode, AclZNode, KafkaZkClient} +import kafka.zookeeper.{StateChangeHandler, ZooKeeperClient} import org.apache.kafka.common.security.auth.KafkaPrincipal -import scala.collection.JavaConverters._ +import org.apache.kafka.common.utils.SecurityUtils import org.apache.log4j.Logger +import org.apache.zookeeper.KeeperException.Code +import scala.collection.JavaConverters._ import scala.util.Random object SimpleAclAuthorizer { @@ -44,33 +47,14 @@ object SimpleAclAuthorizer { //If set to true when no acls are found for a resource , authorizer allows access to everyone. Defaults to false. val AllowEveryoneIfNoAclIsFoundProp = "allow.everyone.if.no.acl.found" - /** - * The root acl storage node. Under this node there will be one child node per resource type (Topic, Cluster, Group). - * under each resourceType there will be a unique child for each resource instance and the data for that child will contain - * list of its acls as a json object. Following gives an example: - * - *

-   * /kafka-acl/Topic/topic-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
-   * /kafka-acl/Cluster/kafka-cluster => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
-   * /kafka-acl/Group/group-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
-   * 
- */ - val AclZkPath = ZkUtils.KafkaAclPath - - //notification node which gets updated with the resource name when acl on a resource is changed. - val AclChangedZkPath = ZkUtils.KafkaAclChangesPath - - //prefix of all the change notification sequence node. - val AclChangedPrefix = "acl_changes_" - - private case class VersionedAcls(acls: Set[Acl], zkVersion: Int) + case class VersionedAcls(acls: Set[Acl], zkVersion: Int) } class SimpleAclAuthorizer extends Authorizer with Logging { private val authorizerLogger = Logger.getLogger("kafka.authorizer.logger") private var superUsers = Set.empty[KafkaPrincipal] private var shouldAllowEveryoneIfNoAclIsFound = false - private var zkUtils: ZkUtils = null + private var zkClient: KafkaZkClient = null private var aclChangeListener: ZkNodeChangeNotificationListener = null private val aclCache = new scala.collection.mutable.HashMap[Resource, VersionedAcls] @@ -92,7 +76,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { configs.foreach { case (key, value) => props.put(key, value.toString) } superUsers = configs.get(SimpleAclAuthorizer.SuperUsersProp).collect { - case str: String if str.nonEmpty => str.split(";").map(s => KafkaPrincipal.fromString(s.trim)).toSet + case str: String if str.nonEmpty => str.split(";").map(s => SecurityUtils.parseKafkaPrincipal(s.trim)).toSet }.getOrElse(Set.empty[KafkaPrincipal]) shouldAllowEveryoneIfNoAclIsFound = configs.get(SimpleAclAuthorizer.AllowEveryoneIfNoAclIsFoundProp).exists(_.toString.toBoolean) @@ -105,16 +89,20 @@ class SimpleAclAuthorizer extends Authorizer with Logging { val zkConnectionTimeoutMs = configs.get(SimpleAclAuthorizer.ZkConnectionTimeOutProp).map(_.toString.toInt).getOrElse(kafkaConfig.zkConnectionTimeoutMs) val zkSessionTimeOutMs = configs.get(SimpleAclAuthorizer.ZkSessionTimeOutProp).map(_.toString.toInt).getOrElse(kafkaConfig.zkSessionTimeoutMs) - zkUtils = ZkUtils(zkUrl, - sessionTimeout = zkSessionTimeOutMs, - connectionTimeout = zkConnectionTimeoutMs, - kafkaConfig.zkEnableSecureAcls) - zkUtils.makeSurePersistentPathExists(SimpleAclAuthorizer.AclZkPath) + val zooKeeperClient = new ZooKeeperClient(zkUrl, zkSessionTimeOutMs, + zkConnectionTimeoutMs, new StateChangeHandler { + override val name: String = SimpleAclAuthorizer.getClass.getName + override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") + override def afterInitializingSession(): Unit = debug("Initialised Session.") + override def onAuthFailure(): Unit = error("Auth failure.") + }) + + zkClient = new KafkaZkClient(zooKeeperClient, kafkaConfig.zkEnableSecureAcls) + zkClient.createAclPaths() loadCache() - zkUtils.makeSurePersistentPathExists(SimpleAclAuthorizer.AclChangedZkPath) - aclChangeListener = new ZkNodeChangeNotificationListener(zkUtils, SimpleAclAuthorizer.AclChangedZkPath, SimpleAclAuthorizer.AclChangedPrefix, AclChangedNotificationHandler) + aclChangeListener = new ZkNodeChangeNotificationListener(zkClient, AclChangeNotificationZNode.path, AclChangeNotificationSequenceZNode.SequenceNumberPrefix, AclChangedNotificationHandler) aclChangeListener.init() } @@ -192,7 +180,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { override def removeAcls(resource: Resource): Boolean = { inWriteLock(lock) { - val result = zkUtils.deletePath(toResourcePath(resource)) + val result = zkClient.deleteResource(resource) updateCache(resource, VersionedAcls(Set(), 0)) updateAclChangedFlag(resource) result @@ -223,18 +211,17 @@ class SimpleAclAuthorizer extends Authorizer with Logging { def close() { if (aclChangeListener != null) aclChangeListener.close() - if (zkUtils != null) zkUtils.close() + if (zkClient != null ) zkClient.close() } private def loadCache() { inWriteLock(lock) { - val resourceTypes = zkUtils.getChildren(SimpleAclAuthorizer.AclZkPath) + val resourceTypes = zkClient.getResourceTypes() for (rType <- resourceTypes) { val resourceType = ResourceType.fromString(rType) - val resourceTypePath = SimpleAclAuthorizer.AclZkPath + "/" + resourceType.name - val resourceNames = zkUtils.getChildren(resourceTypePath) + val resourceNames = zkClient.getResourceNames(resourceType.name) for (resourceName <- resourceNames) { - val versionedAcls = getAclsFromZk(Resource(resourceType, resourceName.toString)) + val versionedAcls = getAclsFromZk(Resource(resourceType, resourceName)) updateCache(new Resource(resourceType, resourceName), versionedAcls) } } @@ -242,7 +229,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } def toResourcePath(resource: Resource): String = { - SimpleAclAuthorizer.AclZkPath + "/" + resource.resourceType + "/" + resource.name + AclZNode.path + "/" + resource.resourceType + "/" + resource.name } private def logAuditMessage(principal: KafkaPrincipal, authorized: Boolean, operation: Operation, resource: Resource, host: String) { @@ -266,8 +253,6 @@ class SimpleAclAuthorizer extends Authorizer with Logging { * @return boolean indicating if a change was made */ private def updateResourceAcls(resource: Resource)(getNewAcls: Set[Acl] => Set[Acl]): Boolean = { - val path = toResourcePath(resource) - var currentVersionedAcls = if (aclCache.contains(resource)) getAclsFromCache(resource) @@ -278,13 +263,12 @@ class SimpleAclAuthorizer extends Authorizer with Logging { var retries = 0 while (!writeComplete && retries <= maxUpdateRetries) { val newAcls = getNewAcls(currentVersionedAcls.acls) - val data = Json.encode(Acl.toJsonCompatibleMap(newAcls)) val (updateSucceeded, updateVersion) = if (newAcls.nonEmpty) { - updatePath(path, data, currentVersionedAcls.zkVersion) + conditionalUpdate(resource, newAcls, currentVersionedAcls.zkVersion) } else { trace(s"Deleting path for $resource because it had no ACLs remaining") - (zkUtils.conditionalDeletePath(path, currentVersionedAcls.zkVersion), 0) + (zkClient.conditionalDelete(resource, currentVersionedAcls.zkVersion), 0) } if (!updateSucceeded) { @@ -314,23 +298,35 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } /** - * Updates a zookeeper path with an expected version. If the topic does not exist, it will create it. - * Returns if the update was successful and the new version. - */ - private def updatePath(path: String, data: String, expectedVersion: Int): (Boolean, Int) = { - try { - zkUtils.conditionalUpdatePersistentPathIfExists(path, data, expectedVersion) - } catch { - case _: ZkNoNodeException => - try { - debug(s"Node $path does not exist, attempting to create it.") - zkUtils.createPersistentPath(path, data) - (true, 0) - } catch { - case _: ZkNodeExistsException => - debug(s"Failed to create node for $path because it already exists.") + * Updates a resource zk path with an expected version. If the resource does not exist, it will create it. + * @param resource + * @param acls + * @param expectedVersion + * @return true if the update was successful and the new version + */ + private def conditionalUpdate(resource: Resource, acls: Set[Acl], expectedVersion: Int): (Boolean, Int) = { + val setDataResponse = zkClient.setAclForResource(resource, acls, expectedVersion) + setDataResponse.resultCode match { + case Code.OK => + (true, setDataResponse.stat.getVersion) + case Code.NONODE => { + val createResponse = zkClient.createAclForResource(resource, acls) + createResponse.resultCode match { + case Code.OK => + (true, 0) + case Code.NODEEXISTS => (false, 0) + case _ => + error(s"Error while creating acls at $resource") + throw createResponse.resultException.get } + } + case Code.BADVERSION => + debug(s"Failed to update node for $resource due to bad version.") + (false, 0) + case _ => + debug(s"Error while updating node at $resource") + throw setDataResponse.resultException.get } } @@ -339,8 +335,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } private def getAclsFromZk(resource: Resource): VersionedAcls = { - val (aclJson, stat) = zkUtils.readDataMaybeNull(toResourcePath(resource)) - VersionedAcls(aclJson.map(Acl.fromJson).getOrElse(Set()), stat.getVersion) + zkClient.getVersionedAclsForResource(resource) } private def updateCache(resource: Resource, versionedAcls: VersionedAcls) { @@ -352,7 +347,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } private def updateAclChangedFlag(resource: Resource) { - zkUtils.createSequentialPersistentPath(SimpleAclAuthorizer.AclChangedZkPath + "/" + SimpleAclAuthorizer.AclChangedPrefix, resource.toString) + zkClient.createAclChangeNotification(resource.toString) } private def backoffTime = { @@ -368,4 +363,5 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } } } -} + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index 634b0c2ec392e..3524889f85491 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -26,6 +26,7 @@ import scala.collection._ import scala.collection.JavaConverters._ import kafka.admin.AdminUtils import kafka.utils.json.JsonObject +import kafka.zk.KafkaZkClient import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.security.scram.ScramMechanism import org.apache.kafka.common.utils.Time @@ -83,7 +84,8 @@ object ConfigEntityName { * on startup where a change might be missed between the initial config load and registering for change notifications. * */ -class DynamicConfigManager(private val zkUtils: ZkUtils, +class DynamicConfigManager(private val oldZkUtils: ZkUtils, + private val zkClient: KafkaZkClient, private val configHandlers: Map[String, ConfigHandler], private val changeExpirationMs: Long = 15*60*1000, private val time: Time = Time.SYSTEM) extends Logging { @@ -118,7 +120,7 @@ class DynamicConfigManager(private val zkUtils: ZkUtils, throw new IllegalArgumentException("Version 1 config change notification does not specify 'entity_name'. Received: " + json) } - val entityConfig = AdminUtils.fetchEntityConfig(zkUtils, entityType, entity) + val entityConfig = AdminUtils.fetchEntityConfig(oldZkUtils, entityType, entity) logger.info(s"Processing override for entityType: $entityType, entity: $entity with config: $entityConfig") configHandlers(entityType).processConfigChanges(entity, entityConfig) @@ -139,7 +141,7 @@ class DynamicConfigManager(private val zkUtils: ZkUtils, } val fullSanitizedEntityName = entityPath.substring(index + 1) - val entityConfig = AdminUtils.fetchEntityConfig(zkUtils, rootEntityType, fullSanitizedEntityName) + val entityConfig = AdminUtils.fetchEntityConfig(oldZkUtils, rootEntityType, fullSanitizedEntityName) val loggableConfig = entityConfig.asScala.map { case (k, v) => (k, if (ScramMechanism.isScram(k)) Password.HIDDEN else v) } @@ -149,7 +151,7 @@ class DynamicConfigManager(private val zkUtils: ZkUtils, } } - private val configChangeListener = new ZkNodeChangeNotificationListener(zkUtils, ZkUtils.ConfigChangesPath, + private val configChangeListener = new ZkNodeChangeNotificationListener(zkClient, ZkUtils.ConfigChangesPath, AdminUtils.EntityConfigChangeZnodePrefix, ConfigChangedNotificationHandler) /** @@ -161,14 +163,14 @@ class DynamicConfigManager(private val zkUtils: ZkUtils, // Apply all existing client/user configs to the ClientIdConfigHandler/UserConfigHandler to bootstrap the overrides configHandlers.foreach { case (ConfigType.User, handler) => - AdminUtils.fetchAllEntityConfigs(zkUtils, ConfigType.User).foreach { + AdminUtils.fetchAllEntityConfigs(oldZkUtils, ConfigType.User).foreach { case (sanitizedUser, properties) => handler.processConfigChanges(sanitizedUser, properties) } - AdminUtils.fetchAllChildEntityConfigs(zkUtils, ConfigType.User, ConfigType.Client).foreach { + AdminUtils.fetchAllChildEntityConfigs(oldZkUtils, ConfigType.User, ConfigType.Client).foreach { case (sanitizedUserClientId, properties) => handler.processConfigChanges(sanitizedUserClientId, properties) } case (configType, handler) => - AdminUtils.fetchAllEntityConfigs(zkUtils, configType).foreach { + AdminUtils.fetchAllEntityConfigs(oldZkUtils, configType).foreach { case (entityName, properties) => handler.processConfigChanges(entityName, properties) } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index dff83db50ba5a..6cb90a9181c49 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -223,6 +223,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP val zooKeeperClient = new ZooKeeperClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, config.zkMaxInFlightRequests, new StateChangeHandler { + override val name: String = KafkaServer.getClass.getName + override def onReconnectionTimeout(): Unit = { error("Reconnection timeout.") } @@ -291,7 +293,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP ConfigType.Broker -> new BrokerConfigHandler(config, quotaManagers)) // Create the config manager. start listening to notifications - dynamicConfigManager = new DynamicConfigManager(zkUtils, dynamicConfigHandlers) + dynamicConfigManager = new DynamicConfigManager(zkUtils, zkClient, dynamicConfigHandlers) dynamicConfigManager.startup() /* tell everyone we are alive */ diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 3267a74a9f2e6..e1c67088142fd 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -23,6 +23,10 @@ import kafka.api.LeaderAndIsr import kafka.cluster.Broker import kafka.controller.LeaderIsrAndControllerEpoch import kafka.log.LogConfig +import kafka.security.auth.Acl +import kafka.security.auth.Resource +import kafka.security.auth.ResourceType +import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls import kafka.server.ConfigType import kafka.utils._ import kafka.zookeeper._ @@ -627,6 +631,176 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends retryRequestsUntilConnected(deleteRequests) } + //Acl management methods + + /** + * Creates the required zk nodes for Acl storage + */ + def createAclPaths(): Unit = { + createRecursive(AclZNode.path) + createRecursive(AclChangeNotificationZNode.path) + ResourceType.values.foreach(resource => createRecursive(ResourceTypeZNode.path(resource.name))) + } + + /** + * Gets VersionedAcls for a given Resource + * @param resource Resource to get VersionedAcls for + * @return VersionedAcls + */ + def getVersionedAclsForResource(resource: Resource): VersionedAcls = { + val getDataRequest = GetDataRequest(ResourceZNode.path(resource)) + val getDataResponse = retryRequestUntilConnected(getDataRequest) + if (getDataResponse.resultCode == Code.OK) { + ResourceZNode.decode(getDataResponse.data, getDataResponse.stat) + } else { + VersionedAcls(Set(), -1) + } + } + + /** + * Sets Resource znode with the given acls. + * @param resource resource object + * @param acls the acls to sets on the resource znode + * @param expectedVersion expected zk version + * @return SetDataResponse + */ + def setAclForResource(resource: Resource, acls: Set[Acl], expectedVersion: Int): SetDataResponse = { + val setDataRequest = SetDataRequest(ResourceZNode.path(resource), ResourceZNode.encode(acls), expectedVersion) + retryRequestUntilConnected(setDataRequest) + } + + /** + * Creates Resource znode with the given acls + * @param resource resource object + * @param aclsSet the acls to sets on the resource znode + * @return CreateResponse + */ + def createAclForResource(resource: Resource, aclsSet: Set[Acl]): CreateResponse = { + val path = ResourceZNode.path(resource) + val createRequest = CreateRequest(path, ResourceZNode.encode(aclsSet), acls(path), CreateMode.PERSISTENT) + retryRequestUntilConnected(createRequest) + } + + /** + * Creates Acl change notification message + * @param resource resource name + * @return CreateResponse + */ + def createAclChangeNotification(resource: String): CreateResponse = { + val path = AclChangeNotificationSequenceZNode.path + val createRequest = CreateRequest(path, AclChangeNotificationSequenceZNode.encode(resource), acls(path), CreateMode.PERSISTENT_SEQUENTIAL) + retryRequestUntilConnected(createRequest) + } + + /** + * Gets the resource types + */ + def getResourceTypes(): Seq[String] = { + val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(AclZNode.path)) + if (getChildrenResponse.resultCode == Code.OK) { + getChildrenResponse.children + } else if (getChildrenResponse.resultCode == Code.NONODE) { + Seq.empty + } else { + throw getChildrenResponse.resultException.get + } + } + + /** + * Gets the resource names for a give resource type + * @param resourceType + * @return list of resource names + */ + def getResourceNames(resourceType: String): Seq[String] = { + val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(ResourceTypeZNode.path(resourceType))) + if (getChildrenResponse.resultCode == Code.OK) { + getChildrenResponse.children + } else if (getChildrenResponse.resultCode == Code.NONODE) { + Seq.empty + } else { + throw getChildrenResponse.resultException.get + } + } + + /** + * Deletes the given Resource node + */ + def deleteResource(resource: Resource): Boolean = { + deleteRecursive(ResourceZNode.path(resource)) + true + } + + /** + * Conditional delete the resource node + * @param resource + * @param expectedVersion + * @return return true if it succeeds, false otherwise (the current version is not the expected version) + */ + def conditionalDelete(resource: Resource, expectedVersion: Int): Boolean = { + val deleteRequest = DeleteRequest(ResourceZNode.path(resource), expectedVersion) + val deleteResponse = retryRequestUntilConnected(deleteRequest) + if (deleteResponse.resultCode == Code.OK || deleteResponse.resultCode == Code.NONODE) { + true + } else if (deleteResponse.resultCode == Code.BADVERSION) { + false + } else { + throw deleteResponse.resultException.get + } + } + + /** + * Gets all the child nodes at a given zk node path + * @param path + * @return list of child node names + */ + def getChildren(path : String): Seq[String] = { + val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path)) + if (getChildrenResponse.resultCode == Code.OK) { + getChildrenResponse.children + } else if (getChildrenResponse.resultCode == Code.NONODE) { + Seq.empty + } else { + throw getChildrenResponse.resultException.get + } + } + + /** + * Gets the data and Stat at the given zk path + * @param path zk node path + * @return A tuple of 2 elements, where first element is zk node data as string + * and second element is zk node stats. + * returns (None, -1) if node doesn't exists, data is null or for any error. + */ + def getDataAndStat(path: String): (Option[String], Stat) = { + val getDataRequest = GetDataRequest(path) + val getDataResponse = retryRequestUntilConnected(getDataRequest) + + if (getDataResponse.resultCode == Code.OK) { + if (getDataResponse.data == null) + (None, getDataResponse.stat) + else { + val data = new String(getDataResponse.data, "UTF-8") + (Some(data), getDataResponse.stat) + } + } else + (None, new Stat()) + } + + /** + * Deletes th zk node + * @param path + * @return return true if it succeeds, false otherwise + */ + def deletePath(path: String): Boolean = { + val deleteRequest = DeleteRequest(path, -1) + val deleteResponse = retryRequestUntilConnected(deleteRequest) + if (deleteResponse.resultCode == Code.OK || deleteResponse.resultCode == Code.NONODE) { + true + }else { + throw deleteResponse.resultException.get + } + } + /** * This registers a ZNodeChangeHandler and attempts to register a watcher with an ExistsRequest, which allows data * watcher registrations on paths which might not even exist. @@ -677,6 +851,23 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends zooKeeperClient.unregisterZNodeChildChangeHandler(path) } + /** + * + * @param stateChangeHandler + */ + def registerStatusChangeHandler(stateChangeHandler: StateChangeHandler): Unit = { + zooKeeperClient.registerStateChangeHandler(stateChangeHandler) + } + + /** + * + * @param path + */ + def unregisterStateChangeHandler(path: String): Unit = { + zooKeeperClient.unregisterStateChangeHandler(path) + } + + /** * Close the underlying ZooKeeperClient. */ diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index a1ff5594e48aa..6ca333e73d92f 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -22,6 +22,11 @@ import java.util.Properties import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch} +import kafka.common.TopicAndPartition +import kafka.controller.{IsrChangeNotificationListener, LeaderIsrAndControllerEpoch} +import kafka.security.auth.Acl +import kafka.security.auth.Resource +import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls import kafka.utils.Json import org.apache.kafka.common.TopicPartition import org.apache.zookeeper.data.Stat @@ -252,3 +257,39 @@ object ConsumerOffset { object ZkVersion { val NoVersion = -1 } + +/** + * The root acl storage node. Under this node there will be one child node per resource type (Topic, Cluster, Group). + * under each resourceType there will be a unique child for each resource instance and the data for that child will contain + * list of its acls as a json object. Following gives an example: + * + *
+ * /kafka-acl/Topic/topic-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
+ * /kafka-acl/Cluster/kafka-cluster => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
+ * /kafka-acl/Group/group-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
+ * 
+ */ +object AclZNode { + def path = "/kafka-acl" +} + +object ResourceTypeZNode { + def path(resourceType: String) = s"${AclZNode.path}/$resourceType" +} + +object ResourceZNode { + def path(resource: Resource) = s"${AclZNode.path}/${resource.resourceType}/${resource.name}" + def encode(acls: Set[Acl]): Array[Byte] = Json.encodeAsBytes(Acl.toJsonCompatibleMap(acls)) + def decode(bytes: Array[Byte], stat: Stat): VersionedAcls = VersionedAcls(Acl.fromBytes(bytes), stat.getVersion) +} + +object AclChangeNotificationZNode { + def path = "/kafka-acl-changes" +} + +object AclChangeNotificationSequenceZNode { + val SequenceNumberPrefix = "acl_changes_" + def path = s"${AclChangeNotificationZNode.path}/$SequenceNumberPrefix" + def encode(resourceName : String): Array[Byte] = resourceName.getBytes(UTF_8) + def decode(bytes: Array[Byte]): String = new String(bytes, UTF_8) +} diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala index 149e7eb005174..6550c3d7efe68 100644 --- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala +++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala @@ -52,6 +52,8 @@ class ZooKeeperClient(connectString: String, private val zNodeChangeHandlers = new ConcurrentHashMap[String, ZNodeChangeHandler]().asScala private val zNodeChildChangeHandlers = new ConcurrentHashMap[String, ZNodeChildChangeHandler]().asScala private val inFlightRequests = new Semaphore(maxInFlightRequests) + private val stateChangeHandlers = new ConcurrentHashMap[String, StateChangeHandler]().asScala + registerStateChangeHandler(stateChangeHandler) info(s"Initializing a new session to $connectString.") @volatile private var zooKeeper = new ZooKeeper(connectString, sessionTimeoutMs, ZooKeeperClientWatcher) @@ -233,6 +235,15 @@ class ZooKeeperClient(connectString: String, zNodeChildChangeHandlers.remove(path) } + def registerStateChangeHandler(statusChangeHandler: StateChangeHandler): Unit = { + if (statusChangeHandler != null) + stateChangeHandlers.put(statusChangeHandler.name, statusChangeHandler) + } + + def unregisterStateChangeHandler(path: String): Unit = { + stateChangeHandlers.remove(path) + } + def close(): Unit = inWriteLock(initializationLock) { info("Closing.") zNodeChangeHandlers.clear() @@ -266,7 +277,7 @@ class ZooKeeperClient(connectString: String, } } info(s"Timed out waiting for connection during session initialization while in state: ${zooKeeper.getState}") - stateChangeHandler.onReconnectionTimeout() + stateChangeHandlers.foreach(_._2.onReconnectionTimeout()) } } @@ -280,13 +291,13 @@ class ZooKeeperClient(connectString: String, } if (event.getState == KeeperState.AuthFailed) { info("Auth failed.") - stateChangeHandler.onAuthFailure() + stateChangeHandlers.foreach(_._2.onAuthFailure()) } else if (event.getState == KeeperState.Expired) { inWriteLock(initializationLock) { info("Session expired.") - stateChangeHandler.beforeInitializingSession() + stateChangeHandlers.foreach(_._2.beforeInitializingSession()) initialize() - stateChangeHandler.afterInitializingSession() + stateChangeHandlers.foreach(_._2.afterInitializingSession()) } } case Some(path) => @@ -302,6 +313,7 @@ class ZooKeeperClient(connectString: String, } trait StateChangeHandler { + val name: String def beforeInitializingSession(): Unit = {} def afterInitializingSession(): Unit = {} def onAuthFailure(): Unit = {} diff --git a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala index 7fc2436706f9f..99550d5a60821 100644 --- a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala +++ b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala @@ -16,14 +16,11 @@ */ package kafka.common -import kafka.integration.KafkaServerTestHarness -import kafka.server.KafkaConfig import kafka.utils.TestUtils +import kafka.zk.{AclChangeNotificationSequenceZNode, AclChangeNotificationZNode, ZooKeeperTestHarness} import org.junit.Test -class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness { - - override def generateConfigs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) +class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { @Test def testProcessNotification() { @@ -36,18 +33,16 @@ class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness { } } - val seqNodeRoot = "/root" - val seqNodePrefix = "prefix" - val seqNodePath = seqNodeRoot + "/" + seqNodePrefix + zkClient.createAclPaths() val notificationMessage1 = "message1" val notificationMessage2 = "message2" val changeExpirationMs = 1000 - val notificationListener = new ZkNodeChangeNotificationListener(zkUtils, seqNodeRoot, seqNodePrefix, notificationHandler, changeExpirationMs) + val notificationListener = new ZkNodeChangeNotificationListener(zkClient, AclChangeNotificationZNode.path, + AclChangeNotificationSequenceZNode.SequenceNumberPrefix, notificationHandler, changeExpirationMs) notificationListener.init() - zkUtils.createSequentialPersistentPath(seqNodePath, notificationMessage1) - + zkClient.createAclChangeNotification(notificationMessage1) TestUtils.waitUntilTrue(() => invocationCount == 1 && notification == notificationMessage1, "Failed to send/process notification message in the timeout period.") @@ -59,11 +54,11 @@ class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness { * can fail as the second node can be deleted depending on how threads get scheduled. */ - zkUtils.createSequentialPersistentPath(seqNodePath, notificationMessage2) + zkClient.createAclChangeNotification(notificationMessage2) TestUtils.waitUntilTrue(() => invocationCount == 2 && notification == notificationMessage2, "Failed to send/process notification message in the timeout period.") - (3 to 10).foreach(i => zkUtils.createSequentialPersistentPath(seqNodePath, "message" + i)) + (3 to 10).foreach(i => zkClient.createAclChangeNotification("message" + i)) TestUtils.waitUntilTrue(() => invocationCount == 10 , s"Expected 10 invocations of processNotifications, but there were $invocationCount") diff --git a/core/src/test/scala/unit/kafka/security/auth/AclTest.scala b/core/src/test/scala/unit/kafka/security/auth/AclTest.scala index dd33dc47f349c..dfdd85face30b 100644 --- a/core/src/test/scala/unit/kafka/security/auth/AclTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/AclTest.scala @@ -16,9 +16,11 @@ */ package kafka.security.auth +import java.nio.charset.StandardCharsets.UTF_8 + import kafka.utils.Json import org.apache.kafka.common.security.auth.KafkaPrincipal -import org.junit.{Test, Assert} +import org.junit.{Assert, Test} import org.scalatest.junit.JUnitSuite class AclTest extends JUnitSuite { @@ -36,8 +38,8 @@ class AclTest extends JUnitSuite { val acls = Set[Acl](acl1, acl2, acl3) val jsonAcls = Json.encode(Acl.toJsonCompatibleMap(acls)) - Assert.assertEquals(acls, Acl.fromJson(jsonAcls)) - Assert.assertEquals(acls, Acl.fromJson(AclJson)) + Assert.assertEquals(acls, Acl.fromBytes(jsonAcls.getBytes(UTF_8))) + Assert.assertEquals(acls, Acl.fromBytes(AclJson.getBytes(UTF_8))) } } diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala index 22a06e7514265..b647493e7e0ec 100644 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -23,7 +23,7 @@ import kafka.network.RequestChannel.Session import kafka.security.auth.Acl.WildCardHost import kafka.server.KafkaConfig import kafka.utils.TestUtils -import kafka.zk.ZooKeeperTestHarness +import kafka.zk.{AclChangeNotificationZNode, ZooKeeperTestHarness} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.junit.Assert._ import org.junit.{After, Before, Test} @@ -44,6 +44,8 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { override def setUp() { super.setUp() + zkClient.createAclPaths + // Increase maxUpdateRetries to avoid transient failures simpleAclAuthorizer.maxUpdateRetries = Int.MaxValue simpleAclAuthorizer2.maxUpdateRetries = Int.MaxValue @@ -258,7 +260,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { val acls1 = Set[Acl](acl2) simpleAclAuthorizer.addAcls(acls1, resource1) - zkUtils.deletePathRecursive(SimpleAclAuthorizer.AclChangedZkPath) + zkUtils.deletePathRecursive(AclChangeNotificationZNode.path) val authorizer = new SimpleAclAuthorizer try { authorizer.configure(config.originals) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 9d2bb8b7d21b8..dd7bb5ff45159 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -187,7 +187,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { EasyMock.expectLastCall().once() EasyMock.replay(handler) - val configManager = new DynamicConfigManager(zkUtils, Map(ConfigType.Topic -> handler)) + val configManager = new DynamicConfigManager(zkUtils, zkClient, Map(ConfigType.Topic -> handler)) // Notifications created using the old TopicConfigManager are ignored. configManager.ConfigChangedNotificationHandler.processNotification("not json") diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index 775c68e8c33fa..7614347053782 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -18,31 +18,16 @@ package kafka.zk import kafka.server.Defaults import kafka.zookeeper.ZooKeeperClient +import kafka.common.TopicAndPartition import org.apache.kafka.common.TopicPartition import org.junit.Assert.{assertEquals, assertFalse, assertTrue} -import org.junit.{After, Before, Test} +import org.junit.Test class KafkaZkClientTest extends ZooKeeperTestHarness { - private var zooKeeperClient: ZooKeeperClient = null - private var zkClient: KafkaZkClient = null - private val group = "my-group" private val topicPartition = new TopicPartition("topic", 0) - @Before - override def setUp() { - super.setUp() - zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Defaults.ZkMaxInFlightRequests, null) - zkClient = new KafkaZkClient(zooKeeperClient, false) - } - - @After - override def tearDown() { - zkClient.close() - super.tearDown() - } - @Test def testSetAndGetConsumerOffset() { val offset = 123L diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 85a559642fba1..1e6fdd411afc0 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -32,6 +32,7 @@ import scala.collection.JavaConverters._ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.clients.consumer.internals.AbstractCoordinator import kafka.controller.ControllerEventManager +import kafka.zookeeper.{StateChangeHandler, ZooKeeperClient} @Category(Array(classOf[IntegrationTest])) abstract class ZooKeeperTestHarness extends JUnitSuite with Logging { @@ -41,6 +42,7 @@ abstract class ZooKeeperTestHarness extends JUnitSuite with Logging { protected val zkAclsEnabled: Option[Boolean] = None var zkUtils: ZkUtils = null + var zkClient: KafkaZkClient = null var zookeeper: EmbeddedZookeeper = null def zkPort: Int = zookeeper.port @@ -50,12 +52,21 @@ abstract class ZooKeeperTestHarness extends JUnitSuite with Logging { def setUp() { zookeeper = new EmbeddedZookeeper() zkUtils = ZkUtils(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled())) + + val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, + zkConnectionTimeout, new StateChangeHandler { + override val name: String = this.getClass.getName + }) + + zkClient = new KafkaZkClient(zooKeeperClient, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled())) } @After def tearDown() { if (zkUtils != null) CoreUtils.swallow(zkUtils.close()) + if (zkClient != null) + zkClient.close() if (zookeeper != null) CoreUtils.swallow(zookeeper.shutdown()) Configuration.setConfiguration(null) diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index 50a065f0d64a9..d923ea6561ac4 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -328,6 +328,8 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, "no-such-file-exists.conf") val stateChangeHandlerCountDownLatch = new CountDownLatch(1) val stateChangeHandler = new StateChangeHandler { + override val name: String = this.getClass.getName + override def onAuthFailure(): Unit = { stateChangeHandlerCountDownLatch.countDown() } From 4bc0daa95e3e79e3ad419806fb65a975938f998f Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 31 Oct 2017 17:23:34 +0530 Subject: [PATCH 2/8] Address Review comments --- .../ZkNodeChangeNotificationListener.scala | 39 ++-- .../security/auth/SimpleAclAuthorizer.scala | 22 +- .../kafka/server/DynamicConfigManager.scala | 9 +- .../main/scala/kafka/zk/KafkaZkClient.scala | 205 +++++++++++++----- core/src/main/scala/kafka/zk/ZkData.scala | 19 +- .../kafka/zookeeper/ZooKeeperClient.scala | 23 +- ...ZkNodeChangeNotificationListenerTest.scala | 15 +- .../auth/SimpleAclAuthorizerTest.scala | 8 +- .../unit/kafka/zk/KafkaZkClientTest.scala | 118 ++++++++++ .../unit/kafka/zk/ZooKeeperTestHarness.scala | 7 +- .../kafka/zookeeper/ZooKeeperClientTest.scala | 20 -- 11 files changed, 351 insertions(+), 134 deletions(-) diff --git a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala index 11cd449369793..555e4d6953496 100644 --- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala @@ -23,6 +23,7 @@ import kafka.utils.{Logging, ShutdownableThread} import kafka.zk.KafkaZkClient import kafka.zookeeper.{StateChangeHandler, ZNodeChildChangeHandler} import org.apache.kafka.common.utils.Time +import org.apache.zookeeper.data.Stat /** * Handle the notificationMessage. @@ -40,24 +41,22 @@ trait NotificationHandler { * * @param zkClient * @param seqNodeRoot - * @param seqNodePrefix * @param notificationHandler * @param changeExpirationMs * @param time */ -class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, +abstract class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, private val seqNodeRoot: String, - private val seqNodePrefix: String, private val notificationHandler: NotificationHandler, private val changeExpirationMs: Long = 15 * 60 * 1000, private val time: Time = Time.SYSTEM) extends Logging { private var lastExecutedChange = -1L private val queue = new LinkedBlockingQueue[ChangeNotification] - private val thread = new ChangeEventProcessThread("change-event-process-thread") + private val thread = new ChangeEventProcessThread(s"$seqNodeRoot-event-process-thread") private val isClosed = new AtomicBoolean(false) def init() { - zkClient.registerStatusChangeHandler(ZkStateChangeListener) + zkClient.registerStateChangeHandler(ZkStateChangeListener) zkClient.registerZNodeChildChangeHandler(ChangeNotificationHandler) addChangeNotification() thread.start() @@ -76,21 +75,19 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, */ private def processNotifications() { try { - val notifications = zkClient.getChildren(seqNodeRoot).sorted + val notifications = getAllChangeNotifications.sorted if (notifications.nonEmpty) { info(s"Processing notification(s) to $seqNodeRoot") val now = time.milliseconds for (notification <- notifications) { - val changeId = changeNumber(notification) - if (changeId > lastExecutedChange) { - val changeZnode = seqNodeRoot + "/" + notification - val data = zkClient.getDataAndStat(changeZnode)._1.orNull + if (notification.toLong > lastExecutedChange) { + val data = getDataFromChangeNotification(notification)._1.orNull if (data != null) { notificationHandler.processNotification(data) } else { - logger.warn(s"read null data from $changeZnode when processing notification $notification") + logger.warn(s"read null data from $notification sequence node when processing notification for path = $seqNodeRoot") } - lastExecutedChange = changeId + lastExecutedChange = notification.toLong } } purgeObsoleteNotifications(now, notifications) @@ -101,8 +98,14 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, } } + def getAllChangeNotifications() : Seq[String] + + def deleteChangeNotification(sequenceNumber: String) : Boolean + + def getDataFromChangeNotification(sequenceNumber: String) : (Option[String], Stat) + private def addChangeNotification(): Unit = { - if (queue.peek() == null) + if (!isClosed.get && queue.peek() == null) queue.put(new ChangeNotification) } @@ -118,20 +121,16 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, */ private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { for (notification <- notifications.sorted) { - val notificationNode = seqNodeRoot + "/" + notification - val (data, stat) = zkClient.getDataAndStat(notificationNode) + val (data, stat) = getDataFromChangeNotification(notification) if (data.isDefined) { if (now - stat.getCtime > changeExpirationMs) { - debug(s"Purging change notification $notificationNode") - zkClient.deletePath(notificationNode) + debug(s"Purging change notification $notification for path = $seqNodeRoot") + deleteChangeNotification(notification) } } } } - /* get the change number from a change notification znode */ - private def changeNumber(name: String): Long = name.substring(seqNodePrefix.length).toLong - class ChangeEventProcessThread(name: String) extends ShutdownableThread(name = name) { override def doWork(): Unit = queue.take().process } diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index a269d75b12ef1..e3559a8860098 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -25,12 +25,13 @@ import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls import kafka.server.KafkaConfig import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ -import kafka.zk.{AclChangeNotificationSequenceZNode, AclChangeNotificationZNode, AclZNode, KafkaZkClient} +import kafka.zk.{AclChangeNotificationZNode, AclZNode, KafkaZkClient} import kafka.zookeeper.{StateChangeHandler, ZooKeeperClient} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.SecurityUtils import org.apache.log4j.Logger import org.apache.zookeeper.KeeperException.Code +import org.apache.zookeeper.data.Stat import scala.collection.JavaConverters._ import scala.util.Random @@ -102,7 +103,13 @@ class SimpleAclAuthorizer extends Authorizer with Logging { loadCache() - aclChangeListener = new ZkNodeChangeNotificationListener(zkClient, AclChangeNotificationZNode.path, AclChangeNotificationSequenceZNode.SequenceNumberPrefix, AclChangedNotificationHandler) + aclChangeListener = new ZkNodeChangeNotificationListener(zkClient, AclChangeNotificationZNode.path, AclChangedNotificationHandler) { + override def getAllChangeNotifications(): Seq[String] = zkClient.getAclChangeNotifications + override def deleteChangeNotification(sequenceNumber: String): Boolean = zkClient.deleteAclChangeNotification(sequenceNumber) + override def getDataFromChangeNotification(sequenceNumber: String): (Option[String], Stat) = + zkClient.getDataFromAclChangeNotification(sequenceNumber) + } + aclChangeListener.init() } @@ -228,10 +235,6 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } } - def toResourcePath(resource: Resource): String = { - AclZNode.path + "/" + resource.resourceType + "/" + resource.name - } - private def logAuditMessage(principal: KafkaPrincipal, authorized: Boolean, operation: Operation, resource: Resource, host: String) { def logMessage: String = { val authResult = if (authorized) "Allowed" else "Denied" @@ -305,12 +308,12 @@ class SimpleAclAuthorizer extends Authorizer with Logging { * @return true if the update was successful and the new version */ private def conditionalUpdate(resource: Resource, acls: Set[Acl], expectedVersion: Int): (Boolean, Int) = { - val setDataResponse = zkClient.setAclForResource(resource, acls, expectedVersion) + val setDataResponse = zkClient.setAclForResourceRaw(resource, acls, expectedVersion) setDataResponse.resultCode match { case Code.OK => (true, setDataResponse.stat.getVersion) case Code.NONODE => { - val createResponse = zkClient.createAclForResource(resource, acls) + val createResponse = zkClient.createAclForResourceRaw(resource, acls) createResponse.resultCode match { case Code.OK => (true, 0) @@ -347,7 +350,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } private def updateAclChangedFlag(resource: Resource) { - zkClient.createAclChangeNotification(resource.toString) + zkClient.createAclChangeNotificationRaw(resource.toString) } private def backoffTime = { @@ -363,5 +366,4 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } } } - } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index 3524889f85491..2320a3efab848 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -30,6 +30,7 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.security.scram.ScramMechanism import org.apache.kafka.common.utils.Time +import org.apache.zookeeper.data.Stat /** * Represents all the entities that can be configured via ZK @@ -151,8 +152,12 @@ class DynamicConfigManager(private val oldZkUtils: ZkUtils, } } - private val configChangeListener = new ZkNodeChangeNotificationListener(zkClient, ZkUtils.ConfigChangesPath, - AdminUtils.EntityConfigChangeZnodePrefix, ConfigChangedNotificationHandler) + private val configChangeListener = new ZkNodeChangeNotificationListener(zkClient, ZkUtils.ConfigChangesPath, ConfigChangedNotificationHandler) { + override def getAllChangeNotifications(): Seq[String] = zkClient.getConfigChangeNotifications() + override def deleteChangeNotification(sequenceNumber: String): Boolean = zkClient.deleteConfigChangeNotification(sequenceNumber) + override def getDataFromChangeNotification(sequenceNumber: String): (Option[String], Stat) = + zkClient.getDataFromConfigChangeNotification(sequenceNumber) + } /** * Begin watching for config changes diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index e1c67088142fd..e67e494b65419 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -19,13 +19,13 @@ package kafka.zk import java.nio.charset.StandardCharsets.UTF_8 import java.util.Properties +import java.nio.charset.StandardCharsets.UTF_8 + import kafka.api.LeaderAndIsr import kafka.cluster.Broker import kafka.controller.LeaderIsrAndControllerEpoch import kafka.log.LogConfig -import kafka.security.auth.Acl -import kafka.security.auth.Resource -import kafka.security.auth.ResourceType +import kafka.security.auth.{Acl, Resource, ResourceType} import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls import kafka.server.ConfigType import kafka.utils._ @@ -351,6 +351,47 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends throw getDataResponse.resultException.get } + /** + * Gets the data and Stat at the given zk path + * @param path zk node path + * @return A tuple of 2 elements, where first element is zk node data as string + * and second element is zk node stats. + * returns (None, new Stat()) if node doesn't exists and throws exception for any error + */ + private[zk] def getDataAndStat(path: String): (Option[String], Stat) = { + val getDataRequest = GetDataRequest(path) + val getDataResponse = retryRequestUntilConnected(getDataRequest) + + if (getDataResponse.resultCode == Code.OK) { + if (getDataResponse.data == null) + (None, getDataResponse.stat) + else { + val data = new String(getDataResponse.data, UTF_8) + (Some(data), getDataResponse.stat) + } + } else if (getDataResponse.resultCode == Code.NONODE) { + (None, new Stat()) + } else { + throw getDataResponse.resultException.get + } + } + + /** + * Gets all the child nodes at a given zk node path + * @param path + * @return list of child node names + */ + private[zk] def getChildren(path : String): Seq[String] = { + val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path)) + if (getChildrenResponse.resultCode == Code.OK) { + getChildrenResponse.children + } else if (getChildrenResponse.resultCode == Code.NONODE) { + Seq.empty + } else { + throw getChildrenResponse.resultException.get + } + } + /** * Conditional update the persistent path data, return (true, newVersion) if it succeeds, otherwise (the path doesn't * exist, the current version is not the expected version, etc.) return (false, -1) @@ -652,8 +693,10 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends val getDataResponse = retryRequestUntilConnected(getDataRequest) if (getDataResponse.resultCode == Code.OK) { ResourceZNode.decode(getDataResponse.data, getDataResponse.stat) - } else { + } else if (getDataResponse.resultCode == Code.NONODE) { VersionedAcls(Set(), -1) + } else { + throw getDataResponse.resultException.get } } @@ -664,7 +707,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends * @param expectedVersion expected zk version * @return SetDataResponse */ - def setAclForResource(resource: Resource, acls: Set[Acl], expectedVersion: Int): SetDataResponse = { + def setAclForResourceRaw(resource: Resource, acls: Set[Acl], expectedVersion: Int): SetDataResponse = { val setDataRequest = SetDataRequest(ResourceZNode.path(resource), ResourceZNode.encode(acls), expectedVersion) retryRequestUntilConnected(setDataRequest) } @@ -675,7 +718,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends * @param aclsSet the acls to sets on the resource znode * @return CreateResponse */ - def createAclForResource(resource: Resource, aclsSet: Set[Acl]): CreateResponse = { + def createAclForResourceRaw(resource: Resource, aclsSet: Set[Acl]): CreateResponse = { val path = ResourceZNode.path(resource) val createRequest = CreateRequest(path, ResourceZNode.encode(aclsSet), acls(path), CreateMode.PERSISTENT) retryRequestUntilConnected(createRequest) @@ -683,22 +726,25 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends /** * Creates Acl change notification message - * @param resource resource name - * @return CreateResponse + * @param resourceName resource name */ - def createAclChangeNotification(resource: String): CreateResponse = { - val path = AclChangeNotificationSequenceZNode.path - val createRequest = CreateRequest(path, AclChangeNotificationSequenceZNode.encode(resource), acls(path), CreateMode.PERSISTENT_SEQUENTIAL) - retryRequestUntilConnected(createRequest) + def createAclChangeNotificationRaw(resourceName: String): Unit = { + val path = AclChangeNotificationSequenceZNode.createPath + val createRequest = CreateRequest(path, AclChangeNotificationSequenceZNode.encode(resourceName), acls(path), CreateMode.PERSISTENT_SEQUENTIAL) + val createResponse = retryRequestUntilConnected(createRequest) + if (createResponse.resultCode != Code.OK) { + throw createResponse.resultException.get + } } /** - * Gets the resource types + * Gets all Acl change notifications + * @return list of change node sequence numbers */ - def getResourceTypes(): Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(AclZNode.path)) + def getAclChangeNotifications(): Seq[String] = { + val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(AclChangeNotificationZNode.path)) if (getChildrenResponse.resultCode == Code.OK) { - getChildrenResponse.children + getChildrenResponse.children.map(AclChangeNotificationSequenceZNode.sequenceNumber) } else if (getChildrenResponse.resultCode == Code.NONODE) { Seq.empty } else { @@ -706,28 +752,87 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } } + /** + * Get Acl change data and zk node Stat associated with the given sequence number + * @param sequenceNumber + * @return + */ + def getDataFromAclChangeNotification(sequenceNumber: String): (Option[String], Stat) = { + getDataAndStat(AclChangeNotificationSequenceZNode.path(sequenceNumber)) + } + + /** + * Deletes all Acl change notifications. + */ + def deleteAclChangeNotifications(): Unit = { + val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(AclChangeNotificationZNode.path)) + if (getChildrenResponse.resultCode == Code.OK) { + deleteAclChangeNotifications(getChildrenResponse.children) + } else if (getChildrenResponse.resultCode != Code.NONODE) { + throw getChildrenResponse.resultException.get + } + } + + /** + * Deletes the Acl change notifications associated with the given sequence numbers. + * @param sequenceNumbers the sequence numbers associated with the Acl change notifications to be deleted. + */ + def deleteAclChangeNotifications(sequenceNumbers: Seq[String]): Unit = { + val deleteRequests = sequenceNumbers.map { sequenceNumber => + DeleteRequest(AclChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) + } + retryRequestsUntilConnected(deleteRequests) + } + + /** + * Deletes Acl change notification associated with the given sequence number + * @param sequenceNumber + * @return delete status + */ + def deleteAclChangeNotification(sequenceNumber: String): Boolean = { + val deleteRequest = DeleteRequest(AclChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) + val deleteResponse = retryRequestUntilConnected(deleteRequest) + if (deleteResponse.resultCode == Code.OK || deleteResponse.resultCode == Code.NONODE) { + true + }else { + throw deleteResponse.resultException.get + } + } + + /** + * Gets the resource types + * @return list of resource type names + */ + def getResourceTypes(): Seq[String] = { + getChildren(AclZNode.path) + } + /** * Gets the resource names for a give resource type * @param resourceType * @return list of resource names */ def getResourceNames(resourceType: String): Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(ResourceTypeZNode.path(resourceType))) - if (getChildrenResponse.resultCode == Code.OK) { - getChildrenResponse.children - } else if (getChildrenResponse.resultCode == Code.NONODE) { - Seq.empty - } else { - throw getChildrenResponse.resultException.get - } + getChildren(ResourceTypeZNode.path(resourceType)) } /** * Deletes the given Resource node + * @param resource + * @return delete status */ def deleteResource(resource: Resource): Boolean = { deleteRecursive(ResourceZNode.path(resource)) - true + !resourceExists(resource) + } + + /** + * checks the resource existence + * @param resource + * @return existence status + */ + def resourceExists(resource: Resource): Boolean = { + pathExists(ResourceZNode.path(resource)) } /** @@ -749,14 +854,12 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } /** - * Gets all the child nodes at a given zk node path - * @param path - * @return list of child node names + * Gets all config change notifications */ - def getChildren(path : String): Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path)) + def getConfigChangeNotifications(): Seq[String] = { + val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(ConfigEntityChangeNotificationZNode.path)) if (getChildrenResponse.resultCode == Code.OK) { - getChildrenResponse.children + getChildrenResponse.children.map(ConfigEntityChangeNotificationSequenceZNode.sequenceNumber) } else if (getChildrenResponse.resultCode == Code.NONODE) { Seq.empty } else { @@ -765,34 +868,19 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } /** - * Gets the data and Stat at the given zk path - * @param path zk node path - * @return A tuple of 2 elements, where first element is zk node data as string - * and second element is zk node stats. - * returns (None, -1) if node doesn't exists, data is null or for any error. + * Get config change data and zk node Stat associated with the given sequence number + * @param sequenceNumber + * @return returns the data and Stat */ - def getDataAndStat(path: String): (Option[String], Stat) = { - val getDataRequest = GetDataRequest(path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - - if (getDataResponse.resultCode == Code.OK) { - if (getDataResponse.data == null) - (None, getDataResponse.stat) - else { - val data = new String(getDataResponse.data, "UTF-8") - (Some(data), getDataResponse.stat) - } - } else - (None, new Stat()) + def getDataFromConfigChangeNotification(sequenceNumber: String): (Option[String], Stat) = { + getDataAndStat(ConfigEntityChangeNotificationSequenceZNode.path(sequenceNumber)) } /** - * Deletes th zk node - * @param path - * @return return true if it succeeds, false otherwise + * Deletes Config change notification associated with the given sequence number */ - def deletePath(path: String): Boolean = { - val deleteRequest = DeleteRequest(path, -1) + def deleteConfigChangeNotification(sequenceNumber: String): Boolean = { + val deleteRequest = DeleteRequest(ConfigEntityChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) val deleteResponse = retryRequestUntilConnected(deleteRequest) if (deleteResponse.resultCode == Code.OK || deleteResponse.resultCode == Code.NONODE) { true @@ -801,6 +889,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } } + /** * This registers a ZNodeChangeHandler and attempts to register a watcher with an ExistsRequest, which allows data * watcher registrations on paths which might not even exist. @@ -855,16 +944,16 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends * * @param stateChangeHandler */ - def registerStatusChangeHandler(stateChangeHandler: StateChangeHandler): Unit = { + def registerStateChangeHandler(stateChangeHandler: StateChangeHandler): Unit = { zooKeeperClient.registerStateChangeHandler(stateChangeHandler) } /** * - * @param path + * @param name */ - def unregisterStateChangeHandler(path: String): Unit = { - zooKeeperClient.unregisterStateChangeHandler(path) + def unregisterStateChangeHandler(name: String): Unit = { + zooKeeperClient.unregisterStateChangeHandler(name) } diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 6ca333e73d92f..d7cf31dc20de9 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -153,6 +153,19 @@ object ConfigEntityZNode { } } +object ConfigEntityChangeNotificationZNode { + def path = s"${ConfigZNode.path}/changes" +} + +object ConfigEntityChangeNotificationSequenceZNode { + val SequenceNumberPrefix = "config_change_" + def createPath = s"${ConfigEntityChangeNotificationZNode.path}/$SequenceNumberPrefix" + def path(sequenceNumber: String) = s"${ConfigEntityChangeNotificationZNode.path}/$SequenceNumberPrefix$sequenceNumber" + def encode(resourceName : String): Array[Byte] = resourceName.getBytes(UTF_8) + def decode(bytes: Array[Byte]): String = new String(bytes, UTF_8) + def sequenceNumber(path: String) = path.substring(path.lastIndexOf(SequenceNumberPrefix) + SequenceNumberPrefix.length) +} + object IsrChangeNotificationZNode { def path = "/isr_change_notification" } @@ -289,7 +302,9 @@ object AclChangeNotificationZNode { object AclChangeNotificationSequenceZNode { val SequenceNumberPrefix = "acl_changes_" - def path = s"${AclChangeNotificationZNode.path}/$SequenceNumberPrefix" + def createPath = s"${AclChangeNotificationZNode.path}/$SequenceNumberPrefix" + def path(sequenceNumber: String) = s"${AclChangeNotificationZNode.path}/$SequenceNumberPrefix$sequenceNumber" def encode(resourceName : String): Array[Byte] = resourceName.getBytes(UTF_8) def decode(bytes: Array[Byte]): String = new String(bytes, UTF_8) -} + def sequenceNumber(path: String) = path.substring(path.lastIndexOf(SequenceNumberPrefix) + SequenceNumberPrefix.length) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala index 6550c3d7efe68..cdc6e3299b040 100644 --- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala +++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala @@ -53,6 +53,7 @@ class ZooKeeperClient(connectString: String, private val zNodeChildChangeHandlers = new ConcurrentHashMap[String, ZNodeChildChangeHandler]().asScala private val inFlightRequests = new Semaphore(maxInFlightRequests) private val stateChangeHandlers = new ConcurrentHashMap[String, StateChangeHandler]().asScala + registerStateChangeHandler(stateChangeHandler) info(s"Initializing a new session to $connectString.") @@ -235,11 +236,18 @@ class ZooKeeperClient(connectString: String, zNodeChildChangeHandlers.remove(path) } - def registerStateChangeHandler(statusChangeHandler: StateChangeHandler): Unit = { - if (statusChangeHandler != null) - stateChangeHandlers.put(statusChangeHandler.name, statusChangeHandler) + /** + * @param stateChangeHandler + */ + def registerStateChangeHandler(stateChangeHandler: StateChangeHandler): Unit = { + if (stateChangeHandler != null) + stateChangeHandlers.put(stateChangeHandler.name, stateChangeHandler) } + /** + * + * @param path + */ def unregisterStateChangeHandler(path: String): Unit = { stateChangeHandlers.remove(path) } @@ -248,6 +256,7 @@ class ZooKeeperClient(connectString: String, info("Closing.") zNodeChangeHandlers.clear() zNodeChildChangeHandlers.clear() + stateChangeHandlers.clear() zooKeeper.close() info("Closed.") } @@ -277,7 +286,7 @@ class ZooKeeperClient(connectString: String, } } info(s"Timed out waiting for connection during session initialization while in state: ${zooKeeper.getState}") - stateChangeHandlers.foreach(_._2.onReconnectionTimeout()) + stateChangeHandlers.foreach {case (name, handler) => handler.onReconnectionTimeout()} } } @@ -291,13 +300,13 @@ class ZooKeeperClient(connectString: String, } if (event.getState == KeeperState.AuthFailed) { info("Auth failed.") - stateChangeHandlers.foreach(_._2.onAuthFailure()) + stateChangeHandlers.foreach {case (name, handler) => handler.onAuthFailure()} } else if (event.getState == KeeperState.Expired) { inWriteLock(initializationLock) { info("Session expired.") - stateChangeHandlers.foreach(_._2.beforeInitializingSession()) + stateChangeHandlers.foreach {case (name, handler) => handler.beforeInitializingSession()} initialize() - stateChangeHandlers.foreach(_._2.afterInitializingSession()) + stateChangeHandlers.foreach {case (name, handler) => handler.afterInitializingSession()} } } case Some(path) => diff --git a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala index 99550d5a60821..20143c4e95d82 100644 --- a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala +++ b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala @@ -17,7 +17,8 @@ package kafka.common import kafka.utils.TestUtils -import kafka.zk.{AclChangeNotificationSequenceZNode, AclChangeNotificationZNode, ZooKeeperTestHarness} +import kafka.zk.{AclChangeNotificationZNode, ZooKeeperTestHarness} +import org.apache.zookeeper.data.Stat import org.junit.Test class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { @@ -39,10 +40,14 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { val changeExpirationMs = 1000 val notificationListener = new ZkNodeChangeNotificationListener(zkClient, AclChangeNotificationZNode.path, - AclChangeNotificationSequenceZNode.SequenceNumberPrefix, notificationHandler, changeExpirationMs) + notificationHandler, changeExpirationMs) { + override def getAllChangeNotifications(): Seq[String] = zkClient.getAclChangeNotifications + override def deleteChangeNotification(notificationNode: String): Boolean = zkClient.deleteAclChangeNotification(notificationNode) + override def getDataFromChangeNotification(notificationNode: String): (Option[String], Stat) = zkClient.getDataFromAclChangeNotification(notificationNode) + } notificationListener.init() - zkClient.createAclChangeNotification(notificationMessage1) + zkClient.createAclChangeNotificationRaw(notificationMessage1) TestUtils.waitUntilTrue(() => invocationCount == 1 && notification == notificationMessage1, "Failed to send/process notification message in the timeout period.") @@ -54,11 +59,11 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { * can fail as the second node can be deleted depending on how threads get scheduled. */ - zkClient.createAclChangeNotification(notificationMessage2) + zkClient.createAclChangeNotificationRaw(notificationMessage2) TestUtils.waitUntilTrue(() => invocationCount == 2 && notification == notificationMessage2, "Failed to send/process notification message in the timeout period.") - (3 to 10).foreach(i => zkClient.createAclChangeNotification("message" + i)) + (3 to 10).foreach(i => zkClient.createAclChangeNotificationRaw("message" + i)) TestUtils.waitUntilTrue(() => invocationCount == 10 , s"Expected 10 invocations of processNotifications, but there were $invocationCount") diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala index b647493e7e0ec..a60dcc261dcfb 100644 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -44,8 +44,6 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { override def setUp() { super.setUp() - zkClient.createAclPaths - // Increase maxUpdateRetries to avoid transient failures simpleAclAuthorizer.maxUpdateRetries = Int.MaxValue simpleAclAuthorizer2.maxUpdateRetries = Int.MaxValue @@ -239,12 +237,12 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { //test remove all acls for resource simpleAclAuthorizer.removeAcls(resource) TestUtils.waitAndVerifyAcls(Set.empty[Acl], simpleAclAuthorizer, resource) - assertTrue(!zkUtils.pathExists(simpleAclAuthorizer.toResourcePath(resource))) + assertTrue(!zkClient.resourceExists(resource)) //test removing last acl also deletes ZooKeeper path acls = changeAclAndVerify(Set.empty[Acl], Set(acl1), Set.empty[Acl]) changeAclAndVerify(acls, Set.empty[Acl], acls) - assertTrue(!zkUtils.pathExists(simpleAclAuthorizer.toResourcePath(resource))) + assertTrue(!zkClient.resourceExists(resource)) } @Test @@ -260,7 +258,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { val acls1 = Set[Acl](acl2) simpleAclAuthorizer.addAcls(acls1, resource1) - zkUtils.deletePathRecursive(AclChangeNotificationZNode.path) + zkClient.deleteAclChangeNotifications val authorizer = new SimpleAclAuthorizer try { authorizer.configure(config.originals) diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index 7614347053782..cd558fb649a58 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -18,8 +18,12 @@ package kafka.zk import kafka.server.Defaults import kafka.zookeeper.ZooKeeperClient +import java.util.UUID + import kafka.common.TopicAndPartition +import kafka.security.auth._ import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.security.auth.KafkaPrincipal import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import org.junit.Test @@ -164,4 +168,118 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { assertEquals(Map.empty, zkClient.getPartitionReassignment) } + @Test + def testGetDataAndStat() { + val path = "/testpath" + + // test with non-existing path + var dataAndVersion = zkClient.getDataAndStat(path) + assertTrue(dataAndVersion._1.isEmpty) + assertEquals(0, dataAndVersion._2.getVersion) + + // create a test path + zkClient.createRecursive(path) + zkClient.conditionalUpdatePath(path, "version1", 0) + + // test with existing path + dataAndVersion = zkClient.getDataAndStat(path) + assertEquals("version1", dataAndVersion._1.get) + assertEquals(1, dataAndVersion._2.getVersion) + + zkClient.conditionalUpdatePath(path, "version2", 1) + dataAndVersion = zkClient.getDataAndStat(path) + assertEquals("version2", dataAndVersion._1.get) + assertEquals(2, dataAndVersion._2.getVersion) + } + + @Test + def testGetChildren() { + val path = "/testpath" + + // test with non-existing path + assertTrue(zkClient.getChildren(path).isEmpty) + + // create child nodes + zkClient.createRecursive( "/testpath/child1") + zkClient.createRecursive( "/testpath/child2") + zkClient.createRecursive( "/testpath/child3") + + val children = zkClient.getChildren(path) + + assertEquals(3, children.size) + assertEquals(Set("child1","child2","child3"), children.toSet) + + } + + @Test + def testAclManagementMethods() { + + assertFalse(zkClient.pathExists(AclZNode.path)) + assertFalse(zkClient.pathExists(AclChangeNotificationZNode.path)) + ResourceType.values.foreach(resource => assertFalse(zkClient.pathExists(ResourceTypeZNode.path(resource.name)))) + + // create acl paths + zkClient.createAclPaths + + assertTrue(zkClient.pathExists(AclZNode.path)) + assertTrue(zkClient.pathExists(AclChangeNotificationZNode.path)) + ResourceType.values.foreach(resource => assertTrue(zkClient.pathExists(ResourceTypeZNode.path(resource.name)))) + + val resource1 = new Resource(Topic, UUID.randomUUID().toString) + val resource2 = new Resource(Topic, UUID.randomUUID().toString) + + // try getting acls for non-existing resource + var versionedAcls = zkClient.getVersionedAclsForResource(resource1) + assertTrue(versionedAcls.acls.isEmpty) + assertEquals(-1, versionedAcls.zkVersion) + assertFalse(zkClient.resourceExists(resource1)) + + + val acl1 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "alice"), Deny, "host1" , Read) + val acl2 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Allow, "*", Read) + val acl3 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Deny, "host1", Read) + + //create acls for resources + zkClient.createAclForResourceRaw(resource1, Set(acl1, acl2)) + zkClient.createAclForResourceRaw(resource2, Set(acl1, acl3)) + + versionedAcls = zkClient.getVersionedAclsForResource(resource1) + assertEquals(Set(acl1, acl2), versionedAcls.acls) + assertEquals(0, versionedAcls.zkVersion) + assertTrue(zkClient.resourceExists(resource1)) + + //update acls for resource + zkClient.setAclForResourceRaw(resource1, Set(acl1, acl3), 0) + + versionedAcls = zkClient.getVersionedAclsForResource(resource1) + assertEquals(Set(acl1, acl3), versionedAcls.acls) + assertEquals(1, versionedAcls.zkVersion) + + //create Acl change notification message + zkClient.createAclChangeNotificationRaw(resource1.name) + + val aclChangeNotifications = zkClient.getAclChangeNotifications + assertEquals(1, aclChangeNotifications.size) + + //get data for change notification + val dataForNotification = zkClient.getDataFromAclChangeNotification(aclChangeNotifications.head) + assertEquals(resource1.name, dataForNotification._1.get) + + assertTrue(zkClient.pathExists(AclChangeNotificationSequenceZNode.path(aclChangeNotifications.head))) + //delete change notification node + zkClient.deleteAclChangeNotification(aclChangeNotifications.head) + assertFalse(zkClient.pathExists(AclChangeNotificationSequenceZNode.path(aclChangeNotifications.head))) + + //get resource Types + assertTrue(ResourceType.values.map( rt => rt.name).toSet == zkClient.getResourceTypes().toSet) + + //get resource name + val resourceNames = zkClient.getResourceNames(Topic.name) + assertEquals(2, resourceNames.size) + assertTrue(Set(resource1.name,resource2.name) == resourceNames.toSet) + + //delete resource + assertTrue(zkClient.deleteResource(resource1)) + assertFalse(zkClient.pathExists(ResourceZNode.path(resource1))) + } } diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 1e6fdd411afc0..04e9147785777 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -42,6 +42,7 @@ abstract class ZooKeeperTestHarness extends JUnitSuite with Logging { protected val zkAclsEnabled: Option[Boolean] = None var zkUtils: ZkUtils = null + var zooKeeperClient: ZooKeeperClient = null var zkClient: KafkaZkClient = null var zookeeper: EmbeddedZookeeper = null @@ -53,11 +54,7 @@ abstract class ZooKeeperTestHarness extends JUnitSuite with Logging { zookeeper = new EmbeddedZookeeper() zkUtils = ZkUtils(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled())) - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, - zkConnectionTimeout, new StateChangeHandler { - override val name: String = this.getClass.getName - }) - + zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, null) zkClient = new KafkaZkClient(zooKeeperClient, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled())) } diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index d923ea6561ac4..11915e3906a85 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -58,7 +58,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testDeleteNonExistentZNode(): Unit = { - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val deleteResponse = zooKeeperClient.handleRequest(DeleteRequest(mockPath, -1)) assertEquals("Response code should be NONODE", Code.NONODE, deleteResponse.resultCode) } @@ -66,7 +65,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testDeleteExistingZNode(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) val deleteResponse = zooKeeperClient.handleRequest(DeleteRequest(mockPath, -1)) @@ -75,7 +73,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testExistsNonExistentZNode(): Unit = { - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val existsResponse = zooKeeperClient.handleRequest(ExistsRequest(mockPath)) assertEquals("Response code should be NONODE", Code.NONODE, existsResponse.resultCode) } @@ -83,7 +80,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testExistsExistingZNode(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) val existsResponse = zooKeeperClient.handleRequest(ExistsRequest(mockPath)) @@ -92,7 +88,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testGetDataNonExistentZNode(): Unit = { - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val getDataResponse = zooKeeperClient.handleRequest(GetDataRequest(mockPath)) assertEquals("Response code should be NONODE", Code.NONODE, getDataResponse.resultCode) } @@ -101,7 +96,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { def testGetDataExistingZNode(): Unit = { import scala.collection.JavaConverters._ val data = bytes - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, data, ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) @@ -112,7 +106,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testSetDataNonExistentZNode(): Unit = { - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val setDataResponse = zooKeeperClient.handleRequest(SetDataRequest(mockPath, Array.empty[Byte], -1)) assertEquals("Response code should be NONODE", Code.NONODE, setDataResponse.resultCode) } @@ -121,7 +114,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { def testSetDataExistingZNode(): Unit = { import scala.collection.JavaConverters._ val data = bytes - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) @@ -134,7 +126,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testGetAclNonExistentZNode(): Unit = { - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val getAclResponse = zooKeeperClient.handleRequest(GetAclRequest(mockPath)) assertEquals("Response code should be NONODE", Code.NONODE, getAclResponse.resultCode) } @@ -142,7 +133,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testGetAclExistingZNode(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) val getAclResponse = zooKeeperClient.handleRequest(GetAclRequest(mockPath)) @@ -153,14 +143,12 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testSetAclNonExistentZNode(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val setAclResponse = zooKeeperClient.handleRequest(SetAclRequest(mockPath, ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, -1)) assertEquals("Response code should be NONODE", Code.NONODE, setAclResponse.resultCode) } @Test def testGetChildrenNonExistentZNode(): Unit = { - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val getChildrenResponse = zooKeeperClient.handleRequest(GetChildrenRequest(mockPath)) assertEquals("Response code should be NONODE", Code.NONODE, getChildrenResponse.resultCode) } @@ -168,7 +156,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testGetChildrenExistingZNode(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) @@ -184,7 +171,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { val child2 = "child2" val child1Path = mockPath + "/" + child1 val child2Path = mockPath + "/" + child2 - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) @@ -203,7 +189,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testPipelinedGetData(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createRequests = (1 to 3).map(x => CreateRequest("/" + x, (x * 2).toString.getBytes, ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) val createResponses = createRequests.map(zooKeeperClient.handleRequest) createResponses.foreach(createResponse => assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode)) @@ -220,7 +205,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testMixedPipeline(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala, CreateMode.PERSISTENT)) assertEquals("Response code for create should be OK", Code.OK, createResponse.resultCode) @@ -235,7 +219,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChangeHandlerForCreation(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val znodeChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChangeHandler = new ZNodeChangeHandler { override def handleCreation(): Unit = { @@ -256,7 +239,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChangeHandlerForDeletion(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val znodeChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChangeHandler = new ZNodeChangeHandler { override def handleDeletion(): Unit = { @@ -279,7 +261,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChangeHandlerForDataChange(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val znodeChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChangeHandler = new ZNodeChangeHandler { override def handleDataChange(): Unit = { @@ -302,7 +283,6 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testZNodeChildChangeHandlerForChildChange(): Unit = { import scala.collection.JavaConverters._ - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) val zNodeChildChangeHandlerCountDownLatch = new CountDownLatch(1) val zNodeChildChangeHandler = new ZNodeChildChangeHandler { override def handleChildChange(): Unit = { From c1ebfe7781aa4c7bdda751b19e181ed533d5a394 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Fri, 3 Nov 2017 15:45:52 +0530 Subject: [PATCH 3/8] Address review commnets --- .../ZkNodeChangeNotificationListener.scala | 35 ++-- .../kafka/controller/KafkaController.scala | 8 + .../security/auth/SimpleAclAuthorizer.scala | 60 +------ .../kafka/server/DynamicConfigManager.scala | 9 +- .../main/scala/kafka/server/KafkaServer.scala | 4 + .../main/scala/kafka/zk/KafkaZkClient.scala | 167 +++++++----------- core/src/main/scala/kafka/zk/ZkData.scala | 13 -- .../kafka/zookeeper/ZooKeeperClient.scala | 23 ++- ...ZkNodeChangeNotificationListenerTest.scala | 15 +- .../unit/kafka/zk/KafkaZkClientTest.scala | 29 +-- .../unit/kafka/zk/ZooKeeperTestHarness.scala | 2 +- .../kafka/zookeeper/ZooKeeperClientTest.scala | 14 +- 12 files changed, 144 insertions(+), 235 deletions(-) diff --git a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala index 555e4d6953496..5beadbb108272 100644 --- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala @@ -23,7 +23,6 @@ import kafka.utils.{Logging, ShutdownableThread} import kafka.zk.KafkaZkClient import kafka.zookeeper.{StateChangeHandler, ZNodeChildChangeHandler} import org.apache.kafka.common.utils.Time -import org.apache.zookeeper.data.Stat /** * Handle the notificationMessage. @@ -41,12 +40,14 @@ trait NotificationHandler { * * @param zkClient * @param seqNodeRoot + * @param seqNodePrefix * @param notificationHandler * @param changeExpirationMs * @param time */ -abstract class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, +class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, private val seqNodeRoot: String, + private val seqNodePrefix: String, private val notificationHandler: NotificationHandler, private val changeExpirationMs: Long = 15 * 60 * 1000, private val time: Time = Time.SYSTEM) extends Logging { @@ -75,19 +76,21 @@ abstract class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkCli */ private def processNotifications() { try { - val notifications = getAllChangeNotifications.sorted + val notifications = zkClient.getChildren(seqNodeRoot).sorted if (notifications.nonEmpty) { info(s"Processing notification(s) to $seqNodeRoot") val now = time.milliseconds for (notification <- notifications) { - if (notification.toLong > lastExecutedChange) { - val data = getDataFromChangeNotification(notification)._1.orNull + val changeId = changeNumber(notification) + if (changeId > lastExecutedChange) { + val changeZnode = seqNodeRoot + "/" + notification + val data = zkClient.getDataAndStat(changeZnode)._1.orNull if (data != null) { notificationHandler.processNotification(data) } else { - logger.warn(s"read null data from $notification sequence node when processing notification for path = $seqNodeRoot") + logger.warn(s"read null data from $changeZnode when processing notification $notification") } - lastExecutedChange = notification.toLong + lastExecutedChange = changeId } } purgeObsoleteNotifications(now, notifications) @@ -98,12 +101,6 @@ abstract class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkCli } } - def getAllChangeNotifications() : Seq[String] - - def deleteChangeNotification(sequenceNumber: String) : Boolean - - def getDataFromChangeNotification(sequenceNumber: String) : (Option[String], Stat) - private def addChangeNotification(): Unit = { if (!isClosed.get && queue.peek() == null) queue.put(new ChangeNotification) @@ -121,16 +118,20 @@ abstract class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkCli */ private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { for (notification <- notifications.sorted) { - val (data, stat) = getDataFromChangeNotification(notification) + val notificationNode = seqNodeRoot + "/" + notification + val (data, stat) = zkClient.getDataAndStat(notificationNode) if (data.isDefined) { if (now - stat.getCtime > changeExpirationMs) { - debug(s"Purging change notification $notification for path = $seqNodeRoot") - deleteChangeNotification(notification) + debug(s"Purging change notification $notificationNode") + zkClient.deletePath(notificationNode) } } } } + /* get the change number from a change notification znode */ + private def changeNumber(name: String): Long = name.substring(seqNodePrefix.length).toLong + class ChangeEventProcessThread(name: String) extends ShutdownableThread(name = name) { override def doWork(): Unit = queue.take().process } @@ -143,6 +144,8 @@ abstract class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkCli object ZkStateChangeListener extends StateChangeHandler { override val name: String = seqNodeRoot override def afterInitializingSession(): Unit = addChangeNotification + override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") + override def onAuthFailure(): Unit = error("Auth failure.") } } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index ade3ae486e8ff..ec7ab1edd3ba9 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -29,6 +29,7 @@ import kafka.zk._ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult import kafka.zookeeper.{ZNodeChangeHandler, ZNodeChildChangeHandler} import org.apache.kafka.common.TopicPartition +import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler, ZNodeChildChangeHandler} import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -518,6 +519,13 @@ class KafkaController(val config: KafkaConfig, zkClient: KafkaZkClient, time: Ti * elector */ def startup() = { + zkClient.registerStateChangeHandler(new StateChangeHandler { + override val name: String = KafkaController.getClass.getName + override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") + override def afterInitializingSession(): Unit = newSession() + override def onAuthFailure(): Unit = error("Auth failure.") + override def beforeInitializingSession(): Unit = expire() + }) eventManager.put(Startup) eventManager.start() } diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index e3559a8860098..a58a47c0cf0c6 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -25,13 +25,11 @@ import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls import kafka.server.KafkaConfig import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ -import kafka.zk.{AclChangeNotificationZNode, AclZNode, KafkaZkClient} -import kafka.zookeeper.{StateChangeHandler, ZooKeeperClient} +import kafka.zk.{AclChangeNotificationSequenceZNode, AclChangeNotificationZNode, KafkaZkClient} +import kafka.zookeeper.ZooKeeperClient import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.SecurityUtils import org.apache.log4j.Logger -import org.apache.zookeeper.KeeperException.Code -import org.apache.zookeeper.data.Stat import scala.collection.JavaConverters._ import scala.util.Random @@ -90,26 +88,14 @@ class SimpleAclAuthorizer extends Authorizer with Logging { val zkConnectionTimeoutMs = configs.get(SimpleAclAuthorizer.ZkConnectionTimeOutProp).map(_.toString.toInt).getOrElse(kafkaConfig.zkConnectionTimeoutMs) val zkSessionTimeOutMs = configs.get(SimpleAclAuthorizer.ZkSessionTimeOutProp).map(_.toString.toInt).getOrElse(kafkaConfig.zkSessionTimeoutMs) - val zooKeeperClient = new ZooKeeperClient(zkUrl, zkSessionTimeOutMs, - zkConnectionTimeoutMs, new StateChangeHandler { - override val name: String = SimpleAclAuthorizer.getClass.getName - override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") - override def afterInitializingSession(): Unit = debug("Initialised Session.") - override def onAuthFailure(): Unit = error("Auth failure.") - }) + val zooKeeperClient = new ZooKeeperClient(zkUrl, zkSessionTimeOutMs, zkConnectionTimeoutMs) zkClient = new KafkaZkClient(zooKeeperClient, kafkaConfig.zkEnableSecureAcls) zkClient.createAclPaths() loadCache() - aclChangeListener = new ZkNodeChangeNotificationListener(zkClient, AclChangeNotificationZNode.path, AclChangedNotificationHandler) { - override def getAllChangeNotifications(): Seq[String] = zkClient.getAclChangeNotifications - override def deleteChangeNotification(sequenceNumber: String): Boolean = zkClient.deleteAclChangeNotification(sequenceNumber) - override def getDataFromChangeNotification(sequenceNumber: String): (Option[String], Stat) = - zkClient.getDataFromAclChangeNotification(sequenceNumber) - } - + aclChangeListener = new ZkNodeChangeNotificationListener(zkClient, AclChangeNotificationZNode.path, AclChangeNotificationSequenceZNode.SequenceNumberPrefix, AclChangedNotificationHandler) aclChangeListener.init() } @@ -268,7 +254,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { val newAcls = getNewAcls(currentVersionedAcls.acls) val (updateSucceeded, updateVersion) = if (newAcls.nonEmpty) { - conditionalUpdate(resource, newAcls, currentVersionedAcls.zkVersion) + zkClient.conditionalSetOrCreateAclsForResource(resource, newAcls, currentVersionedAcls.zkVersion) } else { trace(s"Deleting path for $resource because it had no ACLs remaining") (zkClient.conditionalDelete(resource, currentVersionedAcls.zkVersion), 0) @@ -300,39 +286,6 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } } - /** - * Updates a resource zk path with an expected version. If the resource does not exist, it will create it. - * @param resource - * @param acls - * @param expectedVersion - * @return true if the update was successful and the new version - */ - private def conditionalUpdate(resource: Resource, acls: Set[Acl], expectedVersion: Int): (Boolean, Int) = { - val setDataResponse = zkClient.setAclForResourceRaw(resource, acls, expectedVersion) - setDataResponse.resultCode match { - case Code.OK => - (true, setDataResponse.stat.getVersion) - case Code.NONODE => { - val createResponse = zkClient.createAclForResourceRaw(resource, acls) - createResponse.resultCode match { - case Code.OK => - (true, 0) - case Code.NODEEXISTS => - (false, 0) - case _ => - error(s"Error while creating acls at $resource") - throw createResponse.resultException.get - } - } - case Code.BADVERSION => - debug(s"Failed to update node for $resource due to bad version.") - (false, 0) - case _ => - debug(s"Error while updating node at $resource") - throw setDataResponse.resultException.get - } - } - private def getAclsFromCache(resource: Resource): VersionedAcls = { aclCache.getOrElse(resource, throw new IllegalArgumentException(s"ACLs do not exist in the cache for resource $resource")) } @@ -350,7 +303,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } private def updateAclChangedFlag(resource: Resource) { - zkClient.createAclChangeNotificationRaw(resource.toString) + zkClient.createAclChangeNotification(resource.toString) } private def backoffTime = { @@ -366,4 +319,5 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } } } + } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index 2320a3efab848..3524889f85491 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -30,7 +30,6 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.security.scram.ScramMechanism import org.apache.kafka.common.utils.Time -import org.apache.zookeeper.data.Stat /** * Represents all the entities that can be configured via ZK @@ -152,12 +151,8 @@ class DynamicConfigManager(private val oldZkUtils: ZkUtils, } } - private val configChangeListener = new ZkNodeChangeNotificationListener(zkClient, ZkUtils.ConfigChangesPath, ConfigChangedNotificationHandler) { - override def getAllChangeNotifications(): Seq[String] = zkClient.getConfigChangeNotifications() - override def deleteChangeNotification(sequenceNumber: String): Boolean = zkClient.deleteConfigChangeNotification(sequenceNumber) - override def getDataFromChangeNotification(sequenceNumber: String): (Option[String], Stat) = - zkClient.getDataFromConfigChangeNotification(sequenceNumber) - } + private val configChangeListener = new ZkNodeChangeNotificationListener(zkClient, ZkUtils.ConfigChangesPath, + AdminUtils.EntityConfigChangeZnodePrefix, ConfigChangedNotificationHandler) /** * Begin watching for config changes diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 6cb90a9181c49..b690c341b9ea6 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -221,6 +221,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size) +<<<<<<< 36496a23e3048a5dbe7862f24d9600fc40f4c839 val zooKeeperClient = new ZooKeeperClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, config.zkMaxInFlightRequests, new StateChangeHandler { override val name: String = KafkaServer.getClass.getName @@ -237,6 +238,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP override def beforeInitializingSession(): Unit = kafkaController.expire() }) +======= + val zooKeeperClient = new ZooKeeperClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) +>>>>>>> Address review commnets zkClient = new KafkaZkClient(zooKeeperClient, zkUtils.isSecure) /* start log manager */ diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index e67e494b65419..877ad8f1b7bcc 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -19,14 +19,12 @@ package kafka.zk import java.nio.charset.StandardCharsets.UTF_8 import java.util.Properties -import java.nio.charset.StandardCharsets.UTF_8 - import kafka.api.LeaderAndIsr import kafka.cluster.Broker import kafka.controller.LeaderIsrAndControllerEpoch import kafka.log.LogConfig -import kafka.security.auth.{Acl, Resource, ResourceType} import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls +import kafka.security.auth.{Acl, Resource, ResourceType} import kafka.server.ConfigType import kafka.utils._ import kafka.zookeeper._ @@ -358,7 +356,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends * and second element is zk node stats. * returns (None, new Stat()) if node doesn't exists and throws exception for any error */ - private[zk] def getDataAndStat(path: String): (Option[String], Stat) = { + def getDataAndStat(path: String): (Option[String], Stat) = { val getDataRequest = GetDataRequest(path) val getDataResponse = retryRequestUntilConnected(getDataRequest) @@ -381,7 +379,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends * @param path * @return list of child node names */ - private[zk] def getChildren(path : String): Seq[String] = { + def getChildren(path : String): Seq[String] = { val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path)) if (getChildrenResponse.resultCode == Code.OK) { getChildrenResponse.children @@ -701,34 +699,57 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } /** - * Sets Resource znode with the given acls. - * @param resource resource object - * @param acls the acls to sets on the resource znode - * @param expectedVersion expected zk version - * @return SetDataResponse + * Sets or creates the resource znode path with the given acls and expected zk version depending + * on whether it already exists or not. + * @param resource + * @param aclsSet + * @param expectedVersion + * @return true if the update was successful and the new version */ - def setAclForResourceRaw(resource: Resource, acls: Set[Acl], expectedVersion: Int): SetDataResponse = { - val setDataRequest = SetDataRequest(ResourceZNode.path(resource), ResourceZNode.encode(acls), expectedVersion) - retryRequestUntilConnected(setDataRequest) - } + def conditionalSetOrCreateAclsForResource(resource: Resource, aclsSet: Set[Acl], expectedVersion: Int): (Boolean, Int) = { + def set(aclData: Array[Byte], expectedVersion: Int): SetDataResponse = { + val setDataRequest = SetDataRequest(ResourceZNode.path(resource), aclData, expectedVersion) + retryRequestUntilConnected(setDataRequest) + } - /** - * Creates Resource znode with the given acls - * @param resource resource object - * @param aclsSet the acls to sets on the resource znode - * @return CreateResponse - */ - def createAclForResourceRaw(resource: Resource, aclsSet: Set[Acl]): CreateResponse = { - val path = ResourceZNode.path(resource) - val createRequest = CreateRequest(path, ResourceZNode.encode(aclsSet), acls(path), CreateMode.PERSISTENT) - retryRequestUntilConnected(createRequest) + def create(aclData: Array[Byte]): CreateResponse = { + val path = ResourceZNode.path(resource) + val createRequest = CreateRequest(path, aclData, acls(path), CreateMode.PERSISTENT) + retryRequestUntilConnected(createRequest) + } + + val aclData = ResourceZNode.encode(aclsSet) + + val setDataResponse = set(aclData, expectedVersion) + setDataResponse.resultCode match { + case Code.OK => + (true, setDataResponse.stat.getVersion) + case Code.NONODE => { + val createResponse = create(aclData) + createResponse.resultCode match { + case Code.OK => + (true, 0) + case Code.NODEEXISTS => + (false, 0) + case _ => + error(s"Error while creating acls at $resource") + throw createResponse.resultException.get + } + } + case Code.BADVERSION => + debug(s"Failed to update node for $resource due to bad version.") + (false, 0) + case _ => + debug(s"Error while updating node at $resource") + throw setDataResponse.resultException.get + } } /** * Creates Acl change notification message * @param resourceName resource name */ - def createAclChangeNotificationRaw(resourceName: String): Unit = { + def createAclChangeNotification(resourceName: String): Unit = { val path = AclChangeNotificationSequenceZNode.createPath val createRequest = CreateRequest(path, AclChangeNotificationSequenceZNode.encode(resourceName), acls(path), CreateMode.PERSISTENT_SEQUENTIAL) val createResponse = retryRequestUntilConnected(createRequest) @@ -737,30 +758,6 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } } - /** - * Gets all Acl change notifications - * @return list of change node sequence numbers - */ - def getAclChangeNotifications(): Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(AclChangeNotificationZNode.path)) - if (getChildrenResponse.resultCode == Code.OK) { - getChildrenResponse.children.map(AclChangeNotificationSequenceZNode.sequenceNumber) - } else if (getChildrenResponse.resultCode == Code.NONODE) { - Seq.empty - } else { - throw getChildrenResponse.resultException.get - } - } - - /** - * Get Acl change data and zk node Stat associated with the given sequence number - * @param sequenceNumber - * @return - */ - def getDataFromAclChangeNotification(sequenceNumber: String): (Option[String], Stat) = { - getDataAndStat(AclChangeNotificationSequenceZNode.path(sequenceNumber)) - } - /** * Deletes all Acl change notifications. */ @@ -777,25 +774,16 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends * Deletes the Acl change notifications associated with the given sequence numbers. * @param sequenceNumbers the sequence numbers associated with the Acl change notifications to be deleted. */ - def deleteAclChangeNotifications(sequenceNumbers: Seq[String]): Unit = { + private def deleteAclChangeNotifications(sequenceNumbers: Seq[String]): Unit = { val deleteRequests = sequenceNumbers.map { sequenceNumber => DeleteRequest(AclChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) } - retryRequestsUntilConnected(deleteRequests) - } - /** - * Deletes Acl change notification associated with the given sequence number - * @param sequenceNumber - * @return delete status - */ - def deleteAclChangeNotification(sequenceNumber: String): Boolean = { - val deleteRequest = DeleteRequest(AclChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) - val deleteResponse = retryRequestUntilConnected(deleteRequest) - if (deleteResponse.resultCode == Code.OK || deleteResponse.resultCode == Code.NONODE) { - true - }else { - throw deleteResponse.resultException.get + val deleteResponses = retryRequestsUntilConnected(deleteRequests) + deleteResponses.foreach { deleteResponse => + if (deleteResponse.resultCode != Code.OK && deleteResponse.resultCode != Code.NONODE) { + throw deleteResponse.resultException.get + } } } @@ -823,7 +811,6 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends */ def deleteResource(resource: Resource): Boolean = { deleteRecursive(ResourceZNode.path(resource)) - !resourceExists(resource) } /** @@ -852,44 +839,16 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends throw deleteResponse.resultException.get } } - + /** - * Gets all config change notifications - */ - def getConfigChangeNotifications(): Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(ConfigEntityChangeNotificationZNode.path)) - if (getChildrenResponse.resultCode == Code.OK) { - getChildrenResponse.children.map(ConfigEntityChangeNotificationSequenceZNode.sequenceNumber) - } else if (getChildrenResponse.resultCode == Code.NONODE) { - Seq.empty - } else { - throw getChildrenResponse.resultException.get - } - } - - /** - * Get config change data and zk node Stat associated with the given sequence number - * @param sequenceNumber - * @return returns the data and Stat - */ - def getDataFromConfigChangeNotification(sequenceNumber: String): (Option[String], Stat) = { - getDataAndStat(ConfigEntityChangeNotificationSequenceZNode.path(sequenceNumber)) - } - - /** - * Deletes Config change notification associated with the given sequence number + * Deletes th zk node recursively + * @param path + * @return return true if it succeeds, false otherwise */ - def deleteConfigChangeNotification(sequenceNumber: String): Boolean = { - val deleteRequest = DeleteRequest(ConfigEntityChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) - val deleteResponse = retryRequestUntilConnected(deleteRequest) - if (deleteResponse.resultCode == Code.OK || deleteResponse.resultCode == Code.NONODE) { - true - }else { - throw deleteResponse.resultException.get - } + def deletePath(path: String): Boolean = { + deleteRecursive(path) } - /** * This registers a ZNodeChangeHandler and attempts to register a watcher with an ExistsRequest, which allows data * watcher registrations on paths which might not even exist. @@ -1018,7 +977,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends createResponse } - private[zk] def deleteRecursive(path: String): Unit = { + private[zk] def deleteRecursive(path: String): Boolean = { val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path)) if (getChildrenResponse.resultCode == Code.OK) { getChildrenResponse.children.foreach(child => deleteRecursive(s"$path/$child")) @@ -1029,12 +988,18 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } else if (getChildrenResponse.resultCode != Code.NONODE) { throw getChildrenResponse.resultException.get } + true } private[zk] def pathExists(path: String): Boolean = { val getDataRequest = GetDataRequest(path) val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode == Code.OK + if (getDataResponse.resultCode == Code.OK) { + true + } else if (getDataResponse.resultCode == Code.NONODE) { + false + } else + throw getDataResponse.resultException.get } private[zk] def createRecursive(path: String): Unit = { diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index d7cf31dc20de9..230a64a86eb8a 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -153,19 +153,6 @@ object ConfigEntityZNode { } } -object ConfigEntityChangeNotificationZNode { - def path = s"${ConfigZNode.path}/changes" -} - -object ConfigEntityChangeNotificationSequenceZNode { - val SequenceNumberPrefix = "config_change_" - def createPath = s"${ConfigEntityChangeNotificationZNode.path}/$SequenceNumberPrefix" - def path(sequenceNumber: String) = s"${ConfigEntityChangeNotificationZNode.path}/$SequenceNumberPrefix$sequenceNumber" - def encode(resourceName : String): Array[Byte] = resourceName.getBytes(UTF_8) - def decode(bytes: Array[Byte]): String = new String(bytes, UTF_8) - def sequenceNumber(path: String) = path.substring(path.lastIndexOf(SequenceNumberPrefix) + SequenceNumberPrefix.length) -} - object IsrChangeNotificationZNode { def path = "/isr_change_notification" } diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala index cdc6e3299b040..389a56290a548 100644 --- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala +++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala @@ -38,14 +38,13 @@ import scala.collection.JavaConverters._ * @param sessionTimeoutMs session timeout in milliseconds * @param connectionTimeoutMs connection timeout in milliseconds * @param maxInFlightRequests maximum number of unacknowledged requests the client will send before blocking. - * @param stateChangeHandler state change handler callbacks called by the underlying zookeeper client's EventThread. */ class ZooKeeperClient(connectString: String, sessionTimeoutMs: Int, connectionTimeoutMs: Int, - maxInFlightRequests: Int, - stateChangeHandler: StateChangeHandler) extends Logging { + maxInFlightRequests: Int) extends Logging { this.logIdent = "[ZooKeeperClient] " + private val initializationLock = new ReentrantReadWriteLock() private val isConnectedOrExpiredLock = new ReentrantLock() private val isConnectedOrExpiredCondition = isConnectedOrExpiredLock.newCondition() @@ -54,8 +53,6 @@ class ZooKeeperClient(connectString: String, private val inFlightRequests = new Semaphore(maxInFlightRequests) private val stateChangeHandlers = new ConcurrentHashMap[String, StateChangeHandler]().asScala - registerStateChangeHandler(stateChangeHandler) - info(s"Initializing a new session to $connectString.") @volatile private var zooKeeper = new ZooKeeper(connectString, sessionTimeoutMs, ZooKeeperClientWatcher) waitUntilConnected(connectionTimeoutMs, TimeUnit.MILLISECONDS) @@ -239,17 +236,17 @@ class ZooKeeperClient(connectString: String, /** * @param stateChangeHandler */ - def registerStateChangeHandler(stateChangeHandler: StateChangeHandler): Unit = { + def registerStateChangeHandler(stateChangeHandler: StateChangeHandler): Unit = inReadLock(initializationLock) { if (stateChangeHandler != null) stateChangeHandlers.put(stateChangeHandler.name, stateChangeHandler) } /** * - * @param path + * @param name */ - def unregisterStateChangeHandler(path: String): Unit = { - stateChangeHandlers.remove(path) + def unregisterStateChangeHandler(name: String): Unit = { + stateChangeHandlers.remove(name) } def close(): Unit = inWriteLock(initializationLock) { @@ -290,6 +287,14 @@ class ZooKeeperClient(connectString: String, } } + /** + * reinitialize method to use in unit tests + */ + private[zookeeper] def reinitialize(): Unit = { + zooKeeper.close() + initialize() + } + private object ZooKeeperClientWatcher extends Watcher { override def process(event: WatchedEvent): Unit = { debug("Received event: " + event) diff --git a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala index 20143c4e95d82..99550d5a60821 100644 --- a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala +++ b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala @@ -17,8 +17,7 @@ package kafka.common import kafka.utils.TestUtils -import kafka.zk.{AclChangeNotificationZNode, ZooKeeperTestHarness} -import org.apache.zookeeper.data.Stat +import kafka.zk.{AclChangeNotificationSequenceZNode, AclChangeNotificationZNode, ZooKeeperTestHarness} import org.junit.Test class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { @@ -40,14 +39,10 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { val changeExpirationMs = 1000 val notificationListener = new ZkNodeChangeNotificationListener(zkClient, AclChangeNotificationZNode.path, - notificationHandler, changeExpirationMs) { - override def getAllChangeNotifications(): Seq[String] = zkClient.getAclChangeNotifications - override def deleteChangeNotification(notificationNode: String): Boolean = zkClient.deleteAclChangeNotification(notificationNode) - override def getDataFromChangeNotification(notificationNode: String): (Option[String], Stat) = zkClient.getDataFromAclChangeNotification(notificationNode) - } + AclChangeNotificationSequenceZNode.SequenceNumberPrefix, notificationHandler, changeExpirationMs) notificationListener.init() - zkClient.createAclChangeNotificationRaw(notificationMessage1) + zkClient.createAclChangeNotification(notificationMessage1) TestUtils.waitUntilTrue(() => invocationCount == 1 && notification == notificationMessage1, "Failed to send/process notification message in the timeout period.") @@ -59,11 +54,11 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { * can fail as the second node can be deleted depending on how threads get scheduled. */ - zkClient.createAclChangeNotificationRaw(notificationMessage2) + zkClient.createAclChangeNotification(notificationMessage2) TestUtils.waitUntilTrue(() => invocationCount == 2 && notification == notificationMessage2, "Failed to send/process notification message in the timeout period.") - (3 to 10).foreach(i => zkClient.createAclChangeNotificationRaw("message" + i)) + (3 to 10).foreach(i => zkClient.createAclChangeNotification("message" + i)) TestUtils.waitUntilTrue(() => invocationCount == 10 , s"Expected 10 invocations of processNotifications, but there were $invocationCount") diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index cd558fb649a58..bc24de86121dd 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -208,7 +208,6 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { assertEquals(3, children.size) assertEquals(Set("child1","child2","child3"), children.toSet) - } @Test @@ -240,8 +239,8 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { val acl3 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob"), Deny, "host1", Read) //create acls for resources - zkClient.createAclForResourceRaw(resource1, Set(acl1, acl2)) - zkClient.createAclForResourceRaw(resource2, Set(acl1, acl3)) + zkClient.conditionalSetOrCreateAclsForResource(resource1, Set(acl1, acl2), 0) + zkClient.conditionalSetOrCreateAclsForResource(resource2, Set(acl1, acl3), 0) versionedAcls = zkClient.getVersionedAclsForResource(resource1) assertEquals(Set(acl1, acl2), versionedAcls.acls) @@ -249,27 +248,12 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { assertTrue(zkClient.resourceExists(resource1)) //update acls for resource - zkClient.setAclForResourceRaw(resource1, Set(acl1, acl3), 0) + zkClient.conditionalSetOrCreateAclsForResource(resource1, Set(acl1, acl3), 0) versionedAcls = zkClient.getVersionedAclsForResource(resource1) assertEquals(Set(acl1, acl3), versionedAcls.acls) assertEquals(1, versionedAcls.zkVersion) - //create Acl change notification message - zkClient.createAclChangeNotificationRaw(resource1.name) - - val aclChangeNotifications = zkClient.getAclChangeNotifications - assertEquals(1, aclChangeNotifications.size) - - //get data for change notification - val dataForNotification = zkClient.getDataFromAclChangeNotification(aclChangeNotifications.head) - assertEquals(resource1.name, dataForNotification._1.get) - - assertTrue(zkClient.pathExists(AclChangeNotificationSequenceZNode.path(aclChangeNotifications.head))) - //delete change notification node - zkClient.deleteAclChangeNotification(aclChangeNotifications.head) - assertFalse(zkClient.pathExists(AclChangeNotificationSequenceZNode.path(aclChangeNotifications.head))) - //get resource Types assertTrue(ResourceType.values.map( rt => rt.name).toSet == zkClient.getResourceTypes().toSet) @@ -280,6 +264,11 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { //delete resource assertTrue(zkClient.deleteResource(resource1)) - assertFalse(zkClient.pathExists(ResourceZNode.path(resource1))) + assertFalse(zkClient.resourceExists(resource1)) + + //delete with invalid expected zk version + assertFalse(zkClient.conditionalDelete(resource2, 10)) + //delete with valid expected zk version + assertTrue(zkClient.conditionalDelete(resource2, 0)) } } diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 04e9147785777..877c896882739 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -54,7 +54,7 @@ abstract class ZooKeeperTestHarness extends JUnitSuite with Logging { zookeeper = new EmbeddedZookeeper() zkUtils = ZkUtils(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled())) - zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, null) + zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout) zkClient = new KafkaZkClient(zooKeeperClient, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled())) } diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index 11915e3906a85..5513438f2922c 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.security.JaasUtils import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.{CreateMode, ZooDefs} import org.junit.Assert.{assertArrayEquals, assertEquals, assertTrue} -import org.junit.{After, Test} +import org.junit.{After, Ignore, Test} class ZooKeeperClientTest extends ZooKeeperTestHarness { private val mockPath = "/foo" @@ -42,18 +42,18 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test(expected = classOf[UnknownHostException]) def testUnresolvableConnectString(): Unit = { - new ZooKeeperClient("some.invalid.hostname.foo.bar.local", -1, -1, Int.MaxValue, null) + new ZooKeeperClient("some.invalid.hostname.foo.bar.local", -1, -1, Int.MaxValue) } @Test(expected = classOf[ZooKeeperClientTimeoutException]) def testConnectionTimeout(): Unit = { zookeeper.shutdown() - new ZooKeeperClient(zkConnect, zkSessionTimeout, connectionTimeoutMs = 100, Int.MaxValue, null) + new ZooKeeperClient(zkConnect, zkSessionTimeout, connectionTimeoutMs = 100, Int.MaxValue) } @Test def testConnection(): Unit = { - new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, null) + new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue) } @Test @@ -314,7 +314,11 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { stateChangeHandlerCountDownLatch.countDown() } } - new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, stateChangeHandler) + + val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue,) + zooKeeperClient.registerStateChangeHandler(stateChangeHandler) + zooKeeperClient.reinitialize() + assertTrue("Failed to receive auth failed notification", stateChangeHandlerCountDownLatch.await(5, TimeUnit.SECONDS)) } From e98987fe31e59df3a5376527eec03fb70e7ff826 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 7 Nov 2017 11:49:13 +0530 Subject: [PATCH 4/8] Address review comments and rebase the PR --- .../ZkNodeChangeNotificationListener.scala | 1 - .../kafka/controller/KafkaController.scala | 8 ++--- .../security/auth/SimpleAclAuthorizer.scala | 6 ++-- .../main/scala/kafka/server/KafkaServer.scala | 21 +----------- .../main/scala/kafka/zk/KafkaZkClient.scala | 33 +++++++++++-------- core/src/main/scala/kafka/zk/ZkData.scala | 12 +++---- .../kafka/zookeeper/ZooKeeperClient.scala | 5 ++- .../auth/SimpleAclAuthorizerTest.scala | 2 +- .../unit/kafka/zk/KafkaZkClientTest.scala | 13 ++++++-- .../unit/kafka/zk/ZooKeeperTestHarness.scala | 6 ++-- .../kafka/zookeeper/ZooKeeperClientTest.scala | 6 ++-- 11 files changed, 53 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala index 5beadbb108272..82ceaed068bdf 100644 --- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala @@ -145,7 +145,6 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, override val name: String = seqNodeRoot override def afterInitializingSession(): Unit = addChangeNotification override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") - override def onAuthFailure(): Unit = error("Auth failure.") } } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index ec7ab1edd3ba9..4bb2f5aaef68c 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -25,11 +25,10 @@ import kafka.common._ import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.server._ import kafka.utils._ -import kafka.zk._ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult -import kafka.zookeeper.{ZNodeChangeHandler, ZNodeChildChangeHandler} -import org.apache.kafka.common.TopicPartition +import kafka.zk._ import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler, ZNodeChildChangeHandler} +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -520,10 +519,9 @@ class KafkaController(val config: KafkaConfig, zkClient: KafkaZkClient, time: Ti */ def startup() = { zkClient.registerStateChangeHandler(new StateChangeHandler { - override val name: String = KafkaController.getClass.getName + override val name: String = StateChangeHandlers.controllerHandler override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") override def afterInitializingSession(): Unit = newSession() - override def onAuthFailure(): Unit = error("Auth failure.") override def beforeInitializingSession(): Unit = expire() }) eventManager.put(Startup) diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index a58a47c0cf0c6..eaebb88c0e183 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -40,6 +40,7 @@ object SimpleAclAuthorizer { val ZkUrlProp = "authorizer.zookeeper.url" val ZkConnectionTimeOutProp = "authorizer.zookeeper.connection.timeout.ms" val ZkSessionTimeOutProp = "authorizer.zookeeper.session.timeout.ms" + val ZkMaxInFlightRequests = "authorizer.zookeeper.max.in.flight.requests" //List of users that will be treated as super users and will have access to all the resources for all actions from all hosts, defaults to no super users. val SuperUsersProp = "super.users" @@ -87,8 +88,9 @@ class SimpleAclAuthorizer extends Authorizer with Logging { val zkUrl = configs.get(SimpleAclAuthorizer.ZkUrlProp).map(_.toString).getOrElse(kafkaConfig.zkConnect) val zkConnectionTimeoutMs = configs.get(SimpleAclAuthorizer.ZkConnectionTimeOutProp).map(_.toString.toInt).getOrElse(kafkaConfig.zkConnectionTimeoutMs) val zkSessionTimeOutMs = configs.get(SimpleAclAuthorizer.ZkSessionTimeOutProp).map(_.toString.toInt).getOrElse(kafkaConfig.zkSessionTimeoutMs) + val zkMaxInFlightRequests = configs.get(SimpleAclAuthorizer.ZkMaxInFlightRequests).map(_.toString.toInt).getOrElse(kafkaConfig.zkMaxInFlightRequests) - val zooKeeperClient = new ZooKeeperClient(zkUrl, zkSessionTimeOutMs, zkConnectionTimeoutMs) + val zooKeeperClient = new ZooKeeperClient(zkUrl, zkSessionTimeOutMs, zkConnectionTimeoutMs, zkMaxInFlightRequests) zkClient = new KafkaZkClient(zooKeeperClient, kafkaConfig.zkEnableSecureAcls) zkClient.createAclPaths() @@ -204,7 +206,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { def close() { if (aclChangeListener != null) aclChangeListener.close() - if (zkClient != null ) zkClient.close() + if (zkClient != null) zkClient.close() } private def loadCache() { diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index b690c341b9ea6..3f23ed1b82035 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -221,26 +221,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size) -<<<<<<< 36496a23e3048a5dbe7862f24d9600fc40f4c839 - val zooKeeperClient = new ZooKeeperClient(config.zkConnect, config.zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, config.zkMaxInFlightRequests, new StateChangeHandler { - override val name: String = KafkaServer.getClass.getName - - override def onReconnectionTimeout(): Unit = { - error("Reconnection timeout.") - } - - override def afterInitializingSession(): Unit = kafkaController.newSession() - - override def onAuthFailure(): Unit = { - error("Auth failure.") - } - - override def beforeInitializingSession(): Unit = kafkaController.expire() - }) -======= - val zooKeeperClient = new ZooKeeperClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) ->>>>>>> Address review commnets + val zooKeeperClient = new ZooKeeperClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, config.zkMaxInFlightRequests) zkClient = new KafkaZkClient(zooKeeperClient, zkUtils.isSecure) /* start log manager */ diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 877ad8f1b7bcc..b6f62f5a8ec6f 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -571,7 +571,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends def deleteIsrChangeNotifications(): Unit = { val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(IsrChangeNotificationZNode.path)) if (getChildrenResponse.resultCode == Code.OK) { - deleteIsrChangeNotifications(getChildrenResponse.children) + deleteIsrChangeNotifications(getChildrenResponse.children.map(IsrChangeNotificationSequenceZNode.sequenceNumber)) } else if (getChildrenResponse.resultCode != Code.NONODE) { throw getChildrenResponse.resultException.get } @@ -732,15 +732,13 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends case Code.NODEEXISTS => (false, 0) case _ => - error(s"Error while creating acls at $resource") throw createResponse.resultException.get } } case Code.BADVERSION => - debug(s"Failed to update node for $resource due to bad version.") (false, 0) case _ => - debug(s"Error while updating node at $resource") + error(s"Error while updating node at $resource") throw setDataResponse.resultException.get } } @@ -760,6 +758,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends /** * Deletes all Acl change notifications. + * @throws KeeperException if there is an error while deleting Acl change notifications */ def deleteAclChangeNotifications(): Unit = { val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(AclChangeNotificationZNode.path)) @@ -771,12 +770,12 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } /** - * Deletes the Acl change notifications associated with the given sequence numbers. - * @param sequenceNumbers the sequence numbers associated with the Acl change notifications to be deleted. + * Deletes the Acl change notifications associated with the given sequence nodes + * @param sequenceNodes */ - private def deleteAclChangeNotifications(sequenceNumbers: Seq[String]): Unit = { - val deleteRequests = sequenceNumbers.map { sequenceNumber => - DeleteRequest(AclChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) + private def deleteAclChangeNotifications(sequenceNodes: Seq[String]): Unit = { + val deleteRequests = sequenceNodes.map { sequenceNode => + DeleteRequest(AclChangeNotificationSequenceZNode.deletePath(sequenceNode), ZkVersion.NoVersion) } val deleteResponses = retryRequestsUntilConnected(deleteRequests) @@ -841,7 +840,7 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends } /** - * Deletes th zk node recursively + * Deletes the zk node recursively * @param path * @return return true if it succeeds, false otherwise */ @@ -915,7 +914,6 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends zooKeeperClient.unregisterStateChangeHandler(name) } - /** * Close the underlying ZooKeeperClient. */ @@ -977,6 +975,12 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends createResponse } + /** + * Deletes the given zk path recursively + * @param path + * @return true if path gets deleted successfully, false if root path doesn't exists + * @throws KeeperException if there is an error while deleting the znodes + */ private[zk] def deleteRecursive(path: String): Boolean = { val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path)) if (getChildrenResponse.resultCode == Code.OK) { @@ -985,10 +989,11 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends if (deleteResponse.resultCode != Code.OK && deleteResponse.resultCode != Code.NONODE) { throw deleteResponse.resultException.get } - } else if (getChildrenResponse.resultCode != Code.NONODE) { + true + } else if (getChildrenResponse.resultCode == Code.NONODE) { + false + } else throw getChildrenResponse.resultException.get - } - true } private[zk] def pathExists(path: String): Boolean = { diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 230a64a86eb8a..64d87038622c7 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -22,10 +22,7 @@ import java.util.Properties import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch} -import kafka.common.TopicAndPartition -import kafka.controller.{IsrChangeNotificationListener, LeaderIsrAndControllerEpoch} -import kafka.security.auth.Acl -import kafka.security.auth.Resource +import kafka.security.auth.{Acl, Resource} import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls import kafka.utils.Json import org.apache.kafka.common.TopicPartition @@ -258,6 +255,10 @@ object ZkVersion { val NoVersion = -1 } +object StateChangeHandlers { + def controllerHandler = "controller-state-change-handler" +} + /** * The root acl storage node. Under this node there will be one child node per resource type (Topic, Cluster, Group). * under each resourceType there will be a unique child for each resource instance and the data for that child will contain @@ -290,8 +291,7 @@ object AclChangeNotificationZNode { object AclChangeNotificationSequenceZNode { val SequenceNumberPrefix = "acl_changes_" def createPath = s"${AclChangeNotificationZNode.path}/$SequenceNumberPrefix" - def path(sequenceNumber: String) = s"${AclChangeNotificationZNode.path}/$SequenceNumberPrefix$sequenceNumber" + def deletePath(sequenceNode: String) = s"${AclChangeNotificationZNode.path}/${sequenceNode}" def encode(resourceName : String): Array[Byte] = resourceName.getBytes(UTF_8) def decode(bytes: Array[Byte]): String = new String(bytes, UTF_8) - def sequenceNumber(path: String) = path.substring(path.lastIndexOf(SequenceNumberPrefix) + SequenceNumberPrefix.length) } \ No newline at end of file diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala index 389a56290a548..ef170597f94e3 100644 --- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala +++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala @@ -44,7 +44,6 @@ class ZooKeeperClient(connectString: String, connectionTimeoutMs: Int, maxInFlightRequests: Int) extends Logging { this.logIdent = "[ZooKeeperClient] " - private val initializationLock = new ReentrantReadWriteLock() private val isConnectedOrExpiredLock = new ReentrantLock() private val isConnectedOrExpiredCondition = isConnectedOrExpiredLock.newCondition() @@ -245,7 +244,7 @@ class ZooKeeperClient(connectString: String, * * @param name */ - def unregisterStateChangeHandler(name: String): Unit = { + def unregisterStateChangeHandler(name: String): Unit = inReadLock(initializationLock) { stateChangeHandlers.remove(name) } @@ -304,7 +303,7 @@ class ZooKeeperClient(connectString: String, isConnectedOrExpiredCondition.signalAll() } if (event.getState == KeeperState.AuthFailed) { - info("Auth failed.") + error("Auth failed.") stateChangeHandlers.foreach {case (name, handler) => handler.onAuthFailure()} } else if (event.getState == KeeperState.Expired) { inWriteLock(initializationLock) { diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala index a60dcc261dcfb..1e18f1d7bce93 100644 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -23,7 +23,7 @@ import kafka.network.RequestChannel.Session import kafka.security.auth.Acl.WildCardHost import kafka.server.KafkaConfig import kafka.utils.TestUtils -import kafka.zk.{AclChangeNotificationZNode, ZooKeeperTestHarness} +import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.security.auth.KafkaPrincipal import org.junit.Assert._ import org.junit.{After, Before, Test} diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index bc24de86121dd..347569dcb6e52 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -16,11 +16,8 @@ */ package kafka.zk -import kafka.server.Defaults -import kafka.zookeeper.ZooKeeperClient import java.util.UUID -import kafka.common.TopicAndPartition import kafka.security.auth._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.security.auth.KafkaPrincipal @@ -270,5 +267,15 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { assertFalse(zkClient.conditionalDelete(resource2, 10)) //delete with valid expected zk version assertTrue(zkClient.conditionalDelete(resource2, 0)) + + + zkClient.createAclChangeNotification("resource1") + zkClient.createAclChangeNotification("resource2") + + assertEquals(2, zkClient.getChildren(AclChangeNotificationZNode.path).size) + + zkClient.deleteAclChangeNotifications() + assertTrue(zkClient.getChildren(AclChangeNotificationZNode.path).isEmpty) + } } diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 877c896882739..03741ef732732 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -32,13 +32,15 @@ import scala.collection.JavaConverters._ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.clients.consumer.internals.AbstractCoordinator import kafka.controller.ControllerEventManager -import kafka.zookeeper.{StateChangeHandler, ZooKeeperClient} +import kafka.zookeeper.ZooKeeperClient @Category(Array(classOf[IntegrationTest])) abstract class ZooKeeperTestHarness extends JUnitSuite with Logging { val zkConnectionTimeout = 10000 val zkSessionTimeout = 6000 + val zkMaxInFlightRequests = Int.MaxValue + protected val zkAclsEnabled: Option[Boolean] = None var zkUtils: ZkUtils = null @@ -54,7 +56,7 @@ abstract class ZooKeeperTestHarness extends JUnitSuite with Logging { zookeeper = new EmbeddedZookeeper() zkUtils = ZkUtils(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled())) - zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout) + zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkMaxInFlightRequests) zkClient = new KafkaZkClient(zooKeeperClient, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled())) } diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index 5513438f2922c..ea6a47518a6ed 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.security.JaasUtils import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.{CreateMode, ZooDefs} import org.junit.Assert.{assertArrayEquals, assertEquals, assertTrue} -import org.junit.{After, Ignore, Test} +import org.junit.{After, Test} class ZooKeeperClientTest extends ZooKeeperTestHarness { private val mockPath = "/foo" @@ -315,7 +315,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { } } - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue,) + val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue) zooKeeperClient.registerStateChangeHandler(stateChangeHandler) zooKeeperClient.reinitialize() @@ -325,7 +325,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { @Test def testConnectionLossRequestTermination(): Unit = { val batchSize = 10 - val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, 2, null) + val zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, 2) zookeeper.shutdown() val requests = (1 to batchSize).map(i => GetDataRequest(s"/$i")) val countDownLatch = new CountDownLatch(1) From 34217cdea31e91bb2a4ffd96fb6db158e75d1ac7 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Wed, 8 Nov 2017 08:11:46 +0530 Subject: [PATCH 5/8] Address review comments --- .../scala/kafka/common/ZkNodeChangeNotificationListener.scala | 4 ++-- .../main/scala/kafka/security/auth/SimpleAclAuthorizer.scala | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- core/src/main/scala/kafka/zk/KafkaZkClient.scala | 1 - core/src/main/scala/kafka/zk/ZkData.scala | 3 ++- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala index 82ceaed068bdf..d10e448238a6e 100644 --- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala @@ -20,7 +20,7 @@ import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.atomic.AtomicBoolean import kafka.utils.{Logging, ShutdownableThread} -import kafka.zk.KafkaZkClient +import kafka.zk.{KafkaZkClient, StateChangeHandlers} import kafka.zookeeper.{StateChangeHandler, ZNodeChildChangeHandler} import org.apache.kafka.common.utils.Time @@ -142,7 +142,7 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, } object ZkStateChangeListener extends StateChangeHandler { - override val name: String = seqNodeRoot + override val name: String = StateChangeHandlers.zkNodeChangeListenerHandler(seqNodeRoot) override def afterInitializingSession(): Unit = addChangeNotification override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") } diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index eaebb88c0e183..aa25653cd20b1 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -256,7 +256,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { val newAcls = getNewAcls(currentVersionedAcls.acls) val (updateSucceeded, updateVersion) = if (newAcls.nonEmpty) { - zkClient.conditionalSetOrCreateAclsForResource(resource, newAcls, currentVersionedAcls.zkVersion) + zkClient.conditionalSetOrCreateAclsForResource(resource, newAcls, currentVersionedAcls.zkVersion) } else { trace(s"Deleting path for $resource because it had no ACLs remaining") (zkClient.conditionalDelete(resource, currentVersionedAcls.zkVersion), 0) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 3f23ed1b82035..4d816b8e74743 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -278,7 +278,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP ConfigType.Broker -> new BrokerConfigHandler(config, quotaManagers)) // Create the config manager. start listening to notifications - dynamicConfigManager = new DynamicConfigManager(zkUtils, zkClient, dynamicConfigHandlers) + dynamicConfigManager = new DynamicConfigManager(zkUtils, zkClient, dynamicConfigHandlers) dynamicConfigManager.startup() /* tell everyone we are alive */ diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index b6f62f5a8ec6f..97b7c98232d6f 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -738,7 +738,6 @@ class KafkaZkClient(zooKeeperClient: ZooKeeperClient, isSecure: Boolean) extends case Code.BADVERSION => (false, 0) case _ => - error(s"Error while updating node at $resource") throw setDataResponse.resultException.get } } diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 64d87038622c7..6a9872789f19d 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -256,7 +256,8 @@ object ZkVersion { } object StateChangeHandlers { - def controllerHandler = "controller-state-change-handler" + val controllerHandler = "controller-state-change-handler" + def zkNodeChangeListenerHandler(seqNodeRoot: String) = s"change-notification-$seqNodeRoot" } /** From 3ae9c01426de1dee70883c0d1080676c011302ea Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Wed, 8 Nov 2017 11:09:24 +0530 Subject: [PATCH 6/8] Address latest review comments --- .../kafka/common/ZkNodeChangeNotificationListener.scala | 6 +++--- core/src/main/scala/kafka/controller/KafkaController.scala | 2 +- core/src/main/scala/kafka/zk/ZkData.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala index d10e448238a6e..f5894304e409d 100644 --- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala @@ -57,7 +57,7 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, private val isClosed = new AtomicBoolean(false) def init() { - zkClient.registerStateChangeHandler(ZkStateChangeListener) + zkClient.registerStateChangeHandler(ZkStateChangeHandler) zkClient.registerZNodeChildChangeHandler(ChangeNotificationHandler) addChangeNotification() thread.start() @@ -65,7 +65,7 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, def close() = { isClosed.set(true) - zkClient.unregisterStateChangeHandler(ZkStateChangeListener.name) + zkClient.unregisterStateChangeHandler(ZkStateChangeHandler.name) zkClient.unregisterZNodeChildChangeHandler(ChangeNotificationHandler.path) queue.clear() thread.shutdown() @@ -141,7 +141,7 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, override def handleChildChange(): Unit = addChangeNotification } - object ZkStateChangeListener extends StateChangeHandler { + object ZkStateChangeHandler extends StateChangeHandler { override val name: String = StateChangeHandlers.zkNodeChangeListenerHandler(seqNodeRoot) override def afterInitializingSession(): Unit = addChangeNotification override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 4bb2f5aaef68c..aa5cf1fef0959 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -519,7 +519,7 @@ class KafkaController(val config: KafkaConfig, zkClient: KafkaZkClient, time: Ti */ def startup() = { zkClient.registerStateChangeHandler(new StateChangeHandler { - override val name: String = StateChangeHandlers.controllerHandler + override val name: String = StateChangeHandlers.ControllerHandler override def onReconnectionTimeout(): Unit = error("Reconnection timeout.") override def afterInitializingSession(): Unit = newSession() override def beforeInitializingSession(): Unit = expire() diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 6a9872789f19d..400f0c732a69e 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -256,7 +256,7 @@ object ZkVersion { } object StateChangeHandlers { - val controllerHandler = "controller-state-change-handler" + val ControllerHandler = "controller-state-change-handler" def zkNodeChangeListenerHandler(seqNodeRoot: String) = s"change-notification-$seqNodeRoot" } From 2c96fb3177ecc5490354383fead0a405763a8f7f Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Wed, 8 Nov 2017 15:25:20 +0530 Subject: [PATCH 7/8] add shutdown method to DynamicConfigManager --- core/src/main/scala/kafka/server/DynamicConfigManager.scala | 4 ++++ core/src/main/scala/kafka/server/KafkaServer.scala | 3 +++ 2 files changed, 7 insertions(+) diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index 3524889f85491..0f1abfc85e9b5 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -175,4 +175,8 @@ class DynamicConfigManager(private val oldZkUtils: ZkUtils, } } } + + def shutdown(): Unit = { + configChangeListener.close() + } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 4d816b8e74743..55367497d4d36 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -523,6 +523,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP if (kafkaHealthcheck != null) CoreUtils.swallow(kafkaHealthcheck.shutdown()) + if (dynamicConfigManager != null) + CoreUtils.swallow(dynamicConfigManager.shutdown()) + if (socketServer != null) CoreUtils.swallow(socketServer.shutdown()) if (requestHandlerPool != null) From 804c6e221ad0907fb2dccbd154b3beed55192e9f Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Wed, 8 Nov 2017 21:39:12 +0530 Subject: [PATCH 8/8] remove unused import --- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 55367497d4d36..df3666275ed1c 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -37,7 +37,7 @@ import kafka.security.CredentialProvider import kafka.security.auth.Authorizer import kafka.utils._ import kafka.zk.KafkaZkClient -import kafka.zookeeper.{StateChangeHandler, ZooKeeperClient} +import kafka.zookeeper.ZooKeeperClient import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient, NetworkClientUtils} import org.apache.kafka.common.internals.ClusterResourceListeners import org.apache.kafka.common.metrics.{JmxReporter, Metrics, _}