From 5af87e404b12c932cd9cd5f8785fc579a97ed8ea Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 24 Mar 2015 15:36:39 -0700 Subject: [PATCH 1/9] KAFKA:2035 added topic config cache. --- core/src/main/scala/kafka/server/KafkaServer.scala | 7 +- .../main/scala/kafka/server/TopicConfigCache.scala | 83 ++++++++++++++++++++++ .../scala/kafka/server/TopicConfigManager.scala | 2 + .../kafka/server/DynamicConfigChangeTest.scala | 13 ++++ .../unit/kafka/server/TopicConfigCacheTest.scala | 54 ++++++++++++++ 5 files changed, 158 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/kafka/server/TopicConfigCache.scala create mode 100644 core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index dddef93..4436bf9 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -61,6 +61,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var topicConfigManager: TopicConfigManager = null + var topicConfigCache: TopicConfigCache = null + var consumerCoordinator: ConsumerCoordinator = null var kafkaController: KafkaController = null @@ -152,8 +154,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Mx4jLoader.maybeLoad() + /*initialize topic config cache*/ + topicConfigCache = new TopicConfigCache(config.brokerId, zkClient, defaultConfig = config) + /* start topic config manager */ - topicConfigManager = new TopicConfigManager(zkClient, logManager) + topicConfigManager = new TopicConfigManager(zkClient, logManager, topicConfigCache) topicConfigManager.startup() /* tell everyone we are alive */ diff --git a/core/src/main/scala/kafka/server/TopicConfigCache.scala b/core/src/main/scala/kafka/server/TopicConfigCache.scala new file mode 100644 index 0000000..428a291 --- /dev/null +++ b/core/src/main/scala/kafka/server/TopicConfigCache.scala @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.concurrent.locks.ReentrantReadWriteLock + +import kafka.admin.AdminUtils +import kafka.log.LogConfig +import kafka.utils.Logging +import kafka.utils.Utils._ +import java.util.{Properties, Map} +import org.I0Itec.zkclient.ZkClient + +import scala.collection.{Set, mutable} + +/** + * A cache for topic configs that is maintained by each broker, this will not just return the overrides but also defaults. + */ +class TopicConfigCache(brokerId: Int, val zkClient: ZkClient, defaultConfig: KafkaConfig) extends Logging { + private val cache: mutable.Map[String, Properties] = new mutable.HashMap[String, Properties]() + private val lock = new ReentrantReadWriteLock() + + this.logIdent = "[Kafka Topic Config Cache on broker %d] ".format(brokerId) + + private def contains(topic: String) : Boolean = { + inReadLock(lock) { + return cache.contains(topic) + } + } + + /** + * Read the topic config from zookeeper and add it to cache. + * @param topic + */ + private def populateTopicConfig(topic: String): Unit = { + inWriteLock(lock) { + val topicConfig: Properties = defaultConfig.toProps + topicConfig.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) + addOrUpdateTopicConfig(topic, topicConfig) + } + } + + /** + * addOrUpdate the topic config cache. + * @param topic + * @param topicConfig + */ + def addOrUpdateTopicConfig(topic: String, topicConfig: Properties) { + inWriteLock(lock) { + cache.put(topic, topicConfig) + } + } + + /** + * returns the topic config, the config has overrides and defaults, if the topic config is not present in the cache + * it will be read from zookeeper and added to the cache. + * @param topic + * @return + */ + def getTopicConfig(topic: String): Properties = { + if(contains(topic)) { + return cache(topic) + } + + populateTopicConfig(topic) + return getTopicConfig(topic) + } +} diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala index 47295d4..529007a 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -60,6 +60,7 @@ import org.I0Itec.zkclient.{IZkChildListener, ZkClient} */ class TopicConfigManager(private val zkClient: ZkClient, private val logManager: LogManager, + private val topicConfigCache: TopicConfigCache, private val changeExpirationMs: Long = 15*60*1000, private val time: Time = SystemTime) extends Logging { private var lastExecutedChange = -1L @@ -103,6 +104,7 @@ class TopicConfigManager(private val zkClient: ZkClient, /* combine the default properties with the overrides in zk to create the new LogConfig */ val props = new Properties(logManager.defaultConfig.toProps) props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) + topicConfigCache.addOrUpdateTopicConfig(topic, props) val logConfig = LogConfig.fromProps(props) for (log <- logsByTopic(topic)) log.config = logConfig diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 93182ae..4043ea4 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -39,10 +39,23 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { val logOpt = this.servers(0).logManager.getLog(tp) assertTrue(logOpt.isDefined) assertEquals(oldVal, logOpt.get.config.flushInterval) + + //check config cache gets populated for a new topic. + val config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) + assertNotNull(config) + assertFalse(config.isEmpty) + assertEquals(oldVal, LogConfig.fromProps(config).flushInterval) } + AdminUtils.changeTopicConfig(zkClient, tp.topic, LogConfig(flushInterval = newVal).toProps) TestUtils.retry(10000) { assertEquals(newVal, this.servers(0).logManager.getLog(tp).get.config.flushInterval) + + //check config cache was updated with the new values. + val config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) + assertNotNull(config) + assertFalse(config.isEmpty) + assertEquals(newVal, LogConfig.fromProps(config).flushInterval) } } diff --git a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala new file mode 100644 index 0000000..8de3245 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package unit.kafka.server + +import java.util.Properties + +import kafka.admin.{AdminOperationException, AdminUtils} +import kafka.cluster.Broker +import kafka.common.TopicAndPartition +import kafka.integration.KafkaServerTestHarness +import kafka.log.LogConfig +import kafka.server.{TopicConfigCache, KafkaConfig, KafkaServer} +import kafka.utils.TestUtils +import junit.framework.Assert._ +import kafka.zk.ZooKeeperTestHarness +import org.scalatest.junit.JUnit3Suite + +class TopicConfigCacheTest extends JUnit3Suite with KafkaServerTestHarness { + + override val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, TestUtils.choosePort))) + + def testConfigCache { + var config = this.servers(0).topicConfigCache.getTopicConfig("not-existing-topic") + assertEquals("even for non existing topic we will return default config.",this.servers(0).config.toProps, config) + + //newly created topics should be populated in cache on first request. + val oldVal = 100000 + val tp = TopicAndPartition("test", 0) + AdminUtils.createTopic(zkClient, tp.topic, 1, 1, LogConfig(flushInterval = oldVal).toProps) + config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) + assertEquals(oldVal, LogConfig.fromProps(config).flushInterval) + + //test that addOrupdate works + val newVal = 20000 + config = LogConfig(flushInterval = newVal).toProps + this.servers(0).topicConfigCache.addOrUpdateTopicConfig(tp.topic, config) + config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) + assertEquals(newVal, LogConfig.fromProps(config).flushInterval) + } +} -- 1.9.5 (Apple Git-50.3) From 1f1d9eb8be228c6fc832347f12a2ca8a23b86850 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Wed, 25 Mar 2015 14:42:16 -0700 Subject: [PATCH 2/9] KAFKA-1688: initial check in. --- core/src/main/scala/kafka/admin/AdminUtils.scala | 11 ++- core/src/main/scala/kafka/api/TopicMetadata.scala | 42 ++++++++++- .../kafka/common/AuthorizationException.scala | 8 ++ .../src/main/scala/kafka/common/ErrorMapping.scala | 4 +- .../main/scala/kafka/network/RequestChannel.scala | 9 ++- core/src/main/scala/kafka/security/auth/Acl.scala | 73 ++++++++++++++++++ .../main/scala/kafka/security/auth/AclStore.scala | 21 ++++++ .../scala/kafka/security/auth/Authorizer.scala | 45 +++++++++++ .../main/scala/kafka/security/auth/Operation.java | 31 ++++++++ .../scala/kafka/security/auth/PermissionType.java | 22 ++++++ .../kafka/security/auth/SimpleAclAuthorizer.scala | 87 ++++++++++++++++++++++ core/src/main/scala/kafka/server/KafkaApis.scala | 82 ++++++++++++++++++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 8 ++ core/src/main/scala/kafka/server/KafkaServer.scala | 12 ++- .../main/scala/kafka/server/MetadataCache.scala | 2 +- .../kafka/server/KafkaConfigConfigDefTest.scala | 1 + 16 files changed, 437 insertions(+), 21 deletions(-) create mode 100644 core/src/main/scala/kafka/common/AuthorizationException.scala create mode 100644 core/src/main/scala/kafka/security/auth/Acl.scala create mode 100644 core/src/main/scala/kafka/security/auth/AclStore.scala create mode 100644 core/src/main/scala/kafka/security/auth/Authorizer.scala create mode 100644 core/src/main/scala/kafka/security/auth/Operation.java create mode 100644 core/src/main/scala/kafka/security/auth/PermissionType.java create mode 100644 core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index b700110..f091bc7 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -20,6 +20,7 @@ package kafka.admin import kafka.common._ import kafka.cluster.Broker import kafka.log.LogConfig +import kafka.security.auth.Acl import kafka.utils._ import kafka.api.{TopicMetadata, PartitionMetadata} @@ -225,7 +226,9 @@ object AdminUtils extends Logging { topic: String, partitions: Int, replicationFactor: Int, - topicConfig: Properties = new Properties) { + topicConfig: Properties = new Properties, + owner: String = Acl.wildCardPrincipal, + acls: Acl = Acl.allowAllAcl ) { val brokerList = ZkUtils.getSortedBrokerList(zkClient) val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig) @@ -235,9 +238,11 @@ object AdminUtils extends Logging { topic: String, partitionReplicaAssignment: Map[Int, Seq[Int]], config: Properties = new Properties, - update: Boolean = false) { + update: Boolean = false, + owner: String = Acl.wildCardPrincipal, + acls: Acl = Acl.allowAllAcl) { // validate arguments - Topic.validate(topic) + Topic.validate(topic) //TODO, we should probably ensure owner can not be changed when its an update request. LogConfig.validate(config) require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.") diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 0190076..2976c48 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -20,10 +20,13 @@ package kafka.api import kafka.cluster.Broker import java.nio.ByteBuffer import kafka.api.ApiUtils._ +import kafka.security.auth.Acl import kafka.utils.Logging import kafka.common._ import org.apache.kafka.common.utils.Utils._ +import scala.collection.immutable.HashSet + object TopicMetadata { val NoLeaderNodeId = -1 @@ -31,21 +34,34 @@ object TopicMetadata { def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): TopicMetadata = { val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue)) val topic = readShortString(buffer) + val numPartitions = readIntInRange(buffer, "number of partitions", (0, Int.MaxValue)) val partitionsMetadata: Array[PartitionMetadata] = new Array[PartitionMetadata](numPartitions) for(i <- 0 until numPartitions) { val partitionMetadata = PartitionMetadata.readFrom(buffer, brokers) partitionsMetadata(partitionMetadata.partitionId) = partitionMetadata } - new TopicMetadata(topic, partitionsMetadata, errorCode) + + val owner = readShortString(buffer) + + var acls: HashSet[Acl] = HashSet[Acl]() + val numAcls = readShortInRange(buffer, "number of acls", (0, Short.MaxValue)) + acls = new HashSet[Acl] + for(i <- 0 until numAcls) { + acls += Acl.readFrom(buffer) + } + + new TopicMetadata(topic, partitionsMetadata, errorCode, owner, acls) } } -case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError) extends Logging { +case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError, owner: String = null, acls: Set[Acl] = HashSet[Acl]()) extends Logging { def sizeInBytes: Int = { - 2 /* error code */ + + 2 /* error code */ + shortStringLength(topic) + - 4 + partitionsMetadata.map(_.sizeInBytes).sum /* size and partition data array */ + 4 + partitionsMetadata.map(_.sizeInBytes).sum + /* size and partition data array */ + shortStringLength(owner) + + 2 + acls.map(_.sizeInBytes).sum/* size and acl set */ } def writeTo(buffer: ByteBuffer) { @@ -56,11 +72,29 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat /* number of partitions */ buffer.putInt(partitionsMetadata.size) partitionsMetadata.foreach(m => m.writeTo(buffer)) + + /*owner*/ + writeShortString(buffer, owner) + + /* acls */ + buffer.putShort(acls.size.toShort) + acls.foreach(acl => acl.writeTo(buffer)) } override def toString(): String = { val topicMetadataInfo = new StringBuilder topicMetadataInfo.append("{TopicMetadata for topic %s -> ".format(topic)) + + if(owner != null && !owner.isEmpty) { + topicMetadataInfo.append(" with owner %s ->".format(owner)) + } + + if(acls != null && !acls.isEmpty) { + topicMetadataInfo.append(" with acls = [") + acls.foreach(acl => topicMetadataInfo.append("\n %s".format(acl.toString))) + topicMetadataInfo.append("] ") + } + errorCode match { case ErrorMapping.NoError => partitionsMetadata.foreach { partitionMetadata => diff --git a/core/src/main/scala/kafka/common/AuthorizationException.scala b/core/src/main/scala/kafka/common/AuthorizationException.scala new file mode 100644 index 0000000..3a714a3 --- /dev/null +++ b/core/src/main/scala/kafka/common/AuthorizationException.scala @@ -0,0 +1,8 @@ +package kafka.common + +/** + * Exception thrown when a principal is not authorized to perform an operation. + * @param message + */ +class AuthorizationException(message: String) extends RuntimeException(message) { +} diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index eb1eb4a..68d4eee 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -52,6 +52,7 @@ object ErrorMapping { // 21: InvalidRequiredAcks // 22: IllegalConsumerGeneration val NoOffsetsCommittedCode: Short = 23 + val AuthorizationCode: Short = 24; private val exceptionToCode = Map[Class[Throwable], Short]( @@ -74,7 +75,8 @@ object ErrorMapping { classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode, classOf[NotEnoughReplicasException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasCode, classOf[NotEnoughReplicasAfterAppendException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasAfterAppendCode, - classOf[NoOffsetsCommittedException].asInstanceOf[Class[Throwable]] -> NoOffsetsCommittedCode + classOf[NoOffsetsCommittedException].asInstanceOf[Class[Throwable]] -> NoOffsetsCommittedCode, + classOf[AuthorizationException].asInstanceOf[Class[Throwable]] -> NoOffsetsCommittedCode ).withDefaultValue(UnknownCode) /* invert the mapping */ diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 7b1db3d..e0f764e 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -17,6 +17,7 @@ package kafka.network +import java.security.Principal import java.util.concurrent._ import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge @@ -30,7 +31,7 @@ import org.apache.log4j.Logger object RequestChannel extends Logging { - val AllDone = new Request(1, 2, getShutdownReceive(), 0) + val AllDone = new Request(processor = 1, requestKey = 2, buffer = getShutdownReceive(), startTimeMs = 0) def getShutdownReceive() = { val emptyProducerRequest = new ProducerRequest(0, 0, "", 0, 0, collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]()) @@ -41,7 +42,7 @@ object RequestChannel extends Logging { byteBuffer } - case class Request(processor: Int, requestKey: Any, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0)) { + case class Request(processor: Int, requestKey: Any, session: Session = null, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0)) { @volatile var requestDequeueTimeMs = -1L @volatile var apiLocalCompleteTimeMs = -1L @volatile var responseCompleteTimeMs = -1L @@ -90,7 +91,9 @@ object RequestChannel extends Logging { } } } - + + case class Session(principal: Principal, host: String) + case class Response(processor: Int, request: Request, responseSend: Send, responseAction: ResponseAction) { request.responseCompleteTimeMs = SystemTime.milliseconds diff --git a/core/src/main/scala/kafka/security/auth/Acl.scala b/core/src/main/scala/kafka/security/auth/Acl.scala new file mode 100644 index 0000000..e5edf8d --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/Acl.scala @@ -0,0 +1,73 @@ +package kafka.security.auth + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ + +import scala.collection.immutable.HashSet + +object Acl { + + val wildCardPrincipal: String = "Anonymous" + val wildCardHost: String = "*" + val allowAllAcl = new Acl(wildCardPrincipal, PermissionType.ALLOW, Set[String](wildCardPrincipal), Set[Operation](Operation.ALL)) + + def readFrom(buffer: ByteBuffer): Acl = { + val principal= readShortString(buffer) + val permissionType = PermissionType.valueOf(readShortString(buffer)) + + val numHosts = readShortInRange(buffer, "number of hosts", (0, Short.MaxValue)) + var hosts = HashSet[String]() + for(i <- 0 until numHosts) { + hosts += readShortString(buffer) + } + + val numOfOperations = readShortInRange(buffer, "number of operations", (0, Short.MaxValue)) + var operations = HashSet[Operation]() + for(i <- 0 until numOfOperations) { + operations += Operation.valueOf(readShortString(buffer)) + } + + return new Acl(principal, permissionType, hosts, operations) + } +} + +/** + * An instance of this class will represent an acl that can express following statement. + *
+ * Principal P has permissionType PT on Operations READ,WRITE from hosts H1,H2.
+ * 
+ * @param principal A value of "Anonymous" indicates all users. + * @param permissionType + * @param hosts A value of * indicates all hosts. + * @param operations A value of ALL indicates all operations. + */ +case class Acl(principal: String, permissionType: PermissionType, hosts: Set[String], operations: Set[Operation]) { + + def shortOperationLength(op: Operation) : Int = { + shortStringLength(op.name()) + } + + def sizeInBytes: Int = { + shortStringLength(principal) + + shortStringLength(permissionType.name()) + + 2 + hosts.map(shortStringLength(_)).sum + + 2 + operations.map(shortOperationLength(_)).sum + } + + override def toString: String = "principal:" + principal + ",hosts:" + hosts+ ",operations:" + operations + + def writeTo(buffer: ByteBuffer) { + writeShortString(buffer, principal) + writeShortString(buffer, permissionType.name()) + + //hosts + buffer.putShort(hosts.size.toShort) + hosts.foreach(h => writeShortString(buffer, h)) + + //operations + buffer.putShort(operations.size.toShort) + operations.foreach(o => writeShortString(buffer, o.name())) + } +} + diff --git a/core/src/main/scala/kafka/security/auth/AclStore.scala b/core/src/main/scala/kafka/security/auth/AclStore.scala new file mode 100644 index 0000000..8bb245e --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/AclStore.scala @@ -0,0 +1,21 @@ +package kafka.security.auth + +import kafka.server.MetadataCache + + +class AclStore { + val metadataCache: MetadataCache = new MetadataCache(1); + + def getAcls(topic: String): Set[Acl] = { + return Set(Acl.allowAllAcl); + } + + def getOwner(topic: String): String = { + return Acl.wildCardPrincipal; + } + + def getClusterAcl(): Set[Acl] = { + return Set(Acl.allowAllAcl); + } + +} diff --git a/core/src/main/scala/kafka/security/auth/Authorizer.scala b/core/src/main/scala/kafka/security/auth/Authorizer.scala new file mode 100644 index 0000000..2410b2c --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/Authorizer.scala @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.security.auth + +import kafka.network.RequestChannel.Session +import kafka.server.{MetadataCache, KafkaConfig} + +/** + * Top level interface that all plugable authorizer must implement. Kafka server will read "authorizer.class" config + * value at startup time, create an instance of the specified class and call initialize method. + * authorizer.class must be a class that implements this interface. + * If authorizer.class has no value specified no authorization will be performed. + * + * From that point onwards, every client request will first be routed to authorize method and the request will only be + * authorized if the method returns true. + */ +trait Authorizer { + /** + * Guaranteed to be called before any authorize call is made. + */ + def initialize(kafkaConfig: KafkaConfig, metadataCache: MetadataCache): Unit + + /** + * @param session The session being authenticated. + * @param operation Type of operation client is trying to perform on resource. + * @param resource Resource the client is trying to access. + * @return + */ + def authorize(session: Session, operation: Operation, resource: String): Boolean +} diff --git a/core/src/main/scala/kafka/security/auth/Operation.java b/core/src/main/scala/kafka/security/auth/Operation.java new file mode 100644 index 0000000..ac1ca8d --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/Operation.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.security.auth; + +/** + * Different operations a client may perform on kafka resources. + */ +public enum Operation { + READ, + WRITE, + CREATE, + DELETE, + EDIT, + DESCRIBE, + REPLICATE, + ALL +} diff --git a/core/src/main/scala/kafka/security/auth/PermissionType.java b/core/src/main/scala/kafka/security/auth/PermissionType.java new file mode 100644 index 0000000..b844d41 --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/PermissionType.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.security.auth; + +public enum PermissionType { + ALLOW, + DENY +} diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala new file mode 100644 index 0000000..ac1150d --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -0,0 +1,87 @@ +package kafka.security.auth + +import java.net.InetAddress +import java.security.Principal + +import kafka.api.TopicMetadata +import kafka.common.{AuthorizationException, KafkaException} +import kafka.network.RequestChannel.Session +import kafka.server.{MetadataCache, KafkaConfig} +import kafka.utils.Logging + +import scala.collection.mutable.ListBuffer + +class SimpleAclAuthorizer extends Authorizer with Logging { + + val supportedOperations: Set[Operation] = Set[Operation](Operation.READ, Operation.WRITE, Operation.DESCRIBE, Operation.EDIT) + var aclStore: AclStore = null; + + override def authorize(session: Session, operation: Operation, resource: String): Boolean = { + //TODO can we assume session will never be null? + if(session == null || session.principal == null || session.host == null) { + warn("session, session.principal and session.host can not be null, failing authorization.") + return false + } + + if(!supportedOperations.contains(operation)) { + error("SimpleAclAuthorizer only supports " + supportedOperations + " but was invoked with operation = " + operation + + " for session = "+ session + " and resource = " + resource + ", failing authorization") + return false + } + + if(resource == null || resource.isEmpty) { + warn("SimpleAclAuthorizer only supports topic operations currently so resource can not be null or empty, failing authorization.") + return false + } + + val principalName: String = session.principal.getName + val remoteAddress: String = session.host + + //TODO super user check. + + val owner: String = aclStore.getOwner(topic = resource) + val acls: Set[Acl] = aclStore.getAcls(topic = resource) + + if(owner.equalsIgnoreCase(principalName)) { + debug("requesting principal = " + principalName + " is owner of the resource " + resource + ", allowing operation.") + return true + } + + if(acls.isEmpty) { + debug("No acl found.For backward compatibility when we find no acl we assume access to everyone , authorization failing open") + return true + } + + //first check if there is any Deny acl that would disallow this operation. + for(acl: Acl <- acls) { + if(acl.principal.equalsIgnoreCase(principalName) + && (acl.operations.contains(operation) || acl.operations.contains(Operation.ALL)) + && (acl.hosts.contains(remoteAddress) || acl.hosts.contains("*")) + && acl.permissionType.equals(PermissionType.DENY)) { + debug("denying operation = " + operation + " on resource = " + resource + " to session = " + session + " based on acl = " + acl) + return false + } + } + + //now check if there is any allow acl that will allow this operation. + for(acl: Acl <- acls) { + if(acl.principal.equalsIgnoreCase(principalName) + && (acl.operations.contains(operation) || acl.operations.contains(Operation.ALL)) + && (acl.hosts.contains(remoteAddress) || acl.hosts.contains("*"))) { + debug("allowing operation = " + operation + " on resource = " + resource + " to session = " + session + " based on acl = " + acl) + return true + } + } + + debug("principal = " + principalName + " is not allowed to perform operation = " + operation + + " from host = " + remoteAddress + " on resource = " + resource) + return false + } + + /** + * Guaranteed to be called before any authorize call is made. + */ + override def initialize(kafkaConfig: KafkaConfig, topicMetadataCache: MetadataCache): Unit = { + metadataCache = topicMetadataCache + } +} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 35af98f..297b925 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,6 +17,8 @@ package kafka.server +import kafka.message.{MessageSet} +import kafka.security.auth.{Operation, Authorizer} import org.apache.kafka.common.requests.JoinGroupResponse import org.apache.kafka.common.requests.HeartbeatResponse import org.apache.kafka.common.TopicPartition @@ -29,12 +31,14 @@ import kafka.coordinator.ConsumerCoordinator import kafka.log._ import kafka.network._ import kafka.network.RequestChannel.Response -import kafka.utils.{SystemTime, Logging} +import kafka.utils.{SystemTime, Logging, Utils} import scala.collection._ import org.I0Itec.zkclient.ZkClient +import scala.collection.immutable.HashMap + /** * Logic to handle the various Kafka requests */ @@ -46,7 +50,8 @@ class KafkaApis(val requestChannel: RequestChannel, val zkClient: ZkClient, val brokerId: Int, val config: KafkaConfig, - val metadataCache: MetadataCache) extends Logging { + val metadataCache: MetadataCache, + val authorizer: Authorizer) extends Logging { this.logIdent = "[KafkaApi-%d] ".format(brokerId) @@ -56,6 +61,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handle(request: RequestChannel.Request) { try{ trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) + request.requestId match { case RequestKeys.ProduceKey => handleProducerRequest(request) case RequestKeys.FetchKey => handleFetchRequest(request) @@ -109,6 +115,18 @@ class KafkaApis(val requestChannel: RequestChannel, def handleUpdateMetadataRequest(request: RequestChannel.Request) { val updateMetadataRequest = request.requestObj.asInstanceOf[UpdateMetadataRequest] + + if(authorizer != null) { + val unauthorizedTopicAndPartition = updateMetadataRequest.partitionStateInfos.filterKeys( + topicAndPartition => authorizer.authorize(request.session, Operation.EDIT, topicAndPartition.topic)).keys + //In this case the response does not allow to selectively report success/failure so if authorization fails, we fail the entire request. + if (unauthorizedTopicAndPartition != null && !unauthorizedTopicAndPartition.isEmpty) { + val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId, ErrorMapping.AuthorizationCode) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(updateMetadataResponse))) + return + } + } + replicaManager.maybeUpdateMetadataCache(updateMetadataRequest, metadataCache) val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId) @@ -164,10 +182,23 @@ class KafkaApis(val requestChannel: RequestChannel, def handleProducerRequest(request: RequestChannel.Request) { val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] + //filter topic partitions which does not pass authorization. + val unauthorizedResponseStatus = scala.collection.mutable.Map[TopicAndPartition, ProducerResponseStatus]() + if(authorizer != null) { + val unauthorizedTopicAndPartition = produceRequest.data.filterKeys( + topicAndPartition => authorizer.authorize(request.session, Operation.WRITE, topicAndPartition.topic)).keys + val unauthorizedResponse = ProducerResponseStatus(ErrorMapping.AuthorizationCode, -1) + for (topicAndPartition <- unauthorizedTopicAndPartition) { + unauthorizedResponseStatus.put(topicAndPartition, unauthorizedResponse) + } + } + // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { var errorInResponse = false - responseStatus.foreach { case (topicAndPartition, status) => + val mergedResponseStatus = responseStatus ++ unauthorizedResponseStatus + + mergedResponseStatus.foreach { case (topicAndPartition, status) => // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { @@ -190,7 +221,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.noOperation(request.processor, request) } } else { - val response = ProducerResponse(produceRequest.correlationId, responseStatus) + val response = ProducerResponse(produceRequest.correlationId, mergedResponseStatus) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } } @@ -219,9 +250,22 @@ class KafkaApis(val requestChannel: RequestChannel, def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] + //filter topic partitions which does not pass authorization. + val unauthorizedPartitionData = scala.collection.mutable.Map[TopicAndPartition, FetchResponsePartitionData]() + if(authorizer != null) { + val unauthorizedTopicAndPartition = fetchRequest.requestInfo.filterKeys( + topicAndPartition => authorizer.authorize(request.session, Operation.READ, topicAndPartition.topic)).keys + val unauthorizedResponse = FetchResponsePartitionData(ErrorMapping.AuthorizationCode, -1, MessageSet.Empty) + for (topicAndPartition <- unauthorizedTopicAndPartition) { + unauthorizedPartitionData.put(topicAndPartition, unauthorizedResponse) + } + } + // the callback for sending a fetch response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { - responsePartitionData.foreach { case (topicAndPartition, data) => + val mergedResponseStatus = responsePartitionData ++ unauthorizedPartitionData + + mergedResponseStatus.foreach { case (topicAndPartition, data) => // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here if (data.error != ErrorMapping.NoError && data.error != ErrorMapping.UnknownCode) { @@ -235,7 +279,7 @@ class KafkaApis(val requestChannel: RequestChannel, BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) } - val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) + val response = FetchResponse(fetchRequest.correlationId, mergedResponseStatus) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) } @@ -253,6 +297,18 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetRequest(request: RequestChannel.Request) { val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest] + + //filter topic partitions which does not pass authorization + val unauthorizedResponseMap = scala.collection.mutable.Map[TopicAndPartition, PartitionOffsetsResponse]() + if(authorizer != null) { + val unauthorizedTopicAndPartition = offsetRequest.requestInfo.filterKeys( + topicAndPartition => authorizer.authorize(request.session, Operation.DESCRIBE, topicAndPartition.topic)).keys + val unauthorizedResponse = PartitionOffsetsResponse(ErrorMapping.AuthorizationCode, Nil) + for (topicAndPartition <- unauthorizedTopicAndPartition) { + unauthorizedResponseMap.put(topicAndPartition, unauthorizedResponse) + } + } + val responseMap = offsetRequest.requestInfo.map(elem => { val (topicAndPartition, partitionOffsetRequestInfo) = elem try { @@ -293,7 +349,9 @@ class KafkaApis(val requestChannel: RequestChannel, (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil) ) } }) - val response = OffsetResponse(offsetRequest.correlationId, responseMap) + + val mergedResponseMap = responseMap ++ unauthorizedResponseMap + val response = OffsetResponse(offsetRequest.correlationId, mergedResponseMap) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -393,10 +451,18 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] + + //filter topics which does not pass authorization. + var unauthorizedTopicMetaData: Seq[TopicMetadata] = List[TopicMetadata]() + if(authorizer != null) { + val unauthorizedTopics = metadataRequest.topics.filter(topic => authorizer.authorize(request.session, Operation.DESCRIBE, topic)) + unauthorizedTopicMetaData = unauthorizedTopics.map(topic => new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.AuthorizationCode)) + } + val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) val brokers = metadataCache.getAliveBrokers trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) - val response = new TopicMetadataResponse(brokers, topicMetadata, metadataRequest.correlationId) + val response = new TopicMetadataResponse(brokers, topicMetadata ++ unauthorizedTopicMetaData, metadataRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 48e3362..ccf2f03 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -39,6 +39,7 @@ object Defaults { val NumIoThreads = 8 val BackgroundThreads = 10 val QueuedMaxRequests = 500 + val AuthorizerClassName = "" /** ********* Socket Server Configuration ***********/ val Port = 9092 @@ -139,6 +140,7 @@ object KafkaConfig { val NumIoThreadsProp = "num.io.threads" val BackgroundThreadsProp = "background.threads" val QueuedMaxRequestsProp = "queued.max.requests" + val AuthorizerClassNameProp = "authorizer.class.name" /** ********* Socket Server Configuration ***********/ val PortProp = "port" val HostNameProp = "host.name" @@ -243,6 +245,7 @@ object KafkaConfig { val NumIoThreadsDoc = "The number of io threads that the server uses for carrying out network requests" val BackgroundThreadsDoc = "The number of threads to use for various background processing tasks" val QueuedMaxRequestsDoc = "The number of queued requests allowed before blocking the network threads" + val AuthorizerClassNameDoc = "The authorizer class that should be used for authorization" /** ********* Socket Server Configuration ***********/ val PortDoc = "the port to listen and accept connections on" val HostNameDoc = "hostname of broker. If this is set, it will only bind to this address. If this is not set, it will bind to all interfaces" @@ -367,6 +370,7 @@ object KafkaConfig { .define(NumIoThreadsProp, INT, Defaults.NumIoThreads, atLeast(1), HIGH, NumIoThreadsDoc) .define(BackgroundThreadsProp, INT, Defaults.BackgroundThreads, atLeast(1), HIGH, BackgroundThreadsDoc) .define(QueuedMaxRequestsProp, INT, Defaults.QueuedMaxRequests, atLeast(1), HIGH, QueuedMaxRequestsDoc) + .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, LOW, AuthorizerClassNameDoc) /** ********* Socket Server Configuration ***********/ .define(PortProp, INT, Defaults.Port, HIGH, PortDoc) @@ -485,6 +489,7 @@ object KafkaConfig { numIoThreads = parsed.get(NumIoThreadsProp).asInstanceOf[Int], backgroundThreads = parsed.get(BackgroundThreadsProp).asInstanceOf[Int], queuedMaxRequests = parsed.get(QueuedMaxRequestsProp).asInstanceOf[Int], + authorizerClassName = parsed.get(AuthorizerClassNameProp).asInstanceOf[String], /** ********* Socket Server Configuration ***********/ port = parsed.get(PortProp).asInstanceOf[Int], @@ -623,6 +628,7 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val numIoThreads: Int = Defaults.NumIoThreads, val backgroundThreads: Int = Defaults.BackgroundThreads, val queuedMaxRequests: Int = Defaults.QueuedMaxRequests, + val authorizerClassName: String = Defaults.AuthorizerClassName, /** ********* Socket Server Configuration ***********/ val port: Int = Defaults.Port, @@ -792,6 +798,8 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ props.put(NumIoThreadsProp, numIoThreads.toString) props.put(BackgroundThreadsProp, backgroundThreads.toString) props.put(QueuedMaxRequestsProp, queuedMaxRequests.toString) + props.put(QueuedMaxRequestsProp, queuedMaxRequests.toString) + props.put(AuthorizerClassNameProp, authorizerClassName.toString) /** ********* Socket Server Configuration ***********/ props.put(PortProp, port.toString) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index dddef93..e8e36af 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -21,6 +21,7 @@ import kafka.admin._ import kafka.log.LogConfig import kafka.log.CleanerConfig import kafka.log.LogManager +import kafka.security.auth.Authorizer import kafka.utils._ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} @@ -144,9 +145,18 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg consumerCoordinator = new ConsumerCoordinator(config, zkClient) consumerCoordinator.startup() + /* Get the authorizer */ + val authorizer: Authorizer = if (config.authorizerClassName != null && !config.authorizerClassName.isEmpty) + Utils.createObject(config.authorizerClassName, classOf[Authorizer]) + else + null + if(authorizer != null) { + authorizer.initialize(config, metadataCache) + } + /* start processing requests */ apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, - kafkaController, zkClient, config.brokerId, config, metadataCache) + kafkaController, zkClient, config.brokerId, config, metadataCache, authorizer) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 6aef6e4..3a5b60d 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -31,7 +31,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock * A cache for the state (e.g., current leader) of each partition. This cache is updated through * UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously. */ -private[server] class MetadataCache(brokerId: Int) extends Logging { +private[kafka] class MetadataCache(brokerId: Int) extends Logging { private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] = new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]() private var aliveBrokers: Map[Int, Broker] = Map() diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index c124c8d..e1432ee 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -267,6 +267,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.AuthorizerClassNameProp => // ignore string case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.HostNameProp => // ignore string -- 1.9.5 (Apple Git-50.3) From d97f61a3b2e4c1dfa6cf4c9d204615293a20878d Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 7 Apr 2015 10:58:45 -0700 Subject: [PATCH 3/9] KAFKA-1688: Add authorization. --- core/src/main/scala/kafka/admin/AdminUtils.scala | 91 ++++++++--- core/src/main/scala/kafka/admin/TopicCommand.scala | 83 +++++++++-- core/src/main/scala/kafka/api/TopicMetadata.scala | 37 +---- core/src/main/scala/kafka/log/LogConfig.scala | 17 +++ core/src/main/scala/kafka/security/auth/Acl.scala | 105 +++++++------ .../main/scala/kafka/security/auth/AclStore.scala | 21 --- .../scala/kafka/security/auth/Authorizer.scala | 4 +- .../kafka/security/auth/ClusterAclCache.scala | 21 +++ .../main/scala/kafka/security/auth/Operation.java | 2 +- .../kafka/security/auth/SimpleAclAuthorizer.scala | 78 +++++----- core/src/main/scala/kafka/server/KafkaApis.scala | 123 ++++++++++++--- core/src/main/scala/kafka/server/KafkaConfig.scala | 31 ++++ core/src/main/scala/kafka/server/KafkaServer.scala | 10 +- core/src/main/scala/kafka/server/TopicConfig.scala | 42 ++++++ .../main/scala/kafka/server/TopicConfigCache.scala | 36 ++--- core/src/main/scala/kafka/utils/Json.scala | 2 +- core/src/test/resources/acl.json | 1 + .../scala/unit/kafka/admin/AddPartitionsTest.scala | 10 +- .../test/scala/unit/kafka/admin/AdminTest.scala | 22 +-- .../scala/unit/kafka/admin/DeleteTopicTest.scala | 10 +- .../scala/unit/kafka/admin/TopicCommandTest.scala | 27 +++- .../integration/UncleanLeaderElectionTest.scala | 13 +- .../scala/unit/kafka/security/auth/AclTest.scala | 24 +++ .../unit/kafka/security/auth/ClusterAclTest.scala | 22 +++ .../security/auth/SimpleAclAuthorizerTest.scala | 166 +++++++++++++++++++++ .../kafka/server/DynamicConfigChangeTest.scala | 12 +- .../kafka/server/KafkaConfigConfigDefTest.scala | 2 + .../unit/kafka/server/TopicConfigCacheTest.scala | 23 ++- .../test/scala/unit/kafka/utils/TestUtils.scala | 3 +- 29 files changed, 766 insertions(+), 272 deletions(-) delete mode 100644 core/src/main/scala/kafka/security/auth/AclStore.scala create mode 100644 core/src/main/scala/kafka/security/auth/ClusterAclCache.scala create mode 100644 core/src/main/scala/kafka/server/TopicConfig.scala create mode 100644 core/src/test/resources/acl.json create mode 100644 core/src/test/scala/unit/kafka/security/auth/AclTest.scala create mode 100644 core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala create mode 100644 core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index f091bc7..c7657fa 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -21,12 +21,12 @@ import kafka.common._ import kafka.cluster.Broker import kafka.log.LogConfig import kafka.security.auth.Acl +import kafka.server.TopicConfig import kafka.utils._ import kafka.api.{TopicMetadata, PartitionMetadata} import java.util.Random import java.util.Properties -import scala.Some import scala.Predef._ import scala.collection._ import mutable.ListBuffer @@ -109,7 +109,9 @@ object AdminUtils extends Logging { numPartitions: Int = 1, replicaAssignmentStr: String = "", checkBrokerAvailable: Boolean = true, - config: Properties = new Properties) { + config: Properties = new Properties, + owner: String, + acls: Option[Set[Acl]]) { val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) if (existingPartitionsReplicaList.size == 0) throw new AdminOperationException("The topic %s does not exist".format(topic)) @@ -136,7 +138,7 @@ object AdminUtils extends Logging { val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2) // add the new list partitionReplicaList ++= newPartitionReplicaList - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true, owner, acls) } def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = { @@ -227,11 +229,11 @@ object AdminUtils extends Logging { partitions: Int, replicationFactor: Int, topicConfig: Properties = new Properties, - owner: String = Acl.wildCardPrincipal, - acls: Acl = Acl.allowAllAcl ) { + owner: String = System.getProperty("user.name"), + acls: Set[Acl] = Set[Acl](Acl.allowAllAcl)) { val brokerList = ZkUtils.getSortedBrokerList(zkClient) val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig, update = false, owner, Some(acls)) } def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient, @@ -239,20 +241,21 @@ object AdminUtils extends Logging { partitionReplicaAssignment: Map[Int, Seq[Int]], config: Properties = new Properties, update: Boolean = false, - owner: String = Acl.wildCardPrincipal, - acls: Acl = Acl.allowAllAcl) { + owner: String, + acls: Option[Set[Acl]]) { // validate arguments - Topic.validate(topic) //TODO, we should probably ensure owner can not be changed when its an update request. + Topic.validate(topic) LogConfig.validate(config) + require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.") val topicPath = ZkUtils.getTopicPath(topic) if(!update && zkClient.exists(topicPath)) throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment)) - + // write out the config if there is any, this isn't transactional with the partition assignments - writeTopicConfig(zkClient, topic, config) + writeTopicConfig(zkClient, topic, config, owner, acls) // create the partition assignment writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) @@ -285,7 +288,7 @@ object AdminUtils extends Logging { * existing configs need to be deleted, it should be done prior to invoking this API * */ - def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties) { + def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties, owner: String, acls: Option[Set[Acl]]) { if(!topicExists(zkClient, topic)) throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic)) @@ -293,7 +296,7 @@ object AdminUtils extends Logging { LogConfig.validate(configs) // write the new config--may not exist if there were previously no overrides - writeTopicConfig(zkClient, topic, configs) + writeTopicConfig(zkClient, topic, configs, owner, acls) // create the change notification zkClient.createPersistentSequential(ZkUtils.TopicConfigChangesPath + "/" + TopicConfigChangeZnodePrefix, Json.encode(topic)) @@ -301,13 +304,26 @@ object AdminUtils extends Logging { /** * Write out the topic config to zk, if there is any + * TODO may we should just accept a TopicConfig instance here and call toProps on that, however LogConfig in topicConfig also has defaults + * we just want to store the overrides not the defaults for LogConfig. */ - private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties) { + private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties, owner: String, acls: Option[Set[Acl]]) { val configMap: mutable.Map[String, String] = { import JavaConversions._ config } - val map = Map("version" -> 1, "config" -> configMap) + + val aclMap: Map[String, Any] = acls match { + case Some(aclSet: Set[Acl]) => Acl.toJsonCompatibleMap(aclSet.toSet) + case _ => null + } + + //TODO: owner should first be read from jaas login module, if no logged in user is found only then we should default to user.name. + val map = Map(TopicConfig.versionKey -> 2, + TopicConfig.configKey -> configMap, + TopicConfig.ownerKey -> owner, + TopicConfig.aclKey -> aclMap) + ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map)) } @@ -316,25 +332,52 @@ object AdminUtils extends Logging { */ def fetchTopicConfig(zkClient: ZkClient, topic: String): Properties = { val str: String = zkClient.readData(ZkUtils.getTopicConfigPath(topic), true) - val props = new Properties() + var props = new Properties() if(str != null) { Json.parseFull(str) match { case None => // there are no config overrides case Some(map: Map[String, _]) => - require(map("version") == 1) - map.get("config") match { - case Some(config: Map[String, String]) => - for((k,v) <- config) - props.setProperty(k, v) - case _ => throw new IllegalArgumentException("Invalid topic config: " + str) - } - + if (map(TopicConfig.versionKey) == 1) + props = toTopicConfigV1(map, str) + else + props = toTopicConfigV2(map, str) case o => throw new IllegalArgumentException("Unexpected value in config: " + str) } } props } + def toTopicConfigV1(map: Map[String, Any], config: String): Properties = { + val props = new Properties() + map.get(TopicConfig.configKey) match { + case Some(config: Map[String, String]) => + for((k,v) <- config) + props.setProperty(k, v) + case _ => throw new IllegalArgumentException("Invalid topic config: " + config) + } + props + } + + def toTopicConfigV2(map: Map[String, Any], config: String): Properties = { + val props = toTopicConfigV1(map, config) + + props.setProperty(TopicConfig.versionKey, "2") + map.get(TopicConfig.aclKey) match { + case Some(acls: Map[String, Any]) => + props.setProperty(TopicConfig.aclKey, Json.encode(acls)) + case Some(null) => + case _ => throw new IllegalArgumentException("Invalid topic config: " + config) + } + + map.get(TopicConfig.ownerKey) match { + case Some(owner: String) => + props.setProperty(TopicConfig.ownerKey, owner) + case Some(null) => + case _ => throw new IllegalArgumentException("Invalid topic config: " + config) + } + props + } + def fetchAllTopicConfigs(zkClient: ZkClient): Map[String, Properties] = ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchTopicConfig(zkClient, topic))).toMap diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index f400b71..0ff7456 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -17,9 +17,12 @@ package kafka.admin +import java.nio.file.{Paths, Files} + import joptsimple._ import java.util.Properties import kafka.common.AdminCommandFailedException +import kafka.security.auth.Acl import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException @@ -28,7 +31,7 @@ import scala.collection.JavaConversions._ import kafka.cluster.Broker import kafka.log.LogConfig import kafka.consumer.Whitelist -import kafka.server.OffsetManager +import kafka.server.{TopicConfig, OffsetManager} import org.apache.kafka.common.utils.Utils.formatAddress @@ -83,14 +86,20 @@ object TopicCommand { def createTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topic = opts.options.valueOf(opts.topicOpt) val configs = parseTopicConfigsToBeAdded(opts) + val acls: Set[Acl] = parseAcl(opts) + val owner: String = if (opts.options.has(opts.ownerOpt)) + opts.options.valueOf(opts.ownerOpt).trim + else + System.getProperty("user.name") + if (opts.options.has(opts.replicaAssignmentOpt)) { val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs, owner = owner, acls = Some(acls)) } else { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt) val partitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicas = opts.options.valueOf(opts.replicationFactorOpt).intValue - AdminUtils.createTopic(zkClient, topic, partitions, replicas, configs) + AdminUtils.createTopic(zkClient, topic, partitions, replicas, configs, acls = acls) } println("Created topic \"%s\".".format(topic)) } @@ -100,17 +109,38 @@ object TopicCommand { if (topics.length == 0) { println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt))) } + topics.foreach { topic => - val configs = AdminUtils.fetchTopicConfig(zkClient, topic) - if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) { + var configs = AdminUtils.fetchTopicConfig(zkClient, topic) + var topicConfigs: TopicConfig = TopicConfig.fromProps(configs) + //for backward compatibility remove all configs that are not LogConfig properties. + configs = LogConfig.getFilteredProps(configs) + + if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt) || opts.options.has(opts.aclOpt) || opts.options.has(opts.ownerOpt)) { val configsToBeAdded = parseTopicConfigsToBeAdded(opts) val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts) + val acls: Set[Acl] = if (opts.options.has(opts.aclOpt)) + parseAcl(opts) + else + topicConfigs.acls + val owner: String = if (opts.options.has(opts.ownerOpt)) + opts.options.valueOf(opts.ownerOpt).trim + else + topicConfigs.owner + // compile the final set of configs configs.putAll(configsToBeAdded) configsToBeDeleted.foreach(config => configs.remove(config)) - AdminUtils.changeTopicConfig(zkClient, topic, configs) + AdminUtils.changeTopicConfig(zkClient, topic, configs, owner, Some(acls)) println("Updated config for topic \"%s\".".format(topic)) } + + //reload the config from zookeeper as it might have been just updated. + configs = AdminUtils.fetchTopicConfig(zkClient, topic) + topicConfigs = TopicConfig.fromProps(configs) + //for backward compatibility remove all configs that are not LogConfig properties. + configs = LogConfig.getFilteredProps(configs) + if(opts.options.has(opts.partitionsOpt)) { if (topic == OffsetManager.OffsetsTopicName) { throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.") @@ -119,7 +149,7 @@ object TopicCommand { "logic or ordering of the messages will be affected") val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt) - AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs) + AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs, owner = topicConfigs.owner, acls = Some(topicConfigs.acls)) println("Adding partitions succeeded!") } } @@ -167,13 +197,20 @@ object TopicCommand { val describeConfigs: Boolean = !reportUnavailablePartitions && !reportUnderReplicatedPartitions val describePartitions: Boolean = !reportOverriddenConfigs val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) + if (describeConfigs) { val configs = AdminUtils.fetchTopicConfig(zkClient, topic) + val topicConfig: TopicConfig = TopicConfig.fromProps(configs) + val logConfigs = LogConfig.getFilteredProps(configs) + if (!reportOverriddenConfigs || configs.size() != 0) { val numPartitions = topicPartitionAssignment.size val replicationFactor = topicPartitionAssignment.head._2.size - println("Topic:%s\tPartitionCount:%d\tReplicationFactor:%d\tConfigs:%s" - .format(topic, numPartitions, replicationFactor, configs.map(kv => kv._1 + "=" + kv._2).mkString(","))) + println("Topic:%s\tOwner:%s\tPartitionCount:%d\tReplicationFactor:%d\tConfigs:%s" + .format(topic, topicConfig.owner, numPartitions, replicationFactor, + logConfigs.map(kv => kv._1 + "=" + kv._2).mkString(","))) + println("Acls:") + topicConfig.acls.foreach(acl => println(acl)) } } if (describePartitions) { @@ -198,7 +235,20 @@ object TopicCommand { } def formatBroker(broker: Broker) = broker.id + " (" + formatAddress(broker.host, broker.port) + ")" - + + def parseAcl(opts: TopicCommandOptions): Set[Acl] = { + if (opts.options.has(opts.aclOpt)) { + val aclJsonFilePath = opts.options.valueOf(opts.aclOpt).trim + val source = scala.io.Source.fromFile(aclJsonFilePath) + val jsonAcls = source.mkString + source.close() + //validate acls can be parsed + return Acl.fromJson(jsonAcls) + } + + return Set[Acl](Acl.allowAllAcl) + } + def parseTopicConfigsToBeAdded(opts: TopicCommandOptions): Properties = { val configsToBeAdded = opts.options.valuesOf(opts.configOpt).map(_.split("""\s*=\s*""")) require(configsToBeAdded.forall(config => config.length == 2), @@ -286,6 +336,17 @@ object TopicCommand { val topicsWithOverridesOpt = parser.accepts("topics-with-overrides", "if set when describing topics, only show topics that have overridden configs") + val aclOpt = parser.accepts("acl", "Path to the acl json file that describes the acls for the topic. This is not additive, i.e. the new acls will overwrite any old acls.") + .withRequiredArg() + .describedAs("Path to topic acl json file") + .ofType(classOf[String]) + + //TODO: Should we even allow users to specify someone else as owner? or just default to keytab logged in user or the user executing command if no jaas login has happened.? + val ownerOpt = parser.accepts("owner", "User name for the owner of the topic. Default : user running the command.") + .withRequiredArg() + .describedAs("User name for the owner of the topic.") + .ofType(classOf[String]) + val options = parser.parse(args : _*) val allTopicLevelOpts: Set[OptionSpec[_]] = Set(alterOpt, createOpt, describeOpt, listOpt) @@ -298,6 +359,8 @@ object TopicCommand { // check invalid args CommandLineUtils.checkInvalidArgs(parser, options, configOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, aclOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, ownerOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, deleteConfigOpt, allTopicLevelOpts -- Set(alterOpt)) CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, allTopicLevelOpts -- Set(createOpt)) diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 2976c48..5292e60 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -20,13 +20,10 @@ package kafka.api import kafka.cluster.Broker import java.nio.ByteBuffer import kafka.api.ApiUtils._ -import kafka.security.auth.Acl import kafka.utils.Logging import kafka.common._ import org.apache.kafka.common.utils.Utils._ -import scala.collection.immutable.HashSet - object TopicMetadata { val NoLeaderNodeId = -1 @@ -42,26 +39,15 @@ object TopicMetadata { partitionsMetadata(partitionMetadata.partitionId) = partitionMetadata } - val owner = readShortString(buffer) - - var acls: HashSet[Acl] = HashSet[Acl]() - val numAcls = readShortInRange(buffer, "number of acls", (0, Short.MaxValue)) - acls = new HashSet[Acl] - for(i <- 0 until numAcls) { - acls += Acl.readFrom(buffer) - } - - new TopicMetadata(topic, partitionsMetadata, errorCode, owner, acls) + new TopicMetadata(topic, partitionsMetadata, errorCode) } } -case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError, owner: String = null, acls: Set[Acl] = HashSet[Acl]()) extends Logging { +case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError) extends Logging { def sizeInBytes: Int = { 2 /* error code */ + shortStringLength(topic) + - 4 + partitionsMetadata.map(_.sizeInBytes).sum + /* size and partition data array */ - shortStringLength(owner) + - 2 + acls.map(_.sizeInBytes).sum/* size and acl set */ + 4 + partitionsMetadata.map(_.sizeInBytes).sum /* size and partition data array */ } def writeTo(buffer: ByteBuffer) { @@ -72,29 +58,12 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat /* number of partitions */ buffer.putInt(partitionsMetadata.size) partitionsMetadata.foreach(m => m.writeTo(buffer)) - - /*owner*/ - writeShortString(buffer, owner) - - /* acls */ - buffer.putShort(acls.size.toShort) - acls.foreach(acl => acl.writeTo(buffer)) } override def toString(): String = { val topicMetadataInfo = new StringBuilder topicMetadataInfo.append("{TopicMetadata for topic %s -> ".format(topic)) - if(owner != null && !owner.isEmpty) { - topicMetadataInfo.append(" with owner %s ->".format(owner)) - } - - if(acls != null && !acls.isEmpty) { - topicMetadataInfo.append(" with acls = [") - acls.foreach(acl => topicMetadataInfo.append("\n %s".format(acl.toString))) - topicMetadataInfo.append("] ") - } - errorCode match { case ErrorMapping.NoError => partitionsMetadata.foreach { partitionMetadata => diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 8b67aee..70a9030 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -245,4 +245,21 @@ object LogConfig { configDef.parse(props) } + /** + * Removes any properties that are not defined in configDef + * @param props + * @return + */ + def getFilteredProps(props: Properties): Properties = { + val filteredProps: Properties = new Properties() + val names = configDef.names() + + for((k,v) <- props) { + if(names.contains(k)) { + filteredProps.put(k, v) + } + } + filteredProps + } + } diff --git a/core/src/main/scala/kafka/security/auth/Acl.scala b/core/src/main/scala/kafka/security/auth/Acl.scala index e5edf8d..1a7960b 100644 --- a/core/src/main/scala/kafka/security/auth/Acl.scala +++ b/core/src/main/scala/kafka/security/auth/Acl.scala @@ -1,73 +1,82 @@ package kafka.security.auth -import java.nio.ByteBuffer +import kafka.utils.Json -import kafka.api.ApiUtils._ - -import scala.collection.immutable.HashSet +import scala.collection.{mutable} object Acl { - - val wildCardPrincipal: String = "Anonymous" + val wildCardPrincipal: String = "*" val wildCardHost: String = "*" - val allowAllAcl = new Acl(wildCardPrincipal, PermissionType.ALLOW, Set[String](wildCardPrincipal), Set[Operation](Operation.ALL)) - - def readFrom(buffer: ByteBuffer): Acl = { - val principal= readShortString(buffer) - val permissionType = PermissionType.valueOf(readShortString(buffer)) - - val numHosts = readShortInRange(buffer, "number of hosts", (0, Short.MaxValue)) - var hosts = HashSet[String]() - for(i <- 0 until numHosts) { - hosts += readShortString(buffer) + val allowAllAcl = new Acl(wildCardPrincipal, PermissionType.ALLOW, Set[String](wildCardHost), Set[Operation](Operation.ALL)) + val PRINCIPAL_KEY = "principal" + val PERMISSION_TYPE_KEY = "permissionType" + val OPERATIONS_KEY = "operations" + val HOSTS_KEY = "hosts" + val VERSION_KEY = "version" + val CURRENT_VERSION = 1 + val ACLS_KEY = "acls" + + def fromJson(aclJson: String): Set[Acl] = { + if(aclJson == null || aclJson.isEmpty) { + return collection.immutable.Set.empty[Acl] } - - val numOfOperations = readShortInRange(buffer, "number of operations", (0, Short.MaxValue)) - var operations = HashSet[Operation]() - for(i <- 0 until numOfOperations) { - operations += Operation.valueOf(readShortString(buffer)) + var acls: mutable.HashSet[Acl] = new mutable.HashSet[Acl]() + Json.parseFull(aclJson) match { + case Some(m) => + val aclMap = m.asInstanceOf[Map[String, Any]] + //the acl json version. + require(aclMap.get(VERSION_KEY).get == CURRENT_VERSION) + val aclSet: List[Map[String, Any]] = aclMap.get(ACLS_KEY).get.asInstanceOf[List[Map[String, Any]]] + aclSet.foreach(item => { + val principal: String = item(PRINCIPAL_KEY).asInstanceOf[String] + val permissionType: PermissionType = PermissionType.valueOf(item(PERMISSION_TYPE_KEY).asInstanceOf[String]) + val operations: List[Operation] = item(OPERATIONS_KEY).asInstanceOf[List[String]].map(operation => Operation.valueOf(operation)) + val hosts: List[String] = item(HOSTS_KEY).asInstanceOf[List[String]] + acls += new Acl(principal, permissionType, hosts.toSet, operations.toSet) + }) + case None => } + return acls.toSet + } - return new Acl(principal, permissionType, hosts, operations) + def toJsonCompatibleMap(acls: Set[Acl]): Map[String,Any] = { + acls match { + case aclSet: Set[Acl] => Map(Acl.VERSION_KEY -> Acl.CURRENT_VERSION, Acl.ACLS_KEY -> aclSet.map(acl => acl.toMap).toList) + case _ => null + } } } /** * An instance of this class will represent an acl that can express following statement. *
- * Principal P has permissionType PT on Operations READ,WRITE from hosts H1,H2.
+ * Principal P has permissionType PT on Operations O1,O2 from hosts H1,H2.
  * 
- * @param principal A value of "Anonymous" indicates all users. + * @param principal A value of * indicates all users. * @param permissionType * @param hosts A value of * indicates all hosts. * @param operations A value of ALL indicates all operations. */ -case class Acl(principal: String, permissionType: PermissionType, hosts: Set[String], operations: Set[Operation]) { - - def shortOperationLength(op: Operation) : Int = { - shortStringLength(op.name()) - } - - def sizeInBytes: Int = { - shortStringLength(principal) + - shortStringLength(permissionType.name()) + - 2 + hosts.map(shortStringLength(_)).sum + - 2 + operations.map(shortOperationLength(_)).sum +case class Acl(val principal: String,val permissionType: PermissionType,val hosts: Set[String],val operations: Set[Operation]) { + + /** + * TODO: Ideally we would have a symmetric toJson method but our current json library fails to decode double parsed json strings so + * convert to map which then gets converted to json. + * Convert an acl instance to a map + * @return Map representation of the Acl. + */ + def toMap() : Map[String, Any] = { + val map: mutable.HashMap[String, Any] = new mutable.HashMap[String, Any]() + map.put(Acl.PRINCIPAL_KEY, principal) + map.put(Acl.PERMISSION_TYPE_KEY, permissionType.name()) + map.put(Acl.OPERATIONS_KEY, operations.map(operation => operation.name())) + map.put(Acl.HOSTS_KEY, hosts) + + map.toMap } - override def toString: String = "principal:" + principal + ",hosts:" + hosts+ ",operations:" + operations - - def writeTo(buffer: ByteBuffer) { - writeShortString(buffer, principal) - writeShortString(buffer, permissionType.name()) - - //hosts - buffer.putShort(hosts.size.toShort) - hosts.foreach(h => writeShortString(buffer, h)) - - //operations - buffer.putShort(operations.size.toShort) - operations.foreach(o => writeShortString(buffer, o.name())) + override def toString() : String = { + return "%s has %s permission for operations: %s from hosts: %s".format(principal, permissionType.name(), operations.mkString(","), hosts.mkString(",")) } } diff --git a/core/src/main/scala/kafka/security/auth/AclStore.scala b/core/src/main/scala/kafka/security/auth/AclStore.scala deleted file mode 100644 index 8bb245e..0000000 --- a/core/src/main/scala/kafka/security/auth/AclStore.scala +++ /dev/null @@ -1,21 +0,0 @@ -package kafka.security.auth - -import kafka.server.MetadataCache - - -class AclStore { - val metadataCache: MetadataCache = new MetadataCache(1); - - def getAcls(topic: String): Set[Acl] = { - return Set(Acl.allowAllAcl); - } - - def getOwner(topic: String): String = { - return Acl.wildCardPrincipal; - } - - def getClusterAcl(): Set[Acl] = { - return Set(Acl.allowAllAcl); - } - -} diff --git a/core/src/main/scala/kafka/security/auth/Authorizer.scala b/core/src/main/scala/kafka/security/auth/Authorizer.scala index 2410b2c..5a4a7da 100644 --- a/core/src/main/scala/kafka/security/auth/Authorizer.scala +++ b/core/src/main/scala/kafka/security/auth/Authorizer.scala @@ -18,7 +18,7 @@ package kafka.security.auth import kafka.network.RequestChannel.Session -import kafka.server.{MetadataCache, KafkaConfig} +import kafka.server.{TopicConfigCache, MetadataCache, KafkaConfig} /** * Top level interface that all plugable authorizer must implement. Kafka server will read "authorizer.class" config @@ -33,7 +33,7 @@ trait Authorizer { /** * Guaranteed to be called before any authorize call is made. */ - def initialize(kafkaConfig: KafkaConfig, metadataCache: MetadataCache): Unit + def initialize(kafkaConfig: KafkaConfig, topicConfigCache: TopicConfigCache): Unit /** * @param session The session being authenticated. diff --git a/core/src/main/scala/kafka/security/auth/ClusterAclCache.scala b/core/src/main/scala/kafka/security/auth/ClusterAclCache.scala new file mode 100644 index 0000000..d03b362 --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/ClusterAclCache.scala @@ -0,0 +1,21 @@ +package kafka.security.auth + +import java.nio.file.{Files, Paths} + + +/** + * @param clusterAclFilePath local file path to the json file that describes cluster acls. + */ +class ClusterAclCache(clusterAclFilePath: String) { + + val clusterAcl: Set[Acl] = { + if(clusterAclFilePath != null && !clusterAclFilePath.isEmpty && Files.exists(Paths.get(clusterAclFilePath))) { + val source = scala.io.Source.fromFile(clusterAclFilePath) + val jsonAcls = source.mkString + source.close() + Acl.fromJson(jsonAcls) + } else { + collection.immutable.Set.empty[Acl] + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/security/auth/Operation.java b/core/src/main/scala/kafka/security/auth/Operation.java index ac1ca8d..cac5b85 100644 --- a/core/src/main/scala/kafka/security/auth/Operation.java +++ b/core/src/main/scala/kafka/security/auth/Operation.java @@ -26,6 +26,6 @@ public enum Operation { DELETE, EDIT, DESCRIBE, - REPLICATE, + SEND_CONTROL_MSG, ALL } diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index ac1150d..a8f87e1 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -1,63 +1,63 @@ package kafka.security.auth -import java.net.InetAddress -import java.security.Principal - -import kafka.api.TopicMetadata -import kafka.common.{AuthorizationException, KafkaException} import kafka.network.RequestChannel.Session -import kafka.server.{MetadataCache, KafkaConfig} +import kafka.server.{KafkaConfig, TopicConfigCache} import kafka.utils.Logging -import scala.collection.mutable.ListBuffer - class SimpleAclAuthorizer extends Authorizer with Logging { - val supportedOperations: Set[Operation] = Set[Operation](Operation.READ, Operation.WRITE, Operation.DESCRIBE, Operation.EDIT) - var aclStore: AclStore = null; + val topicOperations: Set[Operation] = Set[Operation](Operation.READ, Operation.WRITE, Operation.DESCRIBE, Operation.EDIT) + val supportedOperations: Set[Operation] = topicOperations ++ Set[Operation](Operation.SEND_CONTROL_MSG, Operation.CREATE, Operation.DELETE) + var clusterAclCache: ClusterAclCache = null + var configCache: TopicConfigCache = null + var superUsers: Set[String] = null override def authorize(session: Session, operation: Operation, resource: String): Boolean = { - //TODO can we assume session will never be null? + //can we assume session, principal and host will never be null? if(session == null || session.principal == null || session.host == null) { - warn("session, session.principal and session.host can not be null, failing authorization.") - return false + debug("session, session.principal and session.host can not be null, , programming error so failing open.") + return true } if(!supportedOperations.contains(operation)) { - error("SimpleAclAuthorizer only supports " + supportedOperations + " but was invoked with operation = " + operation - + " for session = "+ session + " and resource = " + resource + ", failing authorization") - return false - } - - if(resource == null || resource.isEmpty) { - warn("SimpleAclAuthorizer only supports topic operations currently so resource can not be null or empty, failing authorization.") - return false + debug("SimpleAclAuthorizer only supports " + supportedOperations + " but was invoked with operation = " + operation + + " for session = "+ session + " and resource = " + resource + ", programming error so failing open.") + return true } val principalName: String = session.principal.getName val remoteAddress: String = session.host - //TODO super user check. + if(superUsers.contains(principalName)) { + debug("principal = " + principalName + " is a super user, allowing operation without checking acls.") + return true + } + + if(topicOperations.contains(operation) && (resource == null || resource.isEmpty)){ + debug("resource is null or empty for a topic operation " + operation + " for session = "+ session + ", " + + "programming error so failing open.") + return true + } - val owner: String = aclStore.getOwner(topic = resource) - val acls: Set[Acl] = aclStore.getAcls(topic = resource) + val owner: String = if(topicOperations.contains(operation)) configCache.getTopicConfig(resource).owner else null + val acls: Set[Acl] = if(topicOperations.contains(operation)) configCache.getTopicConfig(resource).acls else clusterAclCache.clusterAcl - if(owner.equalsIgnoreCase(principalName)) { - debug("requesting principal = " + principalName + " is owner of the resource " + resource + ", allowing operation.") + if(principalName.equalsIgnoreCase(owner)) { + debug("principal = " + principalName + " is owner of the resource " + resource + ", allowing operation without checking acls.") return true } - if(acls.isEmpty) { - debug("No acl found.For backward compatibility when we find no acl we assume access to everyone , authorization failing open") + if(acls == null || acls.isEmpty) { + debug("No acl found. For backward compatibility when we find no acl we assume access to everyone , authorization failing open.") return true } //first check if there is any Deny acl that would disallow this operation. for(acl: Acl <- acls) { - if(acl.principal.equalsIgnoreCase(principalName) + if(acl.permissionType.equals(PermissionType.DENY) + && (acl.principal.equalsIgnoreCase(principalName) || acl.principal.equalsIgnoreCase(Acl.wildCardPrincipal)) && (acl.operations.contains(operation) || acl.operations.contains(Operation.ALL)) - && (acl.hosts.contains(remoteAddress) || acl.hosts.contains("*")) - && acl.permissionType.equals(PermissionType.DENY)) { + && (acl.hosts.contains(remoteAddress) || acl.hosts.contains(Acl.wildCardHost))) { debug("denying operation = " + operation + " on resource = " + resource + " to session = " + session + " based on acl = " + acl) return false } @@ -65,14 +65,16 @@ class SimpleAclAuthorizer extends Authorizer with Logging { //now check if there is any allow acl that will allow this operation. for(acl: Acl <- acls) { - if(acl.principal.equalsIgnoreCase(principalName) + if(acl.permissionType.equals(PermissionType.ALLOW) + && (acl.principal.equalsIgnoreCase(principalName) || acl.principal.equalsIgnoreCase(Acl.wildCardPrincipal)) && (acl.operations.contains(operation) || acl.operations.contains(Operation.ALL)) - && (acl.hosts.contains(remoteAddress) || acl.hosts.contains("*"))) { + && (acl.hosts.contains(remoteAddress) || acl.hosts.contains(Acl.wildCardHost))) { debug("allowing operation = " + operation + " on resource = " + resource + " to session = " + session + " based on acl = " + acl) return true } } + //We have some acls defined and they do not specify any allow ACL for the current session, reject request. debug("principal = " + principalName + " is not allowed to perform operation = " + operation + " from host = " + remoteAddress + " on resource = " + resource) return false @@ -81,7 +83,13 @@ class SimpleAclAuthorizer extends Authorizer with Logging { /** * Guaranteed to be called before any authorize call is made. */ - override def initialize(kafkaConfig: KafkaConfig, topicMetadataCache: MetadataCache): Unit = { - metadataCache = topicMetadataCache + override def initialize(kafkaConfig: KafkaConfig, topicConfigCache: TopicConfigCache): Unit = { + clusterAclCache = new ClusterAclCache(kafkaConfig.clusterAclJsonFilePath) + superUsers = kafkaConfig.superUser match { + case null => Set.empty[String] + case (str: String) => str.split(",").map(s => s.trim).toSet + case _ => throw new IllegalArgumentException("expected a comma seperated list of superusers , found:" + kafkaConfig.superUser) + } + configCache = topicConfigCache } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 297b925..407c227 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -31,14 +31,12 @@ import kafka.coordinator.ConsumerCoordinator import kafka.log._ import kafka.network._ import kafka.network.RequestChannel.Response -import kafka.utils.{SystemTime, Logging, Utils} +import kafka.utils.{SystemTime, Logging} import scala.collection._ import org.I0Itec.zkclient.ZkClient -import scala.collection.immutable.HashMap - /** * Logic to handle the various Kafka requests */ @@ -91,6 +89,13 @@ class KafkaApis(val requestChannel: RequestChannel, // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest] + + if(authorizer != null && !authorizer.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, Map.empty, ErrorMapping.AuthorizationCode) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse))) + return + } + try { val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest, offsetManager) val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error) @@ -107,6 +112,13 @@ class KafkaApis(val requestChannel: RequestChannel, // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest] + + if(authorizer != null && !authorizer.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, Map.empty, ErrorMapping.AuthorizationCode) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse))) + return + } + val (response, error) = replicaManager.stopReplicas(stopReplicaRequest) val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, response.toMap, error) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse))) @@ -118,7 +130,7 @@ class KafkaApis(val requestChannel: RequestChannel, if(authorizer != null) { val unauthorizedTopicAndPartition = updateMetadataRequest.partitionStateInfos.filterKeys( - topicAndPartition => authorizer.authorize(request.session, Operation.EDIT, topicAndPartition.topic)).keys + topicAndPartition => !authorizer.authorize(request.session, Operation.EDIT, topicAndPartition.topic)).keys //In this case the response does not allow to selectively report success/failure so if authorization fails, we fail the entire request. if (unauthorizedTopicAndPartition != null && !unauthorizedTopicAndPartition.isEmpty) { val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId, ErrorMapping.AuthorizationCode) @@ -138,6 +150,15 @@ class KafkaApis(val requestChannel: RequestChannel, // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted val controlledShutdownRequest = request.requestObj.asInstanceOf[ControlledShutdownRequest] + + if(authorizer != null) { + if (!authorizer.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId, ErrorMapping.AuthorizationCode, Set.empty) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) + return + } + } + val partitionsRemaining = controller.shutdownBroker(controlledShutdownRequest.brokerId) val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId, ErrorMapping.NoError, partitionsRemaining) @@ -150,10 +171,22 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetCommitRequest(request: RequestChannel.Request) { val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] + var requestInfo = offsetCommitRequest.requestInfo + + val unAuthorizedResponse = scala.collection.mutable.Map[TopicAndPartition, Short]() + if(authorizer != null) { + val unauthorizedTopicAndPartition = requestInfo.filterKeys( + topicAndPartition => !authorizer.authorize(request.session, Operation.READ, topicAndPartition.topic)).keys + for (topicAndPartition <- unauthorizedTopicAndPartition) { + unAuthorizedResponse.put(topicAndPartition, ErrorMapping.AuthorizationCode) + requestInfo -= topicAndPartition + } + } // the callback for sending an offset commit response def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { - commitStatus.foreach { case (topicAndPartition, errorCode) => + val mergedCommitStatus = commitStatus ++ unAuthorizedResponse + mergedCommitStatus.foreach { case (topicAndPartition, errorCode) => // we only print warnings for known errors here; only replica manager could see an unknown // exception while trying to write the offset message to the local log, and it will log // an error message and write the error code in this case; hence it can be ignored here @@ -164,7 +197,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - val response = OffsetCommitResponse(commitStatus, offsetCommitRequest.correlationId) + val response = OffsetCommitResponse(mergedCommitStatus, offsetCommitRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } // call offset manager to store offsets @@ -172,7 +205,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetCommitRequest.groupId, offsetCommitRequest.consumerId, offsetCommitRequest.groupGenerationId, - offsetCommitRequest.requestInfo, + requestInfo, sendResponseCallback) } @@ -186,10 +219,12 @@ class KafkaApis(val requestChannel: RequestChannel, val unauthorizedResponseStatus = scala.collection.mutable.Map[TopicAndPartition, ProducerResponseStatus]() if(authorizer != null) { val unauthorizedTopicAndPartition = produceRequest.data.filterKeys( - topicAndPartition => authorizer.authorize(request.session, Operation.WRITE, topicAndPartition.topic)).keys + topicAndPartition => !authorizer.authorize(request.session, Operation.WRITE, topicAndPartition.topic)).keys val unauthorizedResponse = ProducerResponseStatus(ErrorMapping.AuthorizationCode, -1) for (topicAndPartition <- unauthorizedTopicAndPartition) { unauthorizedResponseStatus.put(topicAndPartition, unauthorizedResponse) + //remove unauthorized topics from the original request. + produceRequest.data.remove(topicAndPartition) } } @@ -249,15 +284,17 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] + var requestInfo: Map[TopicAndPartition, PartitionFetchInfo] = fetchRequest.requestInfo //filter topic partitions which does not pass authorization. val unauthorizedPartitionData = scala.collection.mutable.Map[TopicAndPartition, FetchResponsePartitionData]() if(authorizer != null) { val unauthorizedTopicAndPartition = fetchRequest.requestInfo.filterKeys( - topicAndPartition => authorizer.authorize(request.session, Operation.READ, topicAndPartition.topic)).keys + topicAndPartition => !authorizer.authorize(request.session, Operation.READ, topicAndPartition.topic)).keys val unauthorizedResponse = FetchResponsePartitionData(ErrorMapping.AuthorizationCode, -1, MessageSet.Empty) for (topicAndPartition <- unauthorizedTopicAndPartition) { unauthorizedPartitionData.put(topicAndPartition, unauthorizedResponse) + requestInfo -= topicAndPartition } } @@ -288,7 +325,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest.maxWait.toLong, fetchRequest.replicaId, fetchRequest.minBytes, - fetchRequest.requestInfo, + requestInfo, sendResponseCallback) } @@ -297,19 +334,21 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetRequest(request: RequestChannel.Request) { val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest] + var requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo] = offsetRequest.requestInfo //filter topic partitions which does not pass authorization val unauthorizedResponseMap = scala.collection.mutable.Map[TopicAndPartition, PartitionOffsetsResponse]() if(authorizer != null) { val unauthorizedTopicAndPartition = offsetRequest.requestInfo.filterKeys( - topicAndPartition => authorizer.authorize(request.session, Operation.DESCRIBE, topicAndPartition.topic)).keys + topicAndPartition => !authorizer.authorize(request.session, Operation.DESCRIBE, topicAndPartition.topic)).keys val unauthorizedResponse = PartitionOffsetsResponse(ErrorMapping.AuthorizationCode, Nil) for (topicAndPartition <- unauthorizedTopicAndPartition) { unauthorizedResponseMap.put(topicAndPartition, unauthorizedResponse) + requestInfo -= topicAndPartition } } - val responseMap = offsetRequest.requestInfo.map(elem => { + val responseMap = requestInfo.map(elem => { val (topicAndPartition, partitionOffsetRequestInfo) = elem try { // ensure leader exists @@ -351,7 +390,7 @@ class KafkaApis(val requestChannel: RequestChannel, }) val mergedResponseMap = responseMap ++ unauthorizedResponseMap - val response = OffsetResponse(offsetRequest.correlationId, mergedResponseMap) + val response = OffsetResponse(offsetRequest.correlationId, mergedResponseMap.toMap) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -451,15 +490,19 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] + var topics = metadataRequest.topics.toSet //filter topics which does not pass authorization. var unauthorizedTopicMetaData: Seq[TopicMetadata] = List[TopicMetadata]() if(authorizer != null) { - val unauthorizedTopics = metadataRequest.topics.filter(topic => authorizer.authorize(request.session, Operation.DESCRIBE, topic)) + val unauthorizedTopics = metadataRequest.topics.filter(topic => !authorizer.authorize(request.session, Operation.DESCRIBE, topic)) unauthorizedTopicMetaData = unauthorizedTopics.map(topic => new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.AuthorizationCode)) + for(topic <- unauthorizedTopics) { + topics -= topic + } } - val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) + val topicMetadata = getTopicMetadata(topics) val brokers = metadataCache.getAliveBrokers trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) val response = new TopicMetadataResponse(brokers, topicMetadata ++ unauthorizedTopicMetaData, metadataRequest.correlationId) @@ -472,7 +515,20 @@ class KafkaApis(val requestChannel: RequestChannel, def handleOffsetFetchRequest(request: RequestChannel.Request) { val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] - val (unknownTopicPartitions, knownTopicPartitions) = offsetFetchRequest.requestInfo.partition(topicAndPartition => + val (authorizedTopicPartitions, unauthorizedTopicPartitions) = offsetFetchRequest.requestInfo.partition( + topicAndPartition => { + if(authorizer != null) { + authorizer.authorize(request.session, Operation.DESCRIBE, topicAndPartition.topic) + } else { + true + } + } + ) + + val authorizationError = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.AuthorizationCode) + val unauthorizedStatus = unauthorizedTopicPartitions.map(topicAndPartition => (topicAndPartition, authorizationError)).toMap + + val (unknownTopicPartitions, knownTopicPartitions) = authorizedTopicPartitions.partition(topicAndPartition => metadataCache.getPartitionInfo(topicAndPartition.topic, topicAndPartition.partition).isEmpty ) val unknownStatus = unknownTopicPartitions.map(topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition)).toMap @@ -481,7 +537,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetManager.getOffsets(offsetFetchRequest.groupId, knownTopicPartitions).toMap else Map.empty[TopicAndPartition, OffsetMetadataAndError] - val status = unknownStatus ++ knownStatus + val status = unknownStatus ++ knownStatus ++ unauthorizedStatus val response = OffsetFetchResponse(status, offsetFetchRequest.correlationId) @@ -498,6 +554,13 @@ class KafkaApis(val requestChannel: RequestChannel, val partition = offsetManager.partitionFor(consumerMetadataRequest.group) + //TODO: this can in turn create the topic, so we should check the create permissions if the config is enabled and topic is non existent. + if (authorizer != null && !authorizer.authorize(request.session, Operation.DESCRIBE, OffsetManager.OffsetsTopicName)) { + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(errorResponse))) + return + } + // get metadata (and create the topic if necessary) val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName)).head @@ -520,10 +583,22 @@ class KafkaApis(val requestChannel: RequestChannel, val joinGroupRequest = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader] + val (authorizedTopics, unauthorizedTopics) = joinGroupRequest.body.topics().partition( + topic => { + if(authorizer!= null) + authorizer.authorize(request.session, Operation.DESCRIBE, topic) + else + true + } + ) + + val unauthorizedTopicPartition = unauthorizedTopics.map(topic => new TopicPartition(topic, -1)) + // the callback for sending a join-group response def sendResponseCallback(partitions: List[TopicAndPartition], generationId: Int, errorCode: Short) { - val partitionList = partitions.map(tp => new TopicPartition(tp.topic, tp.partition)).toBuffer - val responseBody = new JoinGroupResponse(errorCode, generationId, joinGroupRequest.body.consumerId, partitionList) + val partitionList = (partitions.map(tp => new TopicPartition(tp.topic, tp.partition)) ++ unauthorizedTopicPartition).toBuffer + val error = if (errorCode == ErrorMapping.NoError && !unauthorizedTopicPartition.isEmpty) ErrorMapping.AuthorizationCode else errorCode + val responseBody = new JoinGroupResponse(error, generationId, joinGroupRequest.body.consumerId, partitionList) val response = new JoinGroupResponseAndHeader(joinGroupRequest.correlationId, responseBody) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -532,7 +607,7 @@ class KafkaApis(val requestChannel: RequestChannel, coordinator.consumerJoinGroup( joinGroupRequest.body.groupId(), joinGroupRequest.body.consumerId(), - joinGroupRequest.body.topics().toList, + authorizedTopics.toList, joinGroupRequest.body.sessionTimeout(), joinGroupRequest.body.strategy(), sendResponseCallback) @@ -541,6 +616,14 @@ class KafkaApis(val requestChannel: RequestChannel, def handleHeartbeatRequest(request: RequestChannel.Request) { val heartbeatRequest = request.requestObj.asInstanceOf[HeartbeatRequestAndHeader] + if(authorizer != null) { + if (!authorizer.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + val heartbeatResponseAndHeader = new HeartbeatResponseAndHeader(heartbeatRequest.correlationId, new HeartbeatResponse(ErrorMapping.AuthorizationCode)) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(heartbeatResponseAndHeader))) + return + } + } + // the callback for sending a heartbeat response def sendResponseCallback(errorCode: Short) { val response = new HeartbeatResponseAndHeader(heartbeatRequest.correlationId, new HeartbeatResponse(errorCode)) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index e9ce37a..dcdfbfd 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -39,7 +39,11 @@ object Defaults { val NumIoThreads = 8 val BackgroundThreads = 10 val QueuedMaxRequests = 500 + + /************* Authorizer Configuration ***********/ val AuthorizerClassName = "" + val SuperUser = "" + val ClusterAclJsonFilePath = "" /** ********* Socket Server Configuration ***********/ val Port = 9092 @@ -141,7 +145,12 @@ object KafkaConfig { val NumIoThreadsProp = "num.io.threads" val BackgroundThreadsProp = "background.threads" val QueuedMaxRequestsProp = "queued.max.requests" + + /************* Authorizer Configuration ***********/ val AuthorizerClassNameProp = "authorizer.class.name" + val SuperUserProp = "super.users" + val ClusterAclJsonFilePathProp = "cluster.acl.json.file.path" + /** ********* Socket Server Configuration ***********/ val PortProp = "port" val HostNameProp = "host.name" @@ -247,7 +256,13 @@ object KafkaConfig { val NumIoThreadsDoc = "The number of io threads that the server uses for carrying out network requests" val BackgroundThreadsDoc = "The number of threads to use for various background processing tasks" val QueuedMaxRequestsDoc = "The number of queued requests allowed before blocking the network threads" + + /************* Authorizer Configuration ***********/ val AuthorizerClassNameDoc = "The authorizer class that should be used for authorization" + val SuperUserDoc = "Comman seperated list of users that will have super user access to the cluster and all the topics." + val ClusterAclJsonFilePathDoc = "Path to the json file describing cluster's acl. These acls are used to determine which users" + + "have access to cluster actions like CREATE topic." + /** ********* Socket Server Configuration ***********/ val PortDoc = "the port to listen and accept connections on" val HostNameDoc = "hostname of broker. If this is set, it will only bind to this address. If this is not set, it will bind to all interfaces" @@ -373,7 +388,11 @@ object KafkaConfig { .define(NumIoThreadsProp, INT, Defaults.NumIoThreads, atLeast(1), HIGH, NumIoThreadsDoc) .define(BackgroundThreadsProp, INT, Defaults.BackgroundThreads, atLeast(1), HIGH, BackgroundThreadsDoc) .define(QueuedMaxRequestsProp, INT, Defaults.QueuedMaxRequests, atLeast(1), HIGH, QueuedMaxRequestsDoc) + + /************* Authorizer Configuration ***********/ .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, LOW, AuthorizerClassNameDoc) + .define(SuperUserProp, STRING, Defaults.SuperUser, LOW, SuperUserDoc) + .define(ClusterAclJsonFilePathProp, STRING, Defaults.ClusterAclJsonFilePath, LOW, ClusterAclJsonFilePathDoc) /** ********* Socket Server Configuration ***********/ .define(PortProp, INT, Defaults.Port, HIGH, PortDoc) @@ -493,7 +512,11 @@ object KafkaConfig { numIoThreads = parsed.get(NumIoThreadsProp).asInstanceOf[Int], backgroundThreads = parsed.get(BackgroundThreadsProp).asInstanceOf[Int], queuedMaxRequests = parsed.get(QueuedMaxRequestsProp).asInstanceOf[Int], + + /************* Authorizer Configuration ***********/ authorizerClassName = parsed.get(AuthorizerClassNameProp).asInstanceOf[String], + superUser = parsed.get(SuperUserProp).asInstanceOf[String], + clusterAclJsonFilePath = parsed.get(ClusterAclJsonFilePathProp).asInstanceOf[String], /** ********* Socket Server Configuration ***********/ port = parsed.get(PortProp).asInstanceOf[Int], @@ -633,7 +656,11 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val numIoThreads: Int = Defaults.NumIoThreads, val backgroundThreads: Int = Defaults.BackgroundThreads, val queuedMaxRequests: Int = Defaults.QueuedMaxRequests, + + /************* Authorizer Configuration ***********/ val authorizerClassName: String = Defaults.AuthorizerClassName, + val superUser: String = Defaults.SuperUser, + val clusterAclJsonFilePath: String = Defaults.ClusterAclJsonFilePath, /** ********* Socket Server Configuration ***********/ val port: Int = Defaults.Port, @@ -805,7 +832,11 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ props.put(BackgroundThreadsProp, backgroundThreads.toString) props.put(QueuedMaxRequestsProp, queuedMaxRequests.toString) props.put(QueuedMaxRequestsProp, queuedMaxRequests.toString) + + /************* Authorizer Configuration ***********/ props.put(AuthorizerClassNameProp, authorizerClassName.toString) + props.put(SuperUserProp, superUser.toString) + props.put(ClusterAclJsonFilePathProp, clusterAclJsonFilePath.toString) /** ********* Socket Server Configuration ***********/ props.put(PortProp, port.toString) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 03df5f6..bd19e4e 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -147,13 +147,16 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg consumerCoordinator = new ConsumerCoordinator(config, zkClient) consumerCoordinator.startup() + /*initialize topic config cache*/ + topicConfigCache = new TopicConfigCache(config.brokerId, zkClient, defaultConfig = config) + /* Get the authorizer */ val authorizer: Authorizer = if (config.authorizerClassName != null && !config.authorizerClassName.isEmpty) - Utils.createObject(config.authorizerClassName, classOf[Authorizer]) + Utils.createObject(config.authorizerClassName) else null if(authorizer != null) { - authorizer.initialize(config, metadataCache) + authorizer.initialize(config, topicConfigCache) } /* start processing requests */ @@ -164,9 +167,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Mx4jLoader.maybeLoad() - /*initialize topic config cache*/ - topicConfigCache = new TopicConfigCache(config.brokerId, zkClient, defaultConfig = config) - /* start topic config manager */ topicConfigManager = new TopicConfigManager(zkClient, logManager, topicConfigCache) topicConfigManager.startup() diff --git a/core/src/main/scala/kafka/server/TopicConfig.scala b/core/src/main/scala/kafka/server/TopicConfig.scala new file mode 100644 index 0000000..ee85d22 --- /dev/null +++ b/core/src/main/scala/kafka/server/TopicConfig.scala @@ -0,0 +1,42 @@ +package kafka.server + +import java.util.Properties + +import kafka.log.LogConfig +import kafka.security.auth.Acl +import kafka.utils.Json + +object TopicConfig { + val versionKey = "version" + val configKey = "config" + val aclKey = "acls" + val ownerKey = "owner" + + def fromProps(properties: Properties) : TopicConfig = { + fromProps(new Properties(), properties) + } + + def fromProps(defaultProperties: Properties, overrideProperties: Properties) : TopicConfig = { + val version: Int = Option(overrideProperties.getProperty(versionKey)) match { + case Some(version: String) => version.toInt + case None => 1 + } + val owner: String = overrideProperties.getProperty(ownerKey) + val logConfig: LogConfig = LogConfig.fromProps(defaultProperties, overrideProperties) + val acls: Set[Acl] = Acl.fromJson(overrideProperties.getProperty(aclKey)) + + new TopicConfig(version, owner, logConfig, acls, overrideProperties) + } +} + +class TopicConfig(val version: Int, val owner: String,val logConfig: LogConfig,val acls: Set[Acl], val overrideProperties: Properties) { + def toProps(): Properties = { + val properties: Properties = new Properties() + properties.put(TopicConfig.ownerKey, owner) + properties.put(TopicConfig.aclKey, Json.encode(acls.map(acl => acl.toMap()).toList)) + properties.putAll(logConfig.toProps) + properties.putAll(overrideProperties) + + properties + } +} diff --git a/core/src/main/scala/kafka/server/TopicConfigCache.scala b/core/src/main/scala/kafka/server/TopicConfigCache.scala index 428a291..d1901a5 100644 --- a/core/src/main/scala/kafka/server/TopicConfigCache.scala +++ b/core/src/main/scala/kafka/server/TopicConfigCache.scala @@ -17,67 +17,61 @@ package kafka.server +import java.util.Properties import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.admin.AdminUtils -import kafka.log.LogConfig import kafka.utils.Logging import kafka.utils.Utils._ -import java.util.{Properties, Map} import org.I0Itec.zkclient.ZkClient -import scala.collection.{Set, mutable} +import scala.collection.mutable /** * A cache for topic configs that is maintained by each broker, this will not just return the overrides but also defaults. */ class TopicConfigCache(brokerId: Int, val zkClient: ZkClient, defaultConfig: KafkaConfig) extends Logging { - private val cache: mutable.Map[String, Properties] = new mutable.HashMap[String, Properties]() + private val cache: mutable.Map[String, TopicConfig] = new mutable.HashMap[String, TopicConfig]() private val lock = new ReentrantReadWriteLock() this.logIdent = "[Kafka Topic Config Cache on broker %d] ".format(brokerId) - private def contains(topic: String) : Boolean = { - inReadLock(lock) { - return cache.contains(topic) - } - } - /** * Read the topic config from zookeeper and add it to cache. * @param topic */ private def populateTopicConfig(topic: String): Unit = { inWriteLock(lock) { - val topicConfig: Properties = defaultConfig.toProps - topicConfig.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) - addOrUpdateTopicConfig(topic, topicConfig) + val overrideProperties: Properties = AdminUtils.fetchTopicConfig(zkClient, topic) + addOrUpdateTopicConfig(topic, overrideProperties) } } /** * addOrUpdate the topic config cache. * @param topic - * @param topicConfig + * @param overrideProperties */ - def addOrUpdateTopicConfig(topic: String, topicConfig: Properties) { + def addOrUpdateTopicConfig(topic: String, overrideProperties: Properties) { inWriteLock(lock) { - cache.put(topic, topicConfig) + cache.put(topic, TopicConfig.fromProps(defaultConfig.toProps, overrideProperties)) } } /** - * returns the topic config, the config has overrides and defaults, if the topic config is not present in the cache - * it will be read from zookeeper and added to the cache. + * Returns the topic config. * @param topic * @return */ - def getTopicConfig(topic: String): Properties = { - if(contains(topic)) { - return cache(topic) + def getTopicConfig(topic: String): TopicConfig = { + inReadLock(lock) { + if(cache.contains(topic)) { + return cache(topic) + } } populateTopicConfig(topic) + return getTopicConfig(topic) } } diff --git a/core/src/main/scala/kafka/utils/Json.scala b/core/src/main/scala/kafka/utils/Json.scala index d110284..ae7acab 100644 --- a/core/src/main/scala/kafka/utils/Json.scala +++ b/core/src/main/scala/kafka/utils/Json.scala @@ -53,7 +53,7 @@ object Json extends Logging { obj match { case null => "null" case b: Boolean => b.toString - case s: String => "\"" + s + "\"" + case s: String => "\"" + s.replace("\"","\\\"") + "\"" case n: Number => n.toString case m: Map[_, _] => "{" + diff --git a/core/src/test/resources/acl.json b/core/src/test/resources/acl.json new file mode 100644 index 0000000..fd9b916 --- /dev/null +++ b/core/src/test/resources/acl.json @@ -0,0 +1 @@ +{"version": 1, "acls": [{"hosts":["host1","host2"],"permissionType":"DENY","operations":["READ","WRITE"],"principal":"alice"},{"hosts":["*"],"permissionType":"ALLOW","operations":["READ","WRITE"],"principal":"bob"},{"hosts":["host1","host2"],"permissionType":"DENY","operations":["READ"],"principal":"bob"}]} diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 8bc1785..4bc94b9 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -78,7 +78,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testTopicDoesNotExist { try { - AdminUtils.addPartitions(zkClient, "Blah", 1) + AdminUtils.addPartitions(zkClient, "Blah", 1, owner = null ,acls = None) fail("Topic should not exist") } catch { case e: AdminOperationException => //this is good @@ -88,7 +88,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testWrongReplicaCount { try { - AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2") + AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2", owner = null ,acls = None) fail("Add partitions should fail") } catch { case e: AdminOperationException => //this is good @@ -97,7 +97,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testIncrementPartitions { - AdminUtils.addPartitions(zkClient, topic1, 3) + AdminUtils.addPartitions(zkClient, topic1, 3, owner = null ,acls = None) // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2) @@ -122,7 +122,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testManualAssignmentOfReplicas { - AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3") + AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3", owner = null ,acls = None) // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2) @@ -148,7 +148,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testReplicaPlacement { - AdminUtils.addPartitions(zkClient, topic3, 7) + AdminUtils.addPartitions(zkClient, topic3, 7, owner = null ,acls = None) // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 1) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index ee0b21e..510b981 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -71,18 +71,18 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // duplicate brokers intercept[IllegalArgumentException] { - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,0))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,0)), owner = null ,acls = None) } // inconsistent replication factor intercept[IllegalArgumentException] { - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,1), 1->Seq(0))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,1), 1->Seq(0)), owner = null ,acls = None) } // good assignment val assignment = Map(0 -> List(0, 1, 2), 1 -> List(1, 2, 3)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", assignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", assignment, owner = null ,acls = None) val found = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq("test")) assertEquals(assignment, found("test")) } @@ -120,7 +120,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val topic = "test" TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3, 4)) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) // create leaders for all partitions TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1) val actualReplicaList = leaderForPartitionMap.keys.toArray.map(p => (p -> ZkUtils.getReplicasForPartition(zkClient, topic, p))).toMap @@ -130,7 +130,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { intercept[TopicExistsException] { // shouldn't be able to create a topic that already exists - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) } } @@ -147,7 +147,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) // reassign partition 0 val newReplicas = Seq(0, 2, 3) val partitionToBeReassigned = 0 @@ -178,7 +178,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) // reassign partition 0 val newReplicas = Seq(1, 2, 3) val partitionToBeReassigned = 0 @@ -209,7 +209,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) // reassign partition 0 val newReplicas = Seq(2, 3) val partitionToBeReassigned = 0 @@ -253,7 +253,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1)) val topic = "test" // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) // put the partition in the reassigned path as well // reassign partition 0 val newReplicas = Seq(0, 1) @@ -300,7 +300,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(KafkaConfig.fromProps) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // broker 2 should be the leader since it was started first val currentLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, oldLeaderOpt = None).get @@ -393,7 +393,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { checkConfig(maxMessageSize, retentionMs) // now double the config values for the topic and check that it is applied - AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs)) + AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs), null, null) checkConfig(2*maxMessageSize, 2 * retentionMs) } finally { server.shutdown() diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index 6258983..106cfe3 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -102,7 +102,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val allServers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) val servers = allServers.filter(s => expectedReplicaAssignment(0).contains(s.config.brokerId)) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) // wait until replica log is created on every broker TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isDefined), @@ -147,7 +147,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val newPartition = TopicAndPartition(topic, 1) follower.shutdown() // add partitions to topic - AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2", false) + AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2", false, owner = null ,acls = None) // start topic deletion AdminUtils.deleteTopic(zkClient, topic) follower.startup() @@ -169,7 +169,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { AdminUtils.deleteTopic(zkClient, topic) // add partitions to topic val newPartition = TopicAndPartition(topic, 1) - AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2") + AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2", owner = null ,acls = None) TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // verify that new partition doesn't exist on any broker either assertTrue("Replica logs not deleted after delete topic is complete", @@ -187,7 +187,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { AdminUtils.deleteTopic(zkClient, topic) TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // re-create topic on same replicas - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) // wait until leader is elected val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) assertTrue("New leader should be elected after re-creating topic test", leaderIdOpt.isDefined) @@ -265,7 +265,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // create brokers val servers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) // wait until replica log is created on every broker TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isDefined), diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index ac6dd20..3753c44 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -17,12 +17,14 @@ package kafka.admin import junit.framework.Assert._ +import kafka.log.LogConfig +import kafka.security.auth.{Operation, PermissionType, Acl} import org.junit.Test import org.scalatest.junit.JUnit3Suite import kafka.utils.Logging import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness -import kafka.server.KafkaConfig +import kafka.server.{TopicConfig, KafkaConfig} import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils @@ -34,30 +36,51 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin val numPartitionsOriginal = 1 val cleanupKey = "cleanup.policy" val cleanupVal = "compact" + + val acl1: Acl = new Acl("alice", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ, Operation.WRITE)) + val acl2: Acl = new Acl("bob", PermissionType.ALLOW, Set[String]("*"), Set[Operation](Operation.READ, Operation.WRITE)) + val acl3: Acl = new Acl("bob", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ)) + // create brokers val brokers = List(0, 1, 2) + val aclFilePath: String = Thread.currentThread().getContextClassLoader.getResource("acl.json").getPath + TestUtils.createBrokersInZk(zkClient, brokers) // create the topic val createOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, "--replication-factor", "1", "--config", cleanupKey + "=" + cleanupVal, - "--topic", topic)) + "--topic", topic, + "--acl", aclFilePath)) + TopicCommand.createTopic(zkClient, createOpts) val props = AdminUtils.fetchTopicConfig(zkClient, topic) + + val topicConfig: TopicConfig = TopicConfig.fromProps(props) assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey)) assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal)) + assertEquals(Set[Acl](acl1, acl2, acl3), topicConfig.acls) + assertEquals(System.getProperty("user.name"), topicConfig.owner) // pre-create the topic config changes path to avoid a NoNodeException ZkUtils.createPersistentPath(zkClient, ZkUtils.TopicConfigChangesPath) // modify the topic to add new partitions val numPartitionsModified = 3 + val testUser: String = "testUser" val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, "--config", cleanupKey + "=" + cleanupVal, + "--owner", testUser, "--topic", topic)) TopicCommand.alterTopic(zkClient, alterOpts) val newProps = AdminUtils.fetchTopicConfig(zkClient, topic) + val newTopicConfig: TopicConfig = TopicConfig.fromProps(newProps) + assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey)) assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal)) + assertEquals(Set[Acl](acl1, acl2, acl3), newTopicConfig.acls) + assertEquals(testUser, newTopicConfig.owner) + + //TODO add test to verify acl can be modified using --acl during alter topic command. } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index 8342cae..f5f2d32 100644 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -103,7 +103,8 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { startBrokers(Seq(configProps1, configProps2)) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), + owner = null ,acls = None) verifyUncleanLeaderElectionEnabled } @@ -115,7 +116,8 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { startBrokers(Seq(configProps1, configProps2)) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), + owner = null ,acls = None) verifyUncleanLeaderElectionDisabled } @@ -130,7 +132,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val topicProps = new Properties() topicProps.put("unclean.leader.election.enable", String.valueOf(true)) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), - topicProps) + topicProps, owner = null ,acls = None) verifyUncleanLeaderElectionEnabled } @@ -145,7 +147,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val topicProps = new Properties() topicProps.put("unclean.leader.election.enable", String.valueOf(false)) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), - topicProps) + topicProps, owner = null ,acls = None) verifyUncleanLeaderElectionDisabled } @@ -158,7 +160,8 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { topicProps.put("unclean.leader.election.enable", "invalid") intercept[ConfigException] { - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1)), topicProps) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1)), topicProps, + owner = null ,acls = None) } } diff --git a/core/src/test/scala/unit/kafka/security/auth/AclTest.scala b/core/src/test/scala/unit/kafka/security/auth/AclTest.scala new file mode 100644 index 0000000..a4827ae --- /dev/null +++ b/core/src/test/scala/unit/kafka/security/auth/AclTest.scala @@ -0,0 +1,24 @@ +package unit.kafka.security.auth + +import kafka.security.auth.{Operation, PermissionType, Acl} +import kafka.utils.Json +import org.junit.{Assert, Test} +import org.scalatest.junit.JUnit3Suite + +/** + * Created by pbrahmbhatt on 3/26/15. + */ +class AclTest extends JUnit3Suite { + + @Test + def testParsing(): Unit = { + val acl1: Acl = new Acl("alice", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ, Operation.WRITE)) + val acl2: Acl = new Acl("bob", PermissionType.ALLOW, Set[String]("*"), Set[Operation](Operation.READ, Operation.WRITE)) + val acl3: Acl = new Acl("bob", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ)) + + val acls: Set[Acl] = Set[Acl](acl1, acl2, acl3) + val jsonAcls: String = Json.encode(Acl.toJsonCompatibleMap(acls)) + + Assert.assertEquals(acls, Acl.fromJson(jsonAcls)) + } +} diff --git a/core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala b/core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala new file mode 100644 index 0000000..fa93efb --- /dev/null +++ b/core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala @@ -0,0 +1,22 @@ +package unit.kafka.security.auth + +import kafka.security.auth.{ClusterAclCache, Acl, Operation, PermissionType} +import org.junit.{Assert, Test} +import org.scalatest.junit.JUnit3Suite + +/** + * Created by pbrahmbhatt on 4/6/15. + */ +class ClusterAclTest extends JUnit3Suite { + + @Test + def testClusterAcl(){ + val acl1: Acl = new Acl("alice", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ, Operation.WRITE)) + val acl2: Acl = new Acl("bob", PermissionType.ALLOW, Set[String]("*"), Set[Operation](Operation.READ, Operation.WRITE)) + val acl3: Acl = new Acl("bob", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ)) + + val acls: Set[Acl] = Set[Acl](acl1, acl2, acl3) + + Assert.assertEquals(Set[Acl](acl1, acl2, acl3), new ClusterAclCache(Thread.currentThread().getContextClassLoader.getResource("acl.json").getPath).clusterAcl) + } +} diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala new file mode 100644 index 0000000..e18b9e5 --- /dev/null +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -0,0 +1,166 @@ +package unit.kafka.security.auth + +import java.security.Principal +import java.util.Properties + +import com.sun.security.auth.UserPrincipal +import kafka.network.RequestChannel.Session +import kafka.security.auth._ +import kafka.server.{KafkaConfig, TopicConfig, TopicConfigCache} +import kafka.utils.TestUtils +import org.easymock.EasyMock +import org.junit.Test +import org.scalatest.junit.JUnit3Suite +import org.junit.Assert._ + + +class SimpleAclAuthorizerTest extends JUnit3Suite { + + val clusterAclCache: ClusterAclCache = EasyMock.createMock(classOf[ClusterAclCache]) + val topicConfigCache: TopicConfigCache = EasyMock.createMock(classOf[TopicConfigCache]) + val simpleAclAuthorizer: SimpleAclAuthorizer = new SimpleAclAuthorizer + val testPrincipal: Principal = new UserPrincipal(Acl.wildCardPrincipal) + val testHostName: String = "test.host.com" + var session: Session = new Session(testPrincipal, testHostName) + val resource: String = "test-topic" + val superUsers: String = "superuser1, superuser2" + + override def setUp() { + super.setUp() + + val props = TestUtils.createBrokerConfig(0, 8181) + props.put(KafkaConfig.ClusterAclJsonFilePathProp, "") + props.put(KafkaConfig.SuperUserProp, superUsers) + + val cfg = KafkaConfig.fromProps(props) + simpleAclAuthorizer.initialize(cfg, topicConfigCache) + } + + def testTopicAcl(): Unit = { + val user1: String = "user1" + val host1: String = "host1" + val host2: String = "host2" + + //user1 has READ access from host1 and host2. + val acl1: Acl = new Acl(user1, PermissionType.ALLOW, Set[String](host1, host2), Set[Operation](Operation.READ)) + + //user1 does not have READ access from host1. + val acl2: Acl = new Acl(user1, PermissionType.DENY, Set[String](host1), Set[Operation](Operation.READ)) + + //user1 has Write access from host1 only. + val acl3: Acl = new Acl(user1, PermissionType.ALLOW, Set[String](host1), Set[Operation](Operation.WRITE)) + + //user1 has DESCRIBE access from all hosts. + val acl4: Acl = new Acl(user1, PermissionType.ALLOW, Set[String](Acl.wildCardHost), Set[Operation](Operation.DESCRIBE)) + + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](acl1, acl2, acl3, acl4), overrideProperties = null) + EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() + EasyMock.replay(clusterAclCache, topicConfigCache) + + val host1Session: Session = new Session(new UserPrincipal(user1), host1) + val host2Session: Session = new Session(new UserPrincipal(user1), host2) + + assertTrue("User1 should have READ access from host2", simpleAclAuthorizer.authorize(host2Session, Operation.READ, resource)) + assertFalse("User1 should not have READ access from host1 due to denyAcl", simpleAclAuthorizer.authorize(host1Session, Operation.READ, resource)) + assertTrue("User1 should have WRITE access from host1", simpleAclAuthorizer.authorize(host1Session, Operation.WRITE, resource)) + assertFalse("User1 should not have WRITE access from host2 as no allow acl is defined", simpleAclAuthorizer.authorize(host2Session, Operation.WRITE, resource)) + assertTrue("User1 should have DESCRIBE access from host1", simpleAclAuthorizer.authorize(host1Session, Operation.DESCRIBE, resource)) + assertTrue("User1 should have DESCRIBE access from host2", simpleAclAuthorizer.authorize(host2Session, Operation.DESCRIBE, resource)) + assertFalse("User1 should not have edit access from host1", simpleAclAuthorizer.authorize(host1Session, Operation.EDIT, resource)) + assertFalse("User1 should not have edit access from host2", simpleAclAuthorizer.authorize(host2Session, Operation.EDIT, resource)) + + EasyMock.verify(clusterAclCache, topicConfigCache) + } + + @Test + def testDenyTakesPrecedence(): Unit = { + val user: String = "random-user" + val host: String = "random-host" + val session: Session = new Session(new UserPrincipal(user), host) + + val allowAll: Acl = Acl.allowAllAcl + val denyAcl: Acl = new Acl(user, PermissionType.DENY, Set[String](host), Set[Operation](Operation.ALL)) + + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](allowAll, denyAcl), overrideProperties = null) + EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() + EasyMock.replay(clusterAclCache, topicConfigCache) + + assertFalse("deny should take precedence over allow.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) + + EasyMock.verify(clusterAclCache, topicConfigCache) + } + + @Test + def testAllowAllAccess(): Unit = { + val allowAllAcl: Acl = Acl.allowAllAcl + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](Acl.allowAllAcl), overrideProperties = null) + EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() + EasyMock.replay(clusterAclCache, topicConfigCache) + + val session: Session = new Session(new UserPrincipal("random"), "random.host") + assertTrue("allow all acl should allow access to all.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) + + EasyMock.verify(clusterAclCache, topicConfigCache) + } + + @Test + def testOwnerHasAccess(): Unit = { + val denyAllAcl: Acl = new Acl(Acl.wildCardPrincipal, PermissionType.DENY, Set[String](Acl.wildCardHost), Set[Operation](Operation.ALL)) + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](denyAllAcl), overrideProperties = null) + EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() + EasyMock.replay(clusterAclCache, topicConfigCache) + + assertTrue("owner always has access, no matter what acls.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) + + EasyMock.verify(clusterAclCache, topicConfigCache) + } + + @Test + def testSuperUserHasAccess(): Unit = { + val denyAllAcl: Acl = new Acl(Acl.wildCardPrincipal, PermissionType.DENY, Set[String](Acl.wildCardHost), Set[Operation](Operation.ALL)) + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](denyAllAcl), overrideProperties = null) + EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() + EasyMock.replay(clusterAclCache, topicConfigCache) + + val session1: Session = new Session(new UserPrincipal("superuser1"), "random.host") + val session2: Session = new Session(new UserPrincipal("superuser2"), "random.host") + + assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session1, Operation.READ, resource)) + assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session2, Operation.READ, resource)) + + + EasyMock.verify(clusterAclCache, topicConfigCache) + } + + + @Test + def testNoAclFound(): Unit = { + val topicConfig1: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = null, overrideProperties = null) + val topicConfig2: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](), overrideProperties = null) + + EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig1).times(2) + EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig2).times(2) + EasyMock.replay(clusterAclCache, topicConfigCache) + + assertTrue("when acls = null, authorizer should fail open.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) + assertTrue("when acls = [], authorizer should fail open.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) + + EasyMock.verify(clusterAclCache, topicConfigCache) + } + + @Test + def testFailOpenOnProgrammingErrors(): Unit = { + EasyMock.replay(clusterAclCache, topicConfigCache) + + assertTrue("null session should fail open.", simpleAclAuthorizer.authorize(null, Operation.READ, resource)) + assertTrue("null principal should fail open.", simpleAclAuthorizer.authorize(new Session(null, testHostName), Operation.READ, resource)) + assertTrue("null host should fail open.", simpleAclAuthorizer.authorize(new Session(testPrincipal, null), Operation.READ, resource)) + + assertTrue("null resource should fail open.", simpleAclAuthorizer.authorize(session, Operation.READ, null)) + assertTrue("empty resource should fail open.", simpleAclAuthorizer.authorize(session, Operation.READ, "")) + + assertTrue("unsupported Operation should fail open.", simpleAclAuthorizer.authorize(session, Operation.CREATE, resource)) + + EasyMock.verify(clusterAclCache, topicConfigCache) + } +} diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 4043ea4..057dff2 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -42,20 +42,16 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { //check config cache gets populated for a new topic. val config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) - assertNotNull(config) - assertFalse(config.isEmpty) - assertEquals(oldVal, LogConfig.fromProps(config).flushInterval) + assertEquals(oldVal, config.logConfig.flushInterval) } - AdminUtils.changeTopicConfig(zkClient, tp.topic, LogConfig(flushInterval = newVal).toProps) + AdminUtils.changeTopicConfig(zkClient, tp.topic, LogConfig(flushInterval = newVal).toProps, null, null) TestUtils.retry(10000) { assertEquals(newVal, this.servers(0).logManager.getLog(tp).get.config.flushInterval) //check config cache was updated with the new values. val config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) - assertNotNull(config) - assertFalse(config.isEmpty) - assertEquals(newVal, LogConfig.fromProps(config).flushInterval) + assertEquals(newVal, config.logConfig.flushInterval) } } @@ -63,7 +59,7 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { def testConfigChangeOnNonExistingTopic() { val topic = TestUtils.tempTopic try { - AdminUtils.changeTopicConfig(zkClient, topic, LogConfig(flushInterval = 10000).toProps) + AdminUtils.changeTopicConfig(zkClient, topic, LogConfig(flushInterval = 10000).toProps, null, null) fail("Should fail with AdminOperationException for topic doesn't exist") } catch { case e: AdminOperationException => // expected diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 14c9837..8700602 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -269,6 +269,8 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.AuthorizerClassNameProp => // ignore string + case KafkaConfig.SuperUserProp => //ignore String + case KafkaConfig.ClusterAclJsonFilePathProp => //ignore String case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.HostNameProp => // ignore string diff --git a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala index 8de3245..fbdb39b 100644 --- a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala @@ -16,17 +16,13 @@ */ package unit.kafka.server -import java.util.Properties - -import kafka.admin.{AdminOperationException, AdminUtils} -import kafka.cluster.Broker +import junit.framework.Assert._ +import kafka.admin.AdminUtils import kafka.common.TopicAndPartition import kafka.integration.KafkaServerTestHarness import kafka.log.LogConfig -import kafka.server.{TopicConfigCache, KafkaConfig, KafkaServer} +import kafka.server.{KafkaConfig, TopicConfig} import kafka.utils.TestUtils -import junit.framework.Assert._ -import kafka.zk.ZooKeeperTestHarness import org.scalatest.junit.JUnit3Suite class TopicConfigCacheTest extends JUnit3Suite with KafkaServerTestHarness { @@ -34,21 +30,22 @@ class TopicConfigCacheTest extends JUnit3Suite with KafkaServerTestHarness { override val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, TestUtils.choosePort))) def testConfigCache { - var config = this.servers(0).topicConfigCache.getTopicConfig("not-existing-topic") - assertEquals("even for non existing topic we will return default config.",this.servers(0).config.toProps, config) + var config: TopicConfig = this.servers(0).topicConfigCache.getTopicConfig("not-existing-topic") + assertNull("for non existing topic owner should be null.",config.owner) + assertTrue("for non existing topic acls should be empty.",config.acls.isEmpty) + assertTrue("for non existing topic override property should be empty. ",config.overrideProperties.isEmpty) //newly created topics should be populated in cache on first request. val oldVal = 100000 val tp = TopicAndPartition("test", 0) AdminUtils.createTopic(zkClient, tp.topic, 1, 1, LogConfig(flushInterval = oldVal).toProps) config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) - assertEquals(oldVal, LogConfig.fromProps(config).flushInterval) + assertEquals(oldVal, config.logConfig.flushInterval) //test that addOrupdate works val newVal = 20000 - config = LogConfig(flushInterval = newVal).toProps - this.servers(0).topicConfigCache.addOrUpdateTopicConfig(tp.topic, config) + this.servers(0).topicConfigCache.addOrUpdateTopicConfig(tp.topic, LogConfig(flushInterval = newVal).toProps) config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) - assertEquals(newVal, LogConfig.fromProps(config).flushInterval) + assertEquals(newVal, config.logConfig.flushInterval) } } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 1682a77..1c1f62d 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -198,7 +198,8 @@ object TestUtils extends Logging { def createTopic(zkClient: ZkClient, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]], servers: Seq[KafkaServer]) : scala.collection.immutable.Map[Int, Option[Int]] = { // create topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaAssignment, + owner = null ,acls = None) // wait until the update metadata request for new topic reaches all servers partitionReplicaAssignment.keySet.map { case i => TestUtils.waitUntilMetadataIsPropagated(servers, topic, i) -- 1.9.5 (Apple Git-50.3) From a92cfbb56906c4f2b9ecb06bd889ef62d945d65d Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 7 Apr 2015 15:37:31 -0700 Subject: [PATCH 4/9] Fixing some merge errors. --- core/src/main/scala/kafka/admin/AdminUtils.scala | 9 ++++---- core/src/main/scala/kafka/server/KafkaServer.scala | 1 - .../main/scala/kafka/server/TopicConfigCache.scala | 2 +- core/src/test/resources/log4j.properties | 6 ++--- .../test/scala/unit/kafka/admin/AdminTest.scala | 26 ++++++++++++++++++++-- .../security/auth/SimpleAclAuthorizerTest.scala | 5 +++-- .../unit/kafka/server/TopicConfigCacheTest.scala | 2 +- 7 files changed, 37 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index d7e43f2..bb9ac6e 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -232,6 +232,9 @@ object AdminUtils extends Logging { partitions: Int, replicationFactor: Int, topicConfig: Properties = new Properties, + //TODO: owner should first be read from jaas login module, + // if no logged in user is found only then we should default to user.name. + // we could avoid storing any acls which currently holds the same meaning as allow all. owner: String = System.getProperty("user.name"), acls: Set[Acl] = Set[Acl](Acl.allowAllAcl)) { val brokerList = ZkUtils.getSortedBrokerList(zkClient) @@ -259,7 +262,7 @@ object AdminUtils extends Logging { // write out the config if there is any, this isn't transactional with the partition assignments writeTopicConfig(zkClient, topic, config, owner, acls) - + // create the partition assignment writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) } @@ -307,8 +310,6 @@ object AdminUtils extends Logging { /** * Write out the topic config to zk, if there is any - * TODO may we should just accept a TopicConfig instance here and call toProps on that, however LogConfig in topicConfig also has defaults - * we just want to store the overrides not the defaults for LogConfig. */ private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties, owner: String, acls: Option[Set[Acl]]) { val configMap: mutable.Map[String, String] = { @@ -321,7 +322,6 @@ object AdminUtils extends Logging { case _ => null } - //TODO: owner should first be read from jaas login module, if no logged in user is found only then we should default to user.name. val map = Map(TopicConfig.versionKey -> 2, TopicConfig.configKey -> configMap, TopicConfig.ownerKey -> owner, @@ -367,6 +367,7 @@ object AdminUtils extends Logging { props.setProperty(TopicConfig.versionKey, "2") map.get(TopicConfig.aclKey) match { case Some(acls: Map[String, Any]) => + //everything must be string so encoding back to Json string. props.setProperty(TopicConfig.aclKey, Json.encode(acls)) case Some(null) => case _ => throw new IllegalArgumentException("Invalid topic config: " + config) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index b558270..c65ffb5 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -179,7 +179,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg (protocol, endpoint) } - /* tell everyone we are alive */ kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, config.zkSessionTimeoutMs, zkClient) kafkaHealthcheck.startup() diff --git a/core/src/main/scala/kafka/server/TopicConfigCache.scala b/core/src/main/scala/kafka/server/TopicConfigCache.scala index d1901a5..271b2cf 100644 --- a/core/src/main/scala/kafka/server/TopicConfigCache.scala +++ b/core/src/main/scala/kafka/server/TopicConfigCache.scala @@ -22,7 +22,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.admin.AdminUtils import kafka.utils.Logging -import kafka.utils.Utils._ +import kafka.utils.CoreUtils._ import org.I0Itec.zkclient.ZkClient import scala.collection.mutable diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 1b7d5d8..6941125 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -log4j.rootLogger=OFF, stdout +log4j.rootLogger=DEBUG, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=ERROR -log4j.logger.org.apache.kafka=ERROR +log4j.logger.kafka=DEBUG +log4j.logger.org.apache.kafka=DEBUG # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index d0a1fbd..5fadc21 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -25,11 +25,10 @@ import kafka.log._ import kafka.zk.ZooKeeperTestHarness import kafka.utils.{Logging, ZkUtils, TestUtils} import kafka.common.{TopicExistsException, TopicAndPartition} -import kafka.server.{KafkaServer, KafkaConfig} +import kafka.server.{TopicConfig, KafkaServer, KafkaConfig} import java.io.File import TestUtils._ - class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @Test @@ -401,4 +400,27 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { } } + /** + * Test we can support both V1 and V2 configs. + */ + @Test + def testTopicConfigV1isSupported() { + val topic: String = "test-topic" + val server = TestUtils.createServer(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) + + + //Write and read a V1 format config. + val props: Properties = new Properties() + props.put("test", "test") + + val configMap: scala.collection.mutable.Map[String, String] = { + import scala.collection.JavaConversions._ + props + } + + val map: Map[String, Any]=Map[String, Any] (TopicConfig.versionKey -> 1, TopicConfig.configKey -> configMap) + ZkUtils.updatePersistentPath(server.zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map)) + assertEquals(props, AdminUtils.fetchTopicConfig(server.zkClient, topic)) + } + } 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 e18b9e5..2da476a 100644 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -8,13 +8,14 @@ import kafka.network.RequestChannel.Session import kafka.security.auth._ import kafka.server.{KafkaConfig, TopicConfig, TopicConfigCache} import kafka.utils.TestUtils +import kafka.zk.ZooKeeperTestHarness import org.easymock.EasyMock import org.junit.Test import org.scalatest.junit.JUnit3Suite import org.junit.Assert._ -class SimpleAclAuthorizerTest extends JUnit3Suite { +class SimpleAclAuthorizerTest extends JUnit3Suite with ZooKeeperTestHarness { val clusterAclCache: ClusterAclCache = EasyMock.createMock(classOf[ClusterAclCache]) val topicConfigCache: TopicConfigCache = EasyMock.createMock(classOf[TopicConfigCache]) @@ -28,7 +29,7 @@ class SimpleAclAuthorizerTest extends JUnit3Suite { override def setUp() { super.setUp() - val props = TestUtils.createBrokerConfig(0, 8181) + val props = TestUtils.createBrokerConfig(0, zkConnect) props.put(KafkaConfig.ClusterAclJsonFilePathProp, "") props.put(KafkaConfig.SuperUserProp, superUsers) diff --git a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala index fbdb39b..f126ac1 100644 --- a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala @@ -27,7 +27,7 @@ import org.scalatest.junit.JUnit3Suite class TopicConfigCacheTest extends JUnit3Suite with KafkaServerTestHarness { - override val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, TestUtils.choosePort))) + def generateConfigs() = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps(_)) def testConfigCache { var config: TopicConfig = this.servers(0).topicConfigCache.getTopicConfig("not-existing-topic") -- 1.9.5 (Apple Git-50.3) From 1d6db4c2d3f9948e61b5c3d1cf9a759c8fe8a224 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Fri, 10 Apr 2015 11:07:18 -0700 Subject: [PATCH 5/9] Converted some code to idomatic scala. --- .../kafka/security/auth/SimpleAclAuthorizer.scala | 1 - core/src/main/scala/kafka/server/KafkaApis.scala | 139 +++++++-------------- core/src/main/scala/kafka/server/KafkaServer.scala | 18 +-- core/src/main/scala/kafka/server/TopicConfig.scala | 5 +- .../security/auth/SimpleAclAuthorizerTest.scala | 15 ++- .../unit/kafka/server/TopicConfigCacheTest.scala | 1 - 6 files changed, 61 insertions(+), 118 deletions(-) diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index a8f87e1..d25eb83 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -88,7 +88,6 @@ class SimpleAclAuthorizer extends Authorizer with Logging { superUsers = kafkaConfig.superUser match { case null => Set.empty[String] case (str: String) => str.split(",").map(s => s.trim).toSet - case _ => throw new IllegalArgumentException("expected a comma seperated list of superusers , found:" + kafkaConfig.superUser) } configCache = topicConfigCache } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 3c7e6b3..7f5b6a7 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -49,7 +49,7 @@ class KafkaApis(val requestChannel: RequestChannel, val brokerId: Int, val config: KafkaConfig, val metadataCache: MetadataCache, - val authorizer: Authorizer) extends Logging { + val authorizer: Option[Authorizer]) extends Logging { this.logIdent = "[KafkaApi-%d] ".format(brokerId) @@ -102,7 +102,7 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest] - if(authorizer != null && !authorizer.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + if(authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, Map.empty, ErrorMapping.AuthorizationCode) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse))) return @@ -125,7 +125,7 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest] - if(authorizer != null && !authorizer.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + if(authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, Map.empty, ErrorMapping.AuthorizationCode) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse))) return @@ -140,11 +140,11 @@ class KafkaApis(val requestChannel: RequestChannel, def handleUpdateMetadataRequest(request: RequestChannel.Request) { val updateMetadataRequest = request.requestObj.asInstanceOf[UpdateMetadataRequest] - if(authorizer != null) { + if(authorizer.isDefined) { val unauthorizedTopicAndPartition = updateMetadataRequest.partitionStateInfos.filterKeys( - topicAndPartition => !authorizer.authorize(request.session, Operation.EDIT, topicAndPartition.topic)).keys + topicAndPartition => !authorizer.get.authorize(request.session, Operation.EDIT, topicAndPartition.topic)).keys //In this case the response does not allow to selectively report success/failure so if authorization fails, we fail the entire request. - if (unauthorizedTopicAndPartition != null && !unauthorizedTopicAndPartition.isEmpty) { + if (!unauthorizedTopicAndPartition.isEmpty) { val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId, ErrorMapping.AuthorizationCode) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(updateMetadataResponse))) return @@ -163,8 +163,8 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val controlledShutdownRequest = request.requestObj.asInstanceOf[ControlledShutdownRequest] - if(authorizer != null) { - if (!authorizer.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + if(authorizer.isDefined) { + if (!authorizer.get.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId, ErrorMapping.AuthorizationCode, Set.empty) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) return @@ -183,21 +183,13 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetCommitRequest(request: RequestChannel.Request) { val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] - var requestInfo = offsetCommitRequest.requestInfo - - val unAuthorizedResponse = scala.collection.mutable.Map[TopicAndPartition, Short]() - if(authorizer != null) { - val unauthorizedTopicAndPartition = requestInfo.filterKeys( - topicAndPartition => !authorizer.authorize(request.session, Operation.READ, topicAndPartition.topic)).keys - for (topicAndPartition <- unauthorizedTopicAndPartition) { - unAuthorizedResponse.put(topicAndPartition, ErrorMapping.AuthorizationCode) - requestInfo -= topicAndPartition - } - } + + val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetCommitRequest.requestInfo.partition( + mapEntry => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.DESCRIBE, mapEntry._1.topic)) // the callback for sending an offset commit response def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { - val mergedCommitStatus = commitStatus ++ unAuthorizedResponse + val mergedCommitStatus = commitStatus ++ unauthorizedRequestInfo.mapValues(_ => ErrorMapping.AuthorizationCode) mergedCommitStatus.foreach { case (topicAndPartition, errorCode) => // we only print warnings for known errors here; only replica manager could see an unknown // exception while trying to write the offset message to the local log, and it will log @@ -232,7 +224,7 @@ class KafkaApis(val requestChannel: RequestChannel, val currentTimestamp = SystemTime.milliseconds val defaultExpireTimestamp = offsetRetention + currentTimestamp - val offsetData = requestInfo.mapValues(offsetAndMetadata => + val offsetData = authorizedRequestInfo.mapValues(offsetAndMetadata => offsetAndMetadata.copy( commitTimestamp = currentTimestamp, expireTimestamp = { @@ -259,23 +251,13 @@ class KafkaApis(val requestChannel: RequestChannel, def handleProducerRequest(request: RequestChannel.Request) { val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] - //filter topic partitions which does not pass authorization. - val unauthorizedResponseStatus = scala.collection.mutable.Map[TopicAndPartition, ProducerResponseStatus]() - if(authorizer != null) { - val unauthorizedTopicAndPartition = produceRequest.data.filterKeys( - topicAndPartition => !authorizer.authorize(request.session, Operation.WRITE, topicAndPartition.topic)).keys - val unauthorizedResponse = ProducerResponseStatus(ErrorMapping.AuthorizationCode, -1) - for (topicAndPartition <- unauthorizedTopicAndPartition) { - unauthorizedResponseStatus.put(topicAndPartition, unauthorizedResponse) - //remove unauthorized topics from the original request. - produceRequest.data.remove(topicAndPartition) - } - } + val (authorizedRequestInfo, unauthorizedRequestInfo) = produceRequest.data.partition( + mapEntry => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.WRITE, mapEntry._1.topic)) // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { var errorInResponse = false - val mergedResponseStatus = responseStatus ++ unauthorizedResponseStatus + val mergedResponseStatus = responseStatus ++ unauthorizedRequestInfo.mapValues(_ => ProducerResponseStatus(ErrorMapping.AuthorizationCode, -1)) mergedResponseStatus.foreach { case (topicAndPartition, status) => // we only print warnings for known errors here; if it is unknown, it will cause @@ -314,7 +296,7 @@ class KafkaApis(val requestChannel: RequestChannel, produceRequest.ackTimeoutMs.toLong, produceRequest.requiredAcks, internalTopicsAllowed, - produceRequest.data, + authorizedRequestInfo, sendResponseCallback) // if the request is put into the purgatory, it will have a held reference @@ -328,23 +310,15 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] - var requestInfo: Map[TopicAndPartition, PartitionFetchInfo] = fetchRequest.requestInfo - - //filter topic partitions which does not pass authorization. - val unauthorizedPartitionData = scala.collection.mutable.Map[TopicAndPartition, FetchResponsePartitionData]() - if(authorizer != null) { - val unauthorizedTopicAndPartition = fetchRequest.requestInfo.filterKeys( - topicAndPartition => !authorizer.authorize(request.session, Operation.READ, topicAndPartition.topic)).keys - val unauthorizedResponse = FetchResponsePartitionData(ErrorMapping.AuthorizationCode, -1, MessageSet.Empty) - for (topicAndPartition <- unauthorizedTopicAndPartition) { - unauthorizedPartitionData.put(topicAndPartition, unauthorizedResponse) - requestInfo -= topicAndPartition - } - } + + val (authorizedRequestInfo, unauthorizedRequestInfo) = fetchRequest.requestInfo.partition( + mapEntry => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.READ, mapEntry._1.topic)) + + val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => FetchResponsePartitionData(ErrorMapping.AuthorizationCode, -1, MessageSet.Empty)) // the callback for sending a fetch response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { - val mergedResponseStatus = responsePartitionData ++ unauthorizedPartitionData + val mergedResponseStatus = responsePartitionData ++ unauthorizedResponseStatus mergedResponseStatus.foreach { case (topicAndPartition, data) => // we only print warnings for known errors here; if it is unknown, it will cause @@ -369,7 +343,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest.maxWait.toLong, fetchRequest.replicaId, fetchRequest.minBytes, - requestInfo, + authorizedRequestInfo, sendResponseCallback) } @@ -378,21 +352,13 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetRequest(request: RequestChannel.Request) { val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest] - var requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo] = offsetRequest.requestInfo - - //filter topic partitions which does not pass authorization - val unauthorizedResponseMap = scala.collection.mutable.Map[TopicAndPartition, PartitionOffsetsResponse]() - if(authorizer != null) { - val unauthorizedTopicAndPartition = offsetRequest.requestInfo.filterKeys( - topicAndPartition => !authorizer.authorize(request.session, Operation.DESCRIBE, topicAndPartition.topic)).keys - val unauthorizedResponse = PartitionOffsetsResponse(ErrorMapping.AuthorizationCode, Nil) - for (topicAndPartition <- unauthorizedTopicAndPartition) { - unauthorizedResponseMap.put(topicAndPartition, unauthorizedResponse) - requestInfo -= topicAndPartition - } - } - val responseMap = requestInfo.map(elem => { + val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.requestInfo.partition( + mapEntry => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.READ, mapEntry._1.topic)) + + val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => PartitionOffsetsResponse(ErrorMapping.AuthorizationCode, Nil)) + + val responseMap = authorizedRequestInfo.map(elem => { val (topicAndPartition, partitionOffsetRequestInfo) = elem try { // ensure leader exists @@ -433,7 +399,7 @@ class KafkaApis(val requestChannel: RequestChannel, } }) - val mergedResponseMap = responseMap ++ unauthorizedResponseMap + val mergedResponseMap = responseMap ++ unauthorizedResponseStatus val response = OffsetResponse(offsetRequest.correlationId, mergedResponseMap.toMap) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -534,19 +500,13 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] - var topics = metadataRequest.topics.toSet - - //filter topics which does not pass authorization. - var unauthorizedTopicMetaData: Seq[TopicMetadata] = List[TopicMetadata]() - if(authorizer != null) { - val unauthorizedTopics = metadataRequest.topics.filter(topic => !authorizer.authorize(request.session, Operation.DESCRIBE, topic)) - unauthorizedTopicMetaData = unauthorizedTopics.map(topic => new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.AuthorizationCode)) - for(topic <- unauthorizedTopics) { - topics -= topic - } - } + val topics = metadataRequest.topics.toSet - val topicMetadata = getTopicMetadata(topics, request.securityProtocol) + val (authorizedTopics, unauthorizedTopics) = topics.partition(topic => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.READ, topic)) + + val unauthorizedTopicMetaData = unauthorizedTopics.map(topic => new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.AuthorizationCode)) + + val topicMetadata = getTopicMetadata(authorizedTopics, request.securityProtocol) val brokers = metadataCache.getAliveBrokers trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) val response = new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(request.securityProtocol)), topicMetadata ++ unauthorizedTopicMetaData, metadataRequest.correlationId) @@ -560,13 +520,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] val (authorizedTopicPartitions, unauthorizedTopicPartitions) = offsetFetchRequest.requestInfo.partition( - topicAndPartition => { - if(authorizer != null) { - authorizer.authorize(request.session, Operation.DESCRIBE, topicAndPartition.topic) - } else { - true - } - } + topicAndPartition => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.DESCRIBE, topicAndPartition.topic) ) val authorizationError = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.AuthorizationCode) @@ -599,7 +553,7 @@ class KafkaApis(val requestChannel: RequestChannel, val partition = offsetManager.partitionFor(consumerMetadataRequest.group) //TODO: this can in turn create the topic, so we should check the create permissions if the config is enabled and topic is non existent. - if (authorizer != null && !authorizer.authorize(request.session, Operation.DESCRIBE, OffsetManager.OffsetsTopicName)) { + if (authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.DESCRIBE, OffsetManager.OffsetsTopicName)) { val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(errorResponse))) return @@ -629,12 +583,7 @@ class KafkaApis(val requestChannel: RequestChannel, val respHeader = new ResponseHeader(request.header.correlationId) val (authorizedTopics, unauthorizedTopics) = joinGroupRequest.topics().partition( - topic => { - if(authorizer!= null) - authorizer.authorize(request.session, Operation.DESCRIBE, topic) - else - true - } + topic => (!authorizer.isDefined || authorizer.get.authorize(request.session, Operation.DESCRIBE, topic)) ) val unauthorizedTopicPartition = unauthorizedTopics.map(topic => new TopicPartition(topic, -1)) @@ -643,7 +592,7 @@ class KafkaApis(val requestChannel: RequestChannel, def sendResponseCallback(partitions: List[TopicAndPartition], generationId: Int, errorCode: Short) { val partitionList = (partitions.map(tp => new TopicPartition(tp.topic, tp.partition)) ++ unauthorizedTopicPartition).toBuffer val error = if (errorCode == ErrorMapping.NoError && !unauthorizedTopicPartition.isEmpty) ErrorMapping.AuthorizationCode else errorCode - val responseBody = new JoinGroupResponse(errorCode, generationId, joinGroupRequest.consumerId, partitionList) + val responseBody = new JoinGroupResponse(error, generationId, joinGroupRequest.consumerId, partitionList) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(respHeader, responseBody))) } @@ -651,7 +600,7 @@ class KafkaApis(val requestChannel: RequestChannel, coordinator.consumerJoinGroup( joinGroupRequest.groupId(), joinGroupRequest.consumerId(), - authorizedTopics.toList, + authorizedTopics.toList, joinGroupRequest.sessionTimeout(), joinGroupRequest.strategy(), sendResponseCallback) @@ -661,12 +610,10 @@ class KafkaApis(val requestChannel: RequestChannel, val heartbeatRequest = request.body.asInstanceOf[HeartbeatRequest] val respHeader = new ResponseHeader(request.header.correlationId) - if(authorizer != null) { - if (!authorizer.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + if (authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { val heartbeatResponse = new HeartbeatResponse(ErrorMapping.AuthorizationCode) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(respHeader, heartbeatResponse))) return - } } // the callback for sending a heartbeat response diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index c65ffb5..bce8507 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -74,8 +74,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var kafkaHealthcheck: KafkaHealthcheck = null val metadataCache: MetadataCache = new MetadataCache(config.brokerId) - - var zkClient: ZkClient = null val correlationId: AtomicInteger = new AtomicInteger(0) val brokerMetaPropsFile = "meta.properties" @@ -150,13 +148,15 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /*initialize topic config cache*/ topicConfigCache = new TopicConfigCache(config.brokerId, zkClient, defaultConfig = config) - /* Get the authorizer */ - val authorizer: Authorizer = if (config.authorizerClassName != null && !config.authorizerClassName.isEmpty) - CoreUtils.createObject(config.authorizerClassName) - else - null - if(authorizer != null) { - authorizer.initialize(config, topicConfigCache) + /* Get the authorizer and initialize it if one is specified.*/ + val authorizer: Option[Authorizer] = if(config.authorizerClassName != null && !config.authorizerClassName.isEmpty) { + Option(CoreUtils.createObject(config.authorizerClassName)) + } else { + None + } + + if(authorizer.isDefined) { + authorizer.get.initialize(config, topicConfigCache) } /* start processing requests */ diff --git a/core/src/main/scala/kafka/server/TopicConfig.scala b/core/src/main/scala/kafka/server/TopicConfig.scala index ee85d22..451ce9b 100644 --- a/core/src/main/scala/kafka/server/TopicConfig.scala +++ b/core/src/main/scala/kafka/server/TopicConfig.scala @@ -25,17 +25,16 @@ object TopicConfig { val logConfig: LogConfig = LogConfig.fromProps(defaultProperties, overrideProperties) val acls: Set[Acl] = Acl.fromJson(overrideProperties.getProperty(aclKey)) - new TopicConfig(version, owner, logConfig, acls, overrideProperties) + new TopicConfig(version, owner, logConfig, acls) } } -class TopicConfig(val version: Int, val owner: String,val logConfig: LogConfig,val acls: Set[Acl], val overrideProperties: Properties) { +class TopicConfig(val version: Int, val owner: String,val logConfig: LogConfig,val acls: Set[Acl]) { def toProps(): Properties = { val properties: Properties = new Properties() properties.put(TopicConfig.ownerKey, owner) properties.put(TopicConfig.aclKey, Json.encode(acls.map(acl => acl.toMap()).toList)) properties.putAll(logConfig.toProps) - properties.putAll(overrideProperties) properties } 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 2da476a..b6313d8 100644 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -54,7 +54,7 @@ class SimpleAclAuthorizerTest extends JUnit3Suite with ZooKeeperTestHarness { //user1 has DESCRIBE access from all hosts. val acl4: Acl = new Acl(user1, PermissionType.ALLOW, Set[String](Acl.wildCardHost), Set[Operation](Operation.DESCRIBE)) - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](acl1, acl2, acl3, acl4), overrideProperties = null) + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](acl1, acl2, acl3, acl4)) EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() EasyMock.replay(clusterAclCache, topicConfigCache) @@ -82,7 +82,7 @@ class SimpleAclAuthorizerTest extends JUnit3Suite with ZooKeeperTestHarness { val allowAll: Acl = Acl.allowAllAcl val denyAcl: Acl = new Acl(user, PermissionType.DENY, Set[String](host), Set[Operation](Operation.ALL)) - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](allowAll, denyAcl), overrideProperties = null) + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](allowAll, denyAcl)) EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() EasyMock.replay(clusterAclCache, topicConfigCache) @@ -94,7 +94,7 @@ class SimpleAclAuthorizerTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testAllowAllAccess(): Unit = { val allowAllAcl: Acl = Acl.allowAllAcl - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](Acl.allowAllAcl), overrideProperties = null) + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](Acl.allowAllAcl)) EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() EasyMock.replay(clusterAclCache, topicConfigCache) @@ -107,7 +107,7 @@ class SimpleAclAuthorizerTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testOwnerHasAccess(): Unit = { val denyAllAcl: Acl = new Acl(Acl.wildCardPrincipal, PermissionType.DENY, Set[String](Acl.wildCardHost), Set[Operation](Operation.ALL)) - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](denyAllAcl), overrideProperties = null) + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](denyAllAcl)) EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() EasyMock.replay(clusterAclCache, topicConfigCache) @@ -119,7 +119,7 @@ class SimpleAclAuthorizerTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testSuperUserHasAccess(): Unit = { val denyAllAcl: Acl = new Acl(Acl.wildCardPrincipal, PermissionType.DENY, Set[String](Acl.wildCardHost), Set[Operation](Operation.ALL)) - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](denyAllAcl), overrideProperties = null) + val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](denyAllAcl)) EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() EasyMock.replay(clusterAclCache, topicConfigCache) @@ -129,15 +129,14 @@ class SimpleAclAuthorizerTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session1, Operation.READ, resource)) assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session2, Operation.READ, resource)) - EasyMock.verify(clusterAclCache, topicConfigCache) } @Test def testNoAclFound(): Unit = { - val topicConfig1: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = null, overrideProperties = null) - val topicConfig2: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](), overrideProperties = null) + val topicConfig1: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = null) + val topicConfig2: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl]()) EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig1).times(2) EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig2).times(2) diff --git a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala index f126ac1..4b8b613 100644 --- a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala @@ -33,7 +33,6 @@ class TopicConfigCacheTest extends JUnit3Suite with KafkaServerTestHarness { var config: TopicConfig = this.servers(0).topicConfigCache.getTopicConfig("not-existing-topic") assertNull("for non existing topic owner should be null.",config.owner) assertTrue("for non existing topic acls should be empty.",config.acls.isEmpty) - assertTrue("for non existing topic override property should be empty. ",config.overrideProperties.isEmpty) //newly created topics should be populated in cache on first request. val oldVal = 100000 -- 1.9.5 (Apple Git-50.3) From d9898a51c84d8316f010156418fa02699cf28982 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Fri, 10 Apr 2015 12:11:52 -0700 Subject: [PATCH 6/9] Removing some unintended changes. --- core/src/main/scala/kafka/server/MetadataCache.scala | 2 +- core/src/main/scala/kafka/utils/Json.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 8cf32df..9a9205f 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -34,7 +34,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock * A cache for the state (e.g., current leader) of each partition. This cache is updated through * UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously. */ -private[kafka] class MetadataCache(brokerId: Int) extends Logging { +private[server] class MetadataCache(brokerId: Int) extends Logging { private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] = new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]() private var aliveBrokers: Map[Int, Broker] = Map() diff --git a/core/src/main/scala/kafka/utils/Json.scala b/core/src/main/scala/kafka/utils/Json.scala index ae7acab..d110284 100644 --- a/core/src/main/scala/kafka/utils/Json.scala +++ b/core/src/main/scala/kafka/utils/Json.scala @@ -53,7 +53,7 @@ object Json extends Logging { obj match { case null => "null" case b: Boolean => b.toString - case s: String => "\"" + s.replace("\"","\\\"") + "\"" + case s: String => "\"" + s + "\"" case n: Number => n.toString case m: Map[_, _] => "{" + -- 1.9.5 (Apple Git-50.3) From 82deb9dc57b130ccf37e9a2c7fdea7bf7427ac87 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 21 Apr 2015 12:00:27 -0700 Subject: [PATCH 7/9] Reverting the topic config related changes --- core/src/main/scala/kafka/admin/AdminUtils.scala | 92 ++++----------- core/src/main/scala/kafka/admin/TopicCommand.scala | 81 ++----------- core/src/main/scala/kafka/api/TopicMetadata.scala | 5 +- core/src/main/scala/kafka/log/LogConfig.scala | 28 +---- .../kafka/security/auth/ClusterAclCache.scala | 21 ---- core/src/main/scala/kafka/server/TopicConfig.scala | 41 ------- .../main/scala/kafka/server/TopicConfigCache.scala | 77 ------------- .../scala/kafka/server/TopicConfigManager.scala | 2 - core/src/test/resources/acl.json | 1 - core/src/test/resources/log4j.properties | 6 +- .../scala/unit/kafka/admin/AddPartitionsTest.scala | 10 +- .../test/scala/unit/kafka/admin/AdminTest.scala | 50 +++------ .../scala/unit/kafka/admin/DeleteTopicTest.scala | 10 +- .../scala/unit/kafka/admin/TopicCommandTest.scala | 26 +---- .../integration/UncleanLeaderElectionTest.scala | 58 ++++------ .../scala/unit/kafka/security/auth/AclTest.scala | 24 ---- .../unit/kafka/security/auth/ClusterAclTest.scala | 22 ---- .../kafka/server/DynamicConfigChangeTest.scala | 13 +-- .../unit/kafka/server/TopicConfigCacheTest.scala | 50 --------- .../test/scala/unit/kafka/utils/TestUtils.scala | 125 ++++++++++++--------- 20 files changed, 161 insertions(+), 581 deletions(-) delete mode 100644 core/src/main/scala/kafka/security/auth/ClusterAclCache.scala delete mode 100644 core/src/main/scala/kafka/server/TopicConfig.scala delete mode 100644 core/src/main/scala/kafka/server/TopicConfigCache.scala delete mode 100644 core/src/test/resources/acl.json delete mode 100644 core/src/test/scala/unit/kafka/security/auth/AclTest.scala delete mode 100644 core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala delete mode 100644 core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index bb9ac6e..eee80f9 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -21,8 +21,6 @@ import kafka.common._ import kafka.cluster.{BrokerEndPoint, Broker} import kafka.log.LogConfig -import kafka.security.auth.Acl -import kafka.server.TopicConfig import kafka.utils._ import kafka.api.{TopicMetadata, PartitionMetadata} @@ -112,9 +110,7 @@ object AdminUtils extends Logging { numPartitions: Int = 1, replicaAssignmentStr: String = "", checkBrokerAvailable: Boolean = true, - config: Properties = new Properties, - owner: String, - acls: Option[Set[Acl]]) { + config: Properties = new Properties) { val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) if (existingPartitionsReplicaList.size == 0) throw new AdminOperationException("The topic %s does not exist".format(topic)) @@ -141,7 +137,7 @@ object AdminUtils extends Logging { val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2) // add the new list partitionReplicaList ++= newPartitionReplicaList - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true, owner, acls) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true) } def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = { @@ -231,38 +227,30 @@ object AdminUtils extends Logging { topic: String, partitions: Int, replicationFactor: Int, - topicConfig: Properties = new Properties, - //TODO: owner should first be read from jaas login module, - // if no logged in user is found only then we should default to user.name. - // we could avoid storing any acls which currently holds the same meaning as allow all. - owner: String = System.getProperty("user.name"), - acls: Set[Acl] = Set[Acl](Acl.allowAllAcl)) { + topicConfig: Properties = new Properties) { val brokerList = ZkUtils.getSortedBrokerList(zkClient) val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig, update = false, owner, Some(acls)) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig) } def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient, topic: String, partitionReplicaAssignment: Map[Int, Seq[Int]], config: Properties = new Properties, - update: Boolean = false, - owner: String, - acls: Option[Set[Acl]]) { + update: Boolean = false) { // validate arguments Topic.validate(topic) LogConfig.validate(config) - require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.") val topicPath = ZkUtils.getTopicPath(topic) if(!update && zkClient.exists(topicPath)) throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment)) - + // write out the config if there is any, this isn't transactional with the partition assignments - writeTopicConfig(zkClient, topic, config, owner, acls) - + writeTopicConfig(zkClient, topic, config) + // create the partition assignment writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) } @@ -294,7 +282,7 @@ object AdminUtils extends Logging { * existing configs need to be deleted, it should be done prior to invoking this API * */ - def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties, owner: String, acls: Option[Set[Acl]]) { + def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties) { if(!topicExists(zkClient, topic)) throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic)) @@ -302,7 +290,7 @@ object AdminUtils extends Logging { LogConfig.validate(configs) // write the new config--may not exist if there were previously no overrides - writeTopicConfig(zkClient, topic, configs, owner, acls) + writeTopicConfig(zkClient, topic, configs) // create the change notification zkClient.createPersistentSequential(ZkUtils.TopicConfigChangesPath + "/" + TopicConfigChangeZnodePrefix, Json.encode(topic)) @@ -311,22 +299,12 @@ object AdminUtils extends Logging { /** * Write out the topic config to zk, if there is any */ - private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties, owner: String, acls: Option[Set[Acl]]) { + private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties) { val configMap: mutable.Map[String, String] = { import JavaConversions._ config } - - val aclMap: Map[String, Any] = acls match { - case Some(aclSet: Set[Acl]) => Acl.toJsonCompatibleMap(aclSet.toSet) - case _ => null - } - - val map = Map(TopicConfig.versionKey -> 2, - TopicConfig.configKey -> configMap, - TopicConfig.ownerKey -> owner, - TopicConfig.aclKey -> aclMap) - + val map = Map("version" -> 1, "config" -> configMap) ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map)) } @@ -335,53 +313,25 @@ object AdminUtils extends Logging { */ def fetchTopicConfig(zkClient: ZkClient, topic: String): Properties = { val str: String = zkClient.readData(ZkUtils.getTopicConfigPath(topic), true) - var props = new Properties() + val props = new Properties() if(str != null) { Json.parseFull(str) match { case None => // there are no config overrides case Some(map: Map[String, _]) => - if (map(TopicConfig.versionKey) == 1) - props = toTopicConfigV1(map, str) - else - props = toTopicConfigV2(map, str) + require(map("version") == 1) + map.get("config") match { + case Some(config: Map[String, String]) => + for((k,v) <- config) + props.setProperty(k, v) + case _ => throw new IllegalArgumentException("Invalid topic config: " + str) + } + case o => throw new IllegalArgumentException("Unexpected value in config: " + str) } } props } - def toTopicConfigV1(map: Map[String, Any], config: String): Properties = { - val props = new Properties() - map.get(TopicConfig.configKey) match { - case Some(config: Map[String, String]) => - for((k,v) <- config) - props.setProperty(k, v) - case _ => throw new IllegalArgumentException("Invalid topic config: " + config) - } - props - } - - def toTopicConfigV2(map: Map[String, Any], config: String): Properties = { - val props = toTopicConfigV1(map, config) - - props.setProperty(TopicConfig.versionKey, "2") - map.get(TopicConfig.aclKey) match { - case Some(acls: Map[String, Any]) => - //everything must be string so encoding back to Json string. - props.setProperty(TopicConfig.aclKey, Json.encode(acls)) - case Some(null) => - case _ => throw new IllegalArgumentException("Invalid topic config: " + config) - } - - map.get(TopicConfig.ownerKey) match { - case Some(owner: String) => - props.setProperty(TopicConfig.ownerKey, owner) - case Some(null) => - case _ => throw new IllegalArgumentException("Invalid topic config: " + config) - } - props - } - def fetchAllTopicConfigs(zkClient: ZkClient): Map[String, Properties] = ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchTopicConfig(zkClient, topic))).toMap diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 5b95603..60f0228 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -17,11 +17,8 @@ package kafka.admin -import java.nio.file.{Paths, Files} - import joptsimple._ import java.util.Properties -import kafka.security.auth.Acl import kafka.common.{Topic, AdminCommandFailedException} import kafka.utils._ import org.I0Itec.zkclient.ZkClient @@ -30,7 +27,7 @@ import scala.collection._ import scala.collection.JavaConversions._ import kafka.log.LogConfig import kafka.consumer.Whitelist -import kafka.server.{TopicConfig, OffsetManager} +import kafka.server.OffsetManager import org.apache.kafka.common.utils.Utils @@ -85,20 +82,14 @@ object TopicCommand { def createTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topic = opts.options.valueOf(opts.topicOpt) val configs = parseTopicConfigsToBeAdded(opts) - val acls: Set[Acl] = parseAcl(opts) - val owner: String = if (opts.options.has(opts.ownerOpt)) - opts.options.valueOf(opts.ownerOpt).trim - else - System.getProperty("user.name") - if (opts.options.has(opts.replicaAssignmentOpt)) { val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs, owner = owner, acls = Some(acls)) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs) } else { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt) val partitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicas = opts.options.valueOf(opts.replicationFactorOpt).intValue - AdminUtils.createTopic(zkClient, topic, partitions, replicas, configs, acls = acls) + AdminUtils.createTopic(zkClient, topic, partitions, replicas, configs) } println("Created topic \"%s\".".format(topic)) } @@ -108,38 +99,17 @@ object TopicCommand { if (topics.length == 0) { println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt))) } - topics.foreach { topic => - var configs = AdminUtils.fetchTopicConfig(zkClient, topic) - var topicConfigs: TopicConfig = TopicConfig.fromProps(configs) - //for backward compatibility remove all configs that are not LogConfig properties. - configs = LogConfig.getFilteredProps(configs) - - if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt) || opts.options.has(opts.aclOpt) || opts.options.has(opts.ownerOpt)) { + val configs = AdminUtils.fetchTopicConfig(zkClient, topic) + if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) { val configsToBeAdded = parseTopicConfigsToBeAdded(opts) val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts) - val acls: Set[Acl] = if (opts.options.has(opts.aclOpt)) - parseAcl(opts) - else - topicConfigs.acls - val owner: String = if (opts.options.has(opts.ownerOpt)) - opts.options.valueOf(opts.ownerOpt).trim - else - topicConfigs.owner - // compile the final set of configs configs.putAll(configsToBeAdded) configsToBeDeleted.foreach(config => configs.remove(config)) - AdminUtils.changeTopicConfig(zkClient, topic, configs, owner, Some(acls)) + AdminUtils.changeTopicConfig(zkClient, topic, configs) println("Updated config for topic \"%s\".".format(topic)) } - - //reload the config from zookeeper as it might have been just updated. - configs = AdminUtils.fetchTopicConfig(zkClient, topic) - topicConfigs = TopicConfig.fromProps(configs) - //for backward compatibility remove all configs that are not LogConfig properties. - configs = LogConfig.getFilteredProps(configs) - if(opts.options.has(opts.partitionsOpt)) { if (topic == OffsetManager.OffsetsTopicName) { throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.") @@ -148,7 +118,7 @@ object TopicCommand { "logic or ordering of the messages will be affected") val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt) - AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs, owner = topicConfigs.owner, acls = Some(topicConfigs.acls)) + AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs) println("Adding partitions succeeded!") } } @@ -202,20 +172,13 @@ object TopicCommand { val describeConfigs: Boolean = !reportUnavailablePartitions && !reportUnderReplicatedPartitions val describePartitions: Boolean = !reportOverriddenConfigs val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) - if (describeConfigs) { val configs = AdminUtils.fetchTopicConfig(zkClient, topic) - val topicConfig: TopicConfig = TopicConfig.fromProps(configs) - val logConfigs = LogConfig.getFilteredProps(configs) - if (!reportOverriddenConfigs || configs.size() != 0) { val numPartitions = topicPartitionAssignment.size val replicationFactor = topicPartitionAssignment.head._2.size - println("Topic:%s\tOwner:%s\tPartitionCount:%d\tReplicationFactor:%d\tConfigs:%s" - .format(topic, topicConfig.owner, numPartitions, replicationFactor, - logConfigs.map(kv => kv._1 + "=" + kv._2).mkString(","))) - println("Acls:") - topicConfig.acls.foreach(acl => println(acl)) + println("Topic:%s\tPartitionCount:%d\tReplicationFactor:%d\tConfigs:%s" + .format(topic, numPartitions, replicationFactor, configs.map(kv => kv._1 + "=" + kv._2).mkString(","))) } } if (describePartitions) { @@ -239,19 +202,6 @@ object TopicCommand { } } - def parseAcl(opts: TopicCommandOptions): Set[Acl] = { - if (opts.options.has(opts.aclOpt)) { - val aclJsonFilePath = opts.options.valueOf(opts.aclOpt).trim - val source = scala.io.Source.fromFile(aclJsonFilePath) - val jsonAcls = source.mkString - source.close() - //validate acls can be parsed - return Acl.fromJson(jsonAcls) - } - - return Set[Acl](Acl.allowAllAcl) - } - def parseTopicConfigsToBeAdded(opts: TopicCommandOptions): Properties = { val configsToBeAdded = opts.options.valuesOf(opts.configOpt).map(_.split("""\s*=\s*""")) require(configsToBeAdded.forall(config => config.length == 2), @@ -339,17 +289,6 @@ object TopicCommand { val topicsWithOverridesOpt = parser.accepts("topics-with-overrides", "if set when describing topics, only show topics that have overridden configs") - val aclOpt = parser.accepts("acl", "Path to the acl json file that describes the acls for the topic. This is not additive, i.e. the new acls will overwrite any old acls.") - .withRequiredArg() - .describedAs("Path to topic acl json file") - .ofType(classOf[String]) - - //TODO: Should we even allow users to specify someone else as owner? or just default to keytab logged in user or the user executing command if no jaas login has happened.? - val ownerOpt = parser.accepts("owner", "User name for the owner of the topic. Default : user running the command.") - .withRequiredArg() - .describedAs("User name for the owner of the topic.") - .ofType(classOf[String]) - val options = parser.parse(args : _*) val allTopicLevelOpts: Set[OptionSpec[_]] = Set(alterOpt, createOpt, describeOpt, listOpt) @@ -362,8 +301,6 @@ object TopicCommand { // check invalid args CommandLineUtils.checkInvalidArgs(parser, options, configOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, aclOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, ownerOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, deleteConfigOpt, allTopicLevelOpts -- Set(alterOpt)) CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, allTopicLevelOpts -- Set(createOpt)) diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 2ef6802..5e39f45 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -30,21 +30,19 @@ object TopicMetadata { def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndPoint]): TopicMetadata = { val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue)) val topic = readShortString(buffer) - val numPartitions = readIntInRange(buffer, "number of partitions", (0, Int.MaxValue)) val partitionsMetadata: Array[PartitionMetadata] = new Array[PartitionMetadata](numPartitions) for(i <- 0 until numPartitions) { val partitionMetadata = PartitionMetadata.readFrom(buffer, brokers) partitionsMetadata(partitionMetadata.partitionId) = partitionMetadata } - new TopicMetadata(topic, partitionsMetadata, errorCode) } } case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError) extends Logging { def sizeInBytes: Int = { - 2 /* error code */ + + 2 /* error code */ + shortStringLength(topic) + 4 + partitionsMetadata.map(_.sizeInBytes).sum /* size and partition data array */ } @@ -62,7 +60,6 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat override def toString(): String = { val topicMetadataInfo = new StringBuilder topicMetadataInfo.append("{TopicMetadata for topic %s -> ".format(topic)) - errorCode match { case ErrorMapping.NoError => partitionsMetadata.foreach { partitionMetadata => diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 87abf07..da55a34 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -21,8 +21,6 @@ import java.util.Properties import org.apache.kafka.common.utils.Utils import scala.collection._ import org.apache.kafka.common.config.ConfigDef -import kafka.common._ -import scala.collection.JavaConversions._ import kafka.message.BrokerCompressionCodec object Defaults { @@ -93,7 +91,7 @@ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, props.put(FlushMessagesProp, flushInterval.toString) props.put(FlushMsProp, flushMs.toString) props.put(RetentionBytesProp, retentionSize.toString) - props.put(RententionMsProp, retentionMs.toString) + props.put(RetentionMsProp, retentionMs.toString) props.put(MaxMessageBytesProp, maxMessageSize.toString) props.put(IndexIntervalBytesProp, indexInterval.toString) props.put(DeleteRetentionMsProp, deleteRetentionMs.toString) @@ -122,7 +120,7 @@ object LogConfig { val FlushMessagesProp = "flush.messages" val FlushMsProp = "flush.ms" val RetentionBytesProp = "retention.bytes" - val RententionMsProp = "retention.ms" + val RetentionMsProp = "retention.ms" val MaxMessageBytesProp = "max.message.bytes" val IndexIntervalBytesProp = "index.interval.bytes" val DeleteRetentionMsProp = "delete.retention.ms" @@ -172,7 +170,8 @@ object LogConfig { .define(FlushMsProp, LONG, Defaults.FlushMs, atLeast(0), MEDIUM, FlushMsDoc) // can be negative. See kafka.log.LogManager.cleanupSegmentsToMaintainSize .define(RetentionBytesProp, LONG, Defaults.RetentionSize, MEDIUM, RetentionSizeDoc) - .define(RententionMsProp, LONG, Defaults.RetentionMs, atLeast(0), MEDIUM, RetentionMsDoc) + // can be negative. See kafka.log.LogManager.cleanupExpiredSegments + .define(RetentionMsProp, LONG, Defaults.RetentionMs, MEDIUM, RetentionMsDoc) .define(MaxMessageBytesProp, INT, Defaults.MaxMessageSize, atLeast(0), MEDIUM, MaxMessageSizeDoc) .define(IndexIntervalBytesProp, INT, Defaults.IndexInterval, atLeast(0), MEDIUM, IndexIntervalDoc) .define(DeleteRetentionMsProp, LONG, Defaults.DeleteRetentionMs, atLeast(0), MEDIUM, DeleteRetentionMsDoc) @@ -206,7 +205,7 @@ object LogConfig { flushInterval = parsed.get(FlushMessagesProp).asInstanceOf[Long], flushMs = parsed.get(FlushMsProp).asInstanceOf[Long], retentionSize = parsed.get(RetentionBytesProp).asInstanceOf[Long], - retentionMs = parsed.get(RententionMsProp).asInstanceOf[Long], + retentionMs = parsed.get(RetentionMsProp).asInstanceOf[Long], maxMessageSize = parsed.get(MaxMessageBytesProp).asInstanceOf[Int], indexInterval = parsed.get(IndexIntervalBytesProp).asInstanceOf[Int], fileDeleteDelayMs = parsed.get(FileDeleteDelayMsProp).asInstanceOf[Long], @@ -245,21 +244,4 @@ object LogConfig { configDef.parse(props) } - /** - * Removes any properties that are not defined in configDef - * @param props - * @return - */ - def getFilteredProps(props: Properties): Properties = { - val filteredProps: Properties = new Properties() - val names = configDef.names() - - for((k,v) <- props) { - if(names.contains(k)) { - filteredProps.put(k, v) - } - } - filteredProps - } - } diff --git a/core/src/main/scala/kafka/security/auth/ClusterAclCache.scala b/core/src/main/scala/kafka/security/auth/ClusterAclCache.scala deleted file mode 100644 index d03b362..0000000 --- a/core/src/main/scala/kafka/security/auth/ClusterAclCache.scala +++ /dev/null @@ -1,21 +0,0 @@ -package kafka.security.auth - -import java.nio.file.{Files, Paths} - - -/** - * @param clusterAclFilePath local file path to the json file that describes cluster acls. - */ -class ClusterAclCache(clusterAclFilePath: String) { - - val clusterAcl: Set[Acl] = { - if(clusterAclFilePath != null && !clusterAclFilePath.isEmpty && Files.exists(Paths.get(clusterAclFilePath))) { - val source = scala.io.Source.fromFile(clusterAclFilePath) - val jsonAcls = source.mkString - source.close() - Acl.fromJson(jsonAcls) - } else { - collection.immutable.Set.empty[Acl] - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/TopicConfig.scala b/core/src/main/scala/kafka/server/TopicConfig.scala deleted file mode 100644 index 451ce9b..0000000 --- a/core/src/main/scala/kafka/server/TopicConfig.scala +++ /dev/null @@ -1,41 +0,0 @@ -package kafka.server - -import java.util.Properties - -import kafka.log.LogConfig -import kafka.security.auth.Acl -import kafka.utils.Json - -object TopicConfig { - val versionKey = "version" - val configKey = "config" - val aclKey = "acls" - val ownerKey = "owner" - - def fromProps(properties: Properties) : TopicConfig = { - fromProps(new Properties(), properties) - } - - def fromProps(defaultProperties: Properties, overrideProperties: Properties) : TopicConfig = { - val version: Int = Option(overrideProperties.getProperty(versionKey)) match { - case Some(version: String) => version.toInt - case None => 1 - } - val owner: String = overrideProperties.getProperty(ownerKey) - val logConfig: LogConfig = LogConfig.fromProps(defaultProperties, overrideProperties) - val acls: Set[Acl] = Acl.fromJson(overrideProperties.getProperty(aclKey)) - - new TopicConfig(version, owner, logConfig, acls) - } -} - -class TopicConfig(val version: Int, val owner: String,val logConfig: LogConfig,val acls: Set[Acl]) { - def toProps(): Properties = { - val properties: Properties = new Properties() - properties.put(TopicConfig.ownerKey, owner) - properties.put(TopicConfig.aclKey, Json.encode(acls.map(acl => acl.toMap()).toList)) - properties.putAll(logConfig.toProps) - - properties - } -} diff --git a/core/src/main/scala/kafka/server/TopicConfigCache.scala b/core/src/main/scala/kafka/server/TopicConfigCache.scala deleted file mode 100644 index 271b2cf..0000000 --- a/core/src/main/scala/kafka/server/TopicConfigCache.scala +++ /dev/null @@ -1,77 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import java.util.Properties -import java.util.concurrent.locks.ReentrantReadWriteLock - -import kafka.admin.AdminUtils -import kafka.utils.Logging -import kafka.utils.CoreUtils._ -import org.I0Itec.zkclient.ZkClient - -import scala.collection.mutable - -/** - * A cache for topic configs that is maintained by each broker, this will not just return the overrides but also defaults. - */ -class TopicConfigCache(brokerId: Int, val zkClient: ZkClient, defaultConfig: KafkaConfig) extends Logging { - private val cache: mutable.Map[String, TopicConfig] = new mutable.HashMap[String, TopicConfig]() - private val lock = new ReentrantReadWriteLock() - - this.logIdent = "[Kafka Topic Config Cache on broker %d] ".format(brokerId) - - /** - * Read the topic config from zookeeper and add it to cache. - * @param topic - */ - private def populateTopicConfig(topic: String): Unit = { - inWriteLock(lock) { - val overrideProperties: Properties = AdminUtils.fetchTopicConfig(zkClient, topic) - addOrUpdateTopicConfig(topic, overrideProperties) - } - } - - /** - * addOrUpdate the topic config cache. - * @param topic - * @param overrideProperties - */ - def addOrUpdateTopicConfig(topic: String, overrideProperties: Properties) { - inWriteLock(lock) { - cache.put(topic, TopicConfig.fromProps(defaultConfig.toProps, overrideProperties)) - } - } - - /** - * Returns the topic config. - * @param topic - * @return - */ - def getTopicConfig(topic: String): TopicConfig = { - inReadLock(lock) { - if(cache.contains(topic)) { - return cache(topic) - } - } - - populateTopicConfig(topic) - - return getTopicConfig(topic) - } -} diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala index 529007a..47295d4 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -60,7 +60,6 @@ import org.I0Itec.zkclient.{IZkChildListener, ZkClient} */ class TopicConfigManager(private val zkClient: ZkClient, private val logManager: LogManager, - private val topicConfigCache: TopicConfigCache, private val changeExpirationMs: Long = 15*60*1000, private val time: Time = SystemTime) extends Logging { private var lastExecutedChange = -1L @@ -104,7 +103,6 @@ class TopicConfigManager(private val zkClient: ZkClient, /* combine the default properties with the overrides in zk to create the new LogConfig */ val props = new Properties(logManager.defaultConfig.toProps) props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) - topicConfigCache.addOrUpdateTopicConfig(topic, props) val logConfig = LogConfig.fromProps(props) for (log <- logsByTopic(topic)) log.config = logConfig diff --git a/core/src/test/resources/acl.json b/core/src/test/resources/acl.json deleted file mode 100644 index fd9b916..0000000 --- a/core/src/test/resources/acl.json +++ /dev/null @@ -1 +0,0 @@ -{"version": 1, "acls": [{"hosts":["host1","host2"],"permissionType":"DENY","operations":["READ","WRITE"],"principal":"alice"},{"hosts":["*"],"permissionType":"ALLOW","operations":["READ","WRITE"],"principal":"bob"},{"hosts":["host1","host2"],"permissionType":"DENY","operations":["READ"],"principal":"bob"}]} diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 6941125..1b7d5d8 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -log4j.rootLogger=DEBUG, stdout +log4j.rootLogger=OFF, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=DEBUG -log4j.logger.org.apache.kafka=DEBUG +log4j.logger.kafka=ERROR +log4j.logger.org.apache.kafka=ERROR # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 9c19660..ab5d16c 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -62,7 +62,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testTopicDoesNotExist { try { - AdminUtils.addPartitions(zkClient, "Blah", 1, owner = null ,acls = None) + AdminUtils.addPartitions(zkClient, "Blah", 1) fail("Topic should not exist") } catch { case e: AdminOperationException => //this is good @@ -72,7 +72,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testWrongReplicaCount { try { - AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2", owner = null ,acls = None) + AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2") fail("Add partitions should fail") } catch { case e: AdminOperationException => //this is good @@ -81,7 +81,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testIncrementPartitions { - AdminUtils.addPartitions(zkClient, topic1, 3, owner = null ,acls = None) + AdminUtils.addPartitions(zkClient, topic1, 3) // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2) @@ -106,7 +106,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testManualAssignmentOfReplicas { - AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3", owner = null ,acls = None) + AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3") // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2) @@ -132,7 +132,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testReplicaPlacement { - AdminUtils.addPartitions(zkClient, topic3, 7, owner = null ,acls = None) + AdminUtils.addPartitions(zkClient, topic3, 7) // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 1) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index 5fadc21..4b728a1 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -25,10 +25,11 @@ import kafka.log._ import kafka.zk.ZooKeeperTestHarness import kafka.utils.{Logging, ZkUtils, TestUtils} import kafka.common.{TopicExistsException, TopicAndPartition} -import kafka.server.{TopicConfig, KafkaServer, KafkaConfig} +import kafka.server.{KafkaServer, KafkaConfig} import java.io.File import TestUtils._ + class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @Test @@ -70,18 +71,18 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // duplicate brokers intercept[IllegalArgumentException] { - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,0)), owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,0))) } // inconsistent replication factor intercept[IllegalArgumentException] { - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,1), 1->Seq(0)), owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,1), 1->Seq(0))) } // good assignment val assignment = Map(0 -> List(0, 1, 2), 1 -> List(1, 2, 3)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", assignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", assignment) val found = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq("test")) assertEquals(assignment, found("test")) } @@ -119,7 +120,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val topic = "test" TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3, 4)) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // create leaders for all partitions TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1) val actualReplicaList = leaderForPartitionMap.keys.toArray.map(p => (p -> ZkUtils.getReplicasForPartition(zkClient, topic, p))).toMap @@ -129,7 +130,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { intercept[TopicExistsException] { // shouldn't be able to create a topic that already exists - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) } } @@ -146,7 +147,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(0, 2, 3) val partitionToBeReassigned = 0 @@ -177,7 +178,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(1, 2, 3) val partitionToBeReassigned = 0 @@ -208,7 +209,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(2, 3) val partitionToBeReassigned = 0 @@ -252,7 +253,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1)) val topic = "test" // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // put the partition in the reassigned path as well // reassign partition 0 val newReplicas = Seq(0, 1) @@ -299,7 +300,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // broker 2 should be the leader since it was started first val currentLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, oldLeaderOpt = None).get @@ -369,7 +370,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { def makeConfig(messageSize: Int, retentionMs: Long) = { var props = new Properties() props.setProperty(LogConfig.MaxMessageBytesProp, messageSize.toString) - props.setProperty(LogConfig.RententionMsProp, retentionMs.toString) + props.setProperty(LogConfig.RetentionMsProp, retentionMs.toString) props } @@ -392,7 +393,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { checkConfig(maxMessageSize, retentionMs) // now double the config values for the topic and check that it is applied - AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs), null, null) + AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs)) checkConfig(2*maxMessageSize, 2 * retentionMs) } finally { server.shutdown() @@ -400,27 +401,4 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { } } - /** - * Test we can support both V1 and V2 configs. - */ - @Test - def testTopicConfigV1isSupported() { - val topic: String = "test-topic" - val server = TestUtils.createServer(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) - - - //Write and read a V1 format config. - val props: Properties = new Properties() - props.put("test", "test") - - val configMap: scala.collection.mutable.Map[String, String] = { - import scala.collection.JavaConversions._ - props - } - - val map: Map[String, Any]=Map[String, Any] (TopicConfig.versionKey -> 1, TopicConfig.configKey -> configMap) - ZkUtils.updatePersistentPath(server.zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map)) - assertEquals(props, AdminUtils.fetchTopicConfig(server.zkClient, topic)) - } - } diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index d013ad5..61cc602 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -102,7 +102,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val allServers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) val servers = allServers.filter(s => expectedReplicaAssignment(0).contains(s.config.brokerId)) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // wait until replica log is created on every broker TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isDefined), @@ -147,7 +147,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val newPartition = TopicAndPartition(topic, 1) follower.shutdown() // add partitions to topic - AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2", false, owner = null ,acls = None) + AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2", false) // start topic deletion AdminUtils.deleteTopic(zkClient, topic) follower.startup() @@ -169,7 +169,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { AdminUtils.deleteTopic(zkClient, topic) // add partitions to topic val newPartition = TopicAndPartition(topic, 1) - AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2", owner = null ,acls = None) + AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2") TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // verify that new partition doesn't exist on any broker either assertTrue("Replica logs not deleted after delete topic is complete", @@ -187,7 +187,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { AdminUtils.deleteTopic(zkClient, topic) TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // re-create topic on same replicas - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // wait until leader is elected val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) assertTrue("New leader should be elected after re-creating topic test", leaderIdOpt.isDefined) @@ -265,7 +265,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // create brokers val servers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment, owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // wait until replica log is created on every broker TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isDefined), diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index f8bc163..c7136f2 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -17,13 +17,12 @@ package kafka.admin import junit.framework.Assert._ -import kafka.security.auth.{Operation, PermissionType, Acl} import org.junit.Test import org.scalatest.junit.JUnit3Suite import kafka.utils.Logging import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness -import kafka.server.{TopicConfig, OffsetManager} +import kafka.server.{OffsetManager, KafkaConfig} import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils @@ -35,52 +34,31 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin val numPartitionsOriginal = 1 val cleanupKey = "cleanup.policy" val cleanupVal = "compact" - - val acl1: Acl = new Acl("alice", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ, Operation.WRITE)) - val acl2: Acl = new Acl("bob", PermissionType.ALLOW, Set[String]("*"), Set[Operation](Operation.READ, Operation.WRITE)) - val acl3: Acl = new Acl("bob", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ)) - // create brokers val brokers = List(0, 1, 2) - val aclFilePath: String = Thread.currentThread().getContextClassLoader.getResource("acl.json").getPath - TestUtils.createBrokersInZk(zkClient, brokers) // create the topic val createOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, "--replication-factor", "1", "--config", cleanupKey + "=" + cleanupVal, - "--topic", topic, - "--acl", aclFilePath)) - + "--topic", topic)) TopicCommand.createTopic(zkClient, createOpts) val props = AdminUtils.fetchTopicConfig(zkClient, topic) - - val topicConfig: TopicConfig = TopicConfig.fromProps(props) assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey)) assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal)) - assertEquals(Set[Acl](acl1, acl2, acl3), topicConfig.acls) - assertEquals(System.getProperty("user.name"), topicConfig.owner) // pre-create the topic config changes path to avoid a NoNodeException ZkUtils.createPersistentPath(zkClient, ZkUtils.TopicConfigChangesPath) // modify the topic to add new partitions val numPartitionsModified = 3 - val testUser: String = "testUser" val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, "--config", cleanupKey + "=" + cleanupVal, - "--owner", testUser, "--topic", topic)) TopicCommand.alterTopic(zkClient, alterOpts) val newProps = AdminUtils.fetchTopicConfig(zkClient, topic) - val newTopicConfig: TopicConfig = TopicConfig.fromProps(newProps) - assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey)) assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal)) - assertEquals(Set[Acl](acl1, acl2, acl3), newTopicConfig.acls) - assertEquals(testUser, newTopicConfig.owner) - - //TODO add test to verify acl can be modified using --acl during alter topic command. } @Test diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index 978f19f..5b7b529 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -29,7 +29,7 @@ import kafka.admin.AdminUtils import kafka.common.FailedToSendMessageException import kafka.consumer.{Consumer, ConsumerConfig, ConsumerTimeoutException} import kafka.producer.{KeyedMessage, Producer} -import kafka.serializer.StringEncoder +import kafka.serializer.StringDecoder import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.CoreUtils import kafka.utils.TestUtils._ @@ -104,8 +104,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { startBrokers(Seq(configProps1, configProps2)) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), - owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2))) verifyUncleanLeaderElectionEnabled } @@ -117,8 +116,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { startBrokers(Seq(configProps1, configProps2)) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), - owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2))) verifyUncleanLeaderElectionDisabled } @@ -133,7 +131,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val topicProps = new Properties() topicProps.put("unclean.leader.election.enable", String.valueOf(true)) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), - topicProps, owner = null ,acls = None) + topicProps) verifyUncleanLeaderElectionEnabled } @@ -148,7 +146,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val topicProps = new Properties() topicProps.put("unclean.leader.election.enable", String.valueOf(false)) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), - topicProps, owner = null ,acls = None) + topicProps) verifyUncleanLeaderElectionDisabled } @@ -161,8 +159,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { topicProps.put("unclean.leader.election.enable", "invalid") intercept[ConfigException] { - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1)), topicProps, - owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1)), topicProps) } } @@ -178,14 +175,14 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val followerId = if (leaderId == brokerId1) brokerId2 else brokerId1 debug("Follower for " + topic + " is: %s".format(followerId)) - produceMessage(topic, "first") + sendMessage(servers, topic, "first") waitUntilMetadataIsPropagated(servers, topic, partitionId) assertEquals(List("first"), consumeAllMessages(topic)) // shutdown follower server servers.filter(server => server.config.brokerId == followerId).map(server => shutdownServer(server)) - produceMessage(topic, "second") + sendMessage(servers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic)) // shutdown leader and then restart follower @@ -195,7 +192,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // wait until new leader is (uncleanly) elected waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(followerId)) - produceMessage(topic, "third") + sendMessage(servers, topic, "third") // second message was lost due to unclean election assertEquals(List("first", "third"), consumeAllMessages(topic)) @@ -213,14 +210,14 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val followerId = if (leaderId == brokerId1) brokerId2 else brokerId1 debug("Follower for " + topic + " is: %s".format(followerId)) - produceMessage(topic, "first") + sendMessage(servers, topic, "first") waitUntilMetadataIsPropagated(servers, topic, partitionId) assertEquals(List("first"), consumeAllMessages(topic)) // shutdown follower server servers.filter(server => server.config.brokerId == followerId).map(server => shutdownServer(server)) - produceMessage(topic, "second") + sendMessage(servers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic)) // shutdown leader and then restart follower @@ -232,7 +229,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // message production and consumption should both fail while leader is down intercept[FailedToSendMessageException] { - produceMessage(topic, "third") + sendMessage(servers, topic, "third") } assertEquals(List.empty[String], consumeAllMessages(topic)) @@ -240,7 +237,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { servers.filter(server => server.config.brokerId == leaderId).map(server => server.startup()) waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(leaderId)) - produceMessage(topic, "third") + sendMessage(servers, topic, "third") waitUntilMetadataIsPropagated(servers, topic, partitionId) servers.filter(server => server.config.brokerId == leaderId).map(server => shutdownServer(server)) @@ -256,33 +253,16 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { server.awaitShutdown() } - private def produceMessage(topic: String, message: String) = { - val producer: Producer[String, Array[Byte]] = createProducer( - getBrokerListStrFromServers(servers), - keyEncoder = classOf[StringEncoder].getName) - producer.send(new KeyedMessage[String, Array[Byte]](topic, topic, message.getBytes)) - producer.close() - } - private def consumeAllMessages(topic: String) : List[String] = { // use a fresh consumer group every time so that we don't need to mess with disabling auto-commit or // resetting the ZK offset val consumerProps = createConsumerProperties(zkConnect, "group" + random.nextLong, "id", 1000) val consumerConnector = Consumer.create(new ConsumerConfig(consumerProps)) - val messageStream = consumerConnector.createMessageStreams(Map(topic -> 1))(topic).head - - val messages = new MutableList[String] - val iter = messageStream.iterator - try { - while(iter.hasNext()) { - messages += new String(iter.next.message) // will throw a timeout exception if the message isn't there - } - } catch { - case e: ConsumerTimeoutException => - debug("consumer timed out after receiving " + messages.length + " message(s).") - } finally { - consumerConnector.shutdown - } - messages.toList + val messageStream = consumerConnector.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) + + val messages = getMessages(messageStream) + consumerConnector.shutdown + + messages } } diff --git a/core/src/test/scala/unit/kafka/security/auth/AclTest.scala b/core/src/test/scala/unit/kafka/security/auth/AclTest.scala deleted file mode 100644 index a4827ae..0000000 --- a/core/src/test/scala/unit/kafka/security/auth/AclTest.scala +++ /dev/null @@ -1,24 +0,0 @@ -package unit.kafka.security.auth - -import kafka.security.auth.{Operation, PermissionType, Acl} -import kafka.utils.Json -import org.junit.{Assert, Test} -import org.scalatest.junit.JUnit3Suite - -/** - * Created by pbrahmbhatt on 3/26/15. - */ -class AclTest extends JUnit3Suite { - - @Test - def testParsing(): Unit = { - val acl1: Acl = new Acl("alice", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ, Operation.WRITE)) - val acl2: Acl = new Acl("bob", PermissionType.ALLOW, Set[String]("*"), Set[Operation](Operation.READ, Operation.WRITE)) - val acl3: Acl = new Acl("bob", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ)) - - val acls: Set[Acl] = Set[Acl](acl1, acl2, acl3) - val jsonAcls: String = Json.encode(Acl.toJsonCompatibleMap(acls)) - - Assert.assertEquals(acls, Acl.fromJson(jsonAcls)) - } -} diff --git a/core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala b/core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala deleted file mode 100644 index fa93efb..0000000 --- a/core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala +++ /dev/null @@ -1,22 +0,0 @@ -package unit.kafka.security.auth - -import kafka.security.auth.{ClusterAclCache, Acl, Operation, PermissionType} -import org.junit.{Assert, Test} -import org.scalatest.junit.JUnit3Suite - -/** - * Created by pbrahmbhatt on 4/6/15. - */ -class ClusterAclTest extends JUnit3Suite { - - @Test - def testClusterAcl(){ - val acl1: Acl = new Acl("alice", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ, Operation.WRITE)) - val acl2: Acl = new Acl("bob", PermissionType.ALLOW, Set[String]("*"), Set[Operation](Operation.READ, Operation.WRITE)) - val acl3: Acl = new Acl("bob", PermissionType.DENY, Set[String]("host1","host2"), Set[Operation](Operation.READ)) - - val acls: Set[Acl] = Set[Acl](acl1, acl2, acl3) - - Assert.assertEquals(Set[Acl](acl1, acl2, acl3), new ClusterAclCache(Thread.currentThread().getContextClassLoader.getResource("acl.json").getPath).clusterAcl) - } -} diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 2bb4ce6..7877f6c 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -38,19 +38,10 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { val logOpt = this.servers(0).logManager.getLog(tp) assertTrue(logOpt.isDefined) assertEquals(oldVal, logOpt.get.config.flushInterval) - - //check config cache gets populated for a new topic. - val config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) - assertEquals(oldVal, config.logConfig.flushInterval) } - - AdminUtils.changeTopicConfig(zkClient, tp.topic, LogConfig(flushInterval = newVal).toProps, null, null) + AdminUtils.changeTopicConfig(zkClient, tp.topic, LogConfig(flushInterval = newVal).toProps) TestUtils.retry(10000) { assertEquals(newVal, this.servers(0).logManager.getLog(tp).get.config.flushInterval) - - //check config cache was updated with the new values. - val config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) - assertEquals(newVal, config.logConfig.flushInterval) } } @@ -58,7 +49,7 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { def testConfigChangeOnNonExistingTopic() { val topic = TestUtils.tempTopic try { - AdminUtils.changeTopicConfig(zkClient, topic, LogConfig(flushInterval = 10000).toProps, null, null) + AdminUtils.changeTopicConfig(zkClient, topic, LogConfig(flushInterval = 10000).toProps) fail("Should fail with AdminOperationException for topic doesn't exist") } catch { case e: AdminOperationException => // expected diff --git a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala b/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala deleted file mode 100644 index 4b8b613..0000000 --- a/core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala +++ /dev/null @@ -1,50 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package unit.kafka.server - -import junit.framework.Assert._ -import kafka.admin.AdminUtils -import kafka.common.TopicAndPartition -import kafka.integration.KafkaServerTestHarness -import kafka.log.LogConfig -import kafka.server.{KafkaConfig, TopicConfig} -import kafka.utils.TestUtils -import org.scalatest.junit.JUnit3Suite - -class TopicConfigCacheTest extends JUnit3Suite with KafkaServerTestHarness { - - def generateConfigs() = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps(_)) - - def testConfigCache { - var config: TopicConfig = this.servers(0).topicConfigCache.getTopicConfig("not-existing-topic") - assertNull("for non existing topic owner should be null.",config.owner) - assertTrue("for non existing topic acls should be empty.",config.acls.isEmpty) - - //newly created topics should be populated in cache on first request. - val oldVal = 100000 - val tp = TopicAndPartition("test", 0) - AdminUtils.createTopic(zkClient, tp.topic, 1, 1, LogConfig(flushInterval = oldVal).toProps) - config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) - assertEquals(oldVal, config.logConfig.flushInterval) - - //test that addOrupdate works - val newVal = 20000 - this.servers(0).topicConfigCache.addOrUpdateTopicConfig(tp.topic, LogConfig(flushInterval = newVal).toProps) - config = this.servers(0).topicConfigCache.getTopicConfig(tp.topic) - assertEquals(newVal, config.logConfig.flushInterval) - } -} diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 3e6828a..8dc99b6 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -36,7 +36,7 @@ import kafka.producer._ import kafka.message._ import kafka.api._ import kafka.cluster.Broker -import kafka.consumer.{KafkaStream, ConsumerConfig} +import kafka.consumer.{ConsumerTimeoutException, KafkaStream, ConsumerConfig} import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition import kafka.admin.AdminUtils @@ -189,8 +189,7 @@ object TestUtils extends Logging { def createTopic(zkClient: ZkClient, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]], servers: Seq[KafkaServer]) : scala.collection.immutable.Map[Int, Option[Int]] = { // create topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaAssignment, - owner = null ,acls = None) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaAssignment) // wait until the update metadata request for new topic reaches all servers partitionReplicaAssignment.keySet.map { case i => TestUtils.waitUntilMetadataIsPropagated(servers, topic, i) @@ -746,71 +745,97 @@ object TestUtils extends Logging { time = time, brokerState = new BrokerState()) } - - def sendMessagesToPartition(servers: Seq[KafkaServer], - topic: String, - partition: Int, - numMessages: Int, - compression: CompressionCodec = NoCompressionCodec): List[String] = { + def sendMessages(servers: Seq[KafkaServer], + topic: String, + numMessages: Int, + partition: Int = -1, + compression: CompressionCodec = NoCompressionCodec): List[String] = { val header = "test-%d".format(partition) val props = new Properties() props.put("compression.codec", compression.codec.toString) - val producer: Producer[Int, String] = - createProducer(TestUtils.getBrokerListStrFromServers(servers), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName, - partitioner = classOf[FixedValuePartitioner].getName, - producerProps = props) - val ms = 0.until(numMessages).map(x => header + "-" + x) - producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) - debug("Sent %d messages for partition [%s,%d]".format(ms.size, topic, partition)) - producer.close() - ms.toList - } - def sendMessages(servers: Seq[KafkaServer], - topic: String, - producerId: String, - messagesPerNode: Int, - header: String, - compression: CompressionCodec, - numParts: Int): List[String]= { - var messages: List[String] = Nil - val props = new Properties() - props.put("compression.codec", compression.codec.toString) - props.put("client.id", producerId) - val producer: Producer[Int, String] = - createProducer(brokerList = TestUtils.getBrokerListStrFromServers(servers), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName, - partitioner = classOf[FixedValuePartitioner].getName, - producerProps = props) + // Specific Partition + if (partition >= 0) { + val producer: Producer[Int, String] = + createProducer(TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[IntEncoder].getName, + partitioner = classOf[FixedValuePartitioner].getName, + producerProps = props) - for (partition <- 0 until numParts) { - val ms = 0.until(messagesPerNode).map(x => header + "-" + partition + "-" + x) producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) - messages ++= ms debug("Sent %d messages for partition [%s,%d]".format(ms.size, topic, partition)) + producer.close() + ms.toList + } else { + // Use topic as the key to determine partition + val producer: Producer[String, String] = createProducer( + TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[DefaultPartitioner].getName, + producerProps = props) + producer.send(ms.map(m => new KeyedMessage[String, String](topic, topic, m)):_*) + producer.close() + debug("Sent %d messages for topic [%s]".format(ms.size, topic)) + ms.toList } + + } + + def sendMessage(servers: Seq[KafkaServer], + topic: String, + message: String) = { + + val producer: Producer[String, String] = + createProducer(TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName(), + keyEncoder = classOf[StringEncoder].getName()) + + producer.send(new KeyedMessage[String, String](topic, topic, message)) producer.close() - messages } - def getMessages(nMessagesPerThread: Int, - topicMessageStreams: Map[String, List[KafkaStream[String, String]]]): List[String] = { + /** + * Consume all messages (or a specific number of messages) + * @param topicMessageStreams the Topic Message Streams + * @param nMessagesPerThread an optional field to specify the exact number of messages to be returned. + * ConsumerTimeoutException will be thrown if there are no messages to be consumed. + * If not specified, then all available messages will be consumed, and no exception is thrown. + * + * + * @return the list of messages consumed. + */ + def getMessages(topicMessageStreams: Map[String, List[KafkaStream[String, String]]], + nMessagesPerThread: Int = -1): List[String] = { + var messages: List[String] = Nil + val shouldGetAllMessages = nMessagesPerThread < 0 for ((topic, messageStreams) <- topicMessageStreams) { for (messageStream <- messageStreams) { - val iterator = messageStream.iterator - for (i <- 0 until nMessagesPerThread) { - assertTrue(iterator.hasNext) - val message = iterator.next.message - messages ::= message - debug("received message: " + message) + val iterator = messageStream.iterator() + try { + var i = 0 + while ((shouldGetAllMessages && iterator.hasNext()) || (i < nMessagesPerThread)) { + assertTrue(iterator.hasNext) + val message = iterator.next.message // will throw a timeout exception if the message isn't there + messages ::= message + debug("received message: " + message) + i += 1 + } + } catch { + case e: ConsumerTimeoutException => + if (shouldGetAllMessages) { + // swallow the exception + debug("consumer timed out after receiving " + messages.length + " message(s).") + } else { + throw e + } } } } + messages.reverse } -- 1.9.5 (Apple Git-50.3) From 3f1c805d252fa33635e9251dc87efe970a3267b7 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 21 Apr 2015 12:02:27 -0700 Subject: [PATCH 8/9] public classes and interfaces to support pluggable authorizer implementation for kafka --- core/src/main/scala/kafka/security/auth/Acl.scala | 19 ++++++---- .../scala/kafka/security/auth/Authorizer.scala | 44 ++++++++++++++++++++-- .../scala/kafka/security/auth/KafkaPrincipal.scala | 36 ++++++++++++++++++ .../main/scala/kafka/security/auth/Operation.java | 16 +++++++- .../main/scala/kafka/security/auth/Resource.scala | 9 +++++ 5 files changed, 112 insertions(+), 12 deletions(-) create mode 100644 core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala create mode 100644 core/src/main/scala/kafka/security/auth/Resource.scala diff --git a/core/src/main/scala/kafka/security/auth/Acl.scala b/core/src/main/scala/kafka/security/auth/Acl.scala index 1a7960b..e596e54 100644 --- a/core/src/main/scala/kafka/security/auth/Acl.scala +++ b/core/src/main/scala/kafka/security/auth/Acl.scala @@ -1,11 +1,13 @@ package kafka.security.auth +import java.security.Principal + import kafka.utils.Json import scala.collection.{mutable} object Acl { - val wildCardPrincipal: String = "*" + val wildCardPrincipal: KafkaPrincipal = new KafkaPrincipal("user", "*") val wildCardHost: String = "*" val allowAllAcl = new Acl(wildCardPrincipal, PermissionType.ALLOW, Set[String](wildCardHost), Set[Operation](Operation.ALL)) val PRINCIPAL_KEY = "principal" @@ -16,7 +18,8 @@ object Acl { val CURRENT_VERSION = 1 val ACLS_KEY = "acls" - def fromJson(aclJson: String): Set[Acl] = { + + private[auth] def fromJson(aclJson: String): Set[Acl] = { if(aclJson == null || aclJson.isEmpty) { return collection.immutable.Set.empty[Acl] } @@ -32,14 +35,14 @@ object Acl { val permissionType: PermissionType = PermissionType.valueOf(item(PERMISSION_TYPE_KEY).asInstanceOf[String]) val operations: List[Operation] = item(OPERATIONS_KEY).asInstanceOf[List[String]].map(operation => Operation.valueOf(operation)) val hosts: List[String] = item(HOSTS_KEY).asInstanceOf[List[String]] - acls += new Acl(principal, permissionType, hosts.toSet, operations.toSet) + acls += new Acl(new KafkaPrincipal(principal), permissionType, hosts.toSet, operations.toSet) }) case None => } return acls.toSet } - def toJsonCompatibleMap(acls: Set[Acl]): Map[String,Any] = { + private[auth] def toJsonCompatibleMap(acls: Set[Acl]): Map[String,Any] = { acls match { case aclSet: Set[Acl] => Map(Acl.VERSION_KEY -> Acl.CURRENT_VERSION, Acl.ACLS_KEY -> aclSet.map(acl => acl.toMap).toList) case _ => null @@ -52,12 +55,12 @@ object Acl { *
  * Principal P has permissionType PT on Operations O1,O2 from hosts H1,H2.
  * 
- * @param principal A value of * indicates all users. + * @param principal A value of *:* indicates all users. * @param permissionType * @param hosts A value of * indicates all hosts. * @param operations A value of ALL indicates all operations. */ -case class Acl(val principal: String,val permissionType: PermissionType,val hosts: Set[String],val operations: Set[Operation]) { +case class Acl(val principal: KafkaPrincipal,val permissionType: PermissionType,val hosts: Set[String],val operations: Set[Operation]) { /** * TODO: Ideally we would have a symmetric toJson method but our current json library fails to decode double parsed json strings so @@ -65,9 +68,9 @@ case class Acl(val principal: String,val permissionType: PermissionType,val host * Convert an acl instance to a map * @return Map representation of the Acl. */ - def toMap() : Map[String, Any] = { + private[auth] def toMap() : Map[String, Any] = { val map: mutable.HashMap[String, Any] = new mutable.HashMap[String, Any]() - map.put(Acl.PRINCIPAL_KEY, principal) + map.put(Acl.PRINCIPAL_KEY, principal.toString) map.put(Acl.PERMISSION_TYPE_KEY, permissionType.name()) map.put(Acl.OPERATIONS_KEY, operations.map(operation => operation.name())) map.put(Acl.HOSTS_KEY, hosts) diff --git a/core/src/main/scala/kafka/security/auth/Authorizer.scala b/core/src/main/scala/kafka/security/auth/Authorizer.scala index 5a4a7da..f2a236a 100644 --- a/core/src/main/scala/kafka/security/auth/Authorizer.scala +++ b/core/src/main/scala/kafka/security/auth/Authorizer.scala @@ -18,7 +18,7 @@ package kafka.security.auth import kafka.network.RequestChannel.Session -import kafka.server.{TopicConfigCache, MetadataCache, KafkaConfig} +import kafka.server.KafkaConfig /** * Top level interface that all plugable authorizer must implement. Kafka server will read "authorizer.class" config @@ -33,7 +33,7 @@ trait Authorizer { /** * Guaranteed to be called before any authorize call is made. */ - def initialize(kafkaConfig: KafkaConfig, topicConfigCache: TopicConfigCache): Unit + def initialize(kafkaConfig: KafkaConfig): Unit /** * @param session The session being authenticated. @@ -41,5 +41,43 @@ trait Authorizer { * @param resource Resource the client is trying to access. * @return */ - def authorize(session: Session, operation: Operation, resource: String): Boolean + def authorize(session: Session, operation: Operation, resource: Resource): Boolean + + /** + * add the acls to resource, this is an additive operation so existing acls will not be overwritten, instead these new + * acls will be added to existing acls. + * @param acls set of acls to add to existing acls + * @param resource the resource to which these acls should be attached. + */ + def addAcls(acls: Set[Acl], resource: Resource): Unit + + /** + * remove these acls from the resource. + * @param acls set of acls to be removed. + * @param resource resource from which the acls should be removed. + * @return true if some acl got removed, false if no acl was removed. + */ + def removeAcls(acls: Set[Acl], resource: Resource): Boolean + + /** + * remove a resource along with all of its acls from acl store. + * @param resource + * @return + */ + def removeAcls(resource: Resource): Boolean + + /** + * get set of acls for this resource + * @param resource + * @return empty set if no acls are found, otherwise the acls for the resource. + */ + def getAcls(resource: Resource): Set[Acl] + + /** + * get the acls for this principal. + * @param principal + * @return empty set if no acls exist for this principal, otherwise the acls for the principal. + */ + def getAcls(principal: KafkaPrincipal): Set[Acl] } + diff --git a/core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala b/core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala new file mode 100644 index 0000000..15583a9 --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala @@ -0,0 +1,36 @@ +package kafka.security.auth + +import java.security.Principal + +object KafkaPrincipal { + val seperator: String = ":" + + private[auth] def fromString(str: String) : KafkaPrincipal = { + val arr: Array[String] = str.split(seperator) + + if(arr.length != 2) { + throw new IllegalArgumentException("expected a string in format principalType:principalName but got " + str) + } + + KafkaPrincipal(arr(0), arr(1)) + } +} + +/** + * + * @param principalType type of principal user,unixgroup, ldapgroup. + * @param name name of the principal + */ +case class KafkaPrincipal(principalType: String, name: String) extends Principal { + + override def getName: String = { + name + } + + override def toString: String = { + principalType + KafkaPrincipal.seperator + name + } +} + + + diff --git a/core/src/main/scala/kafka/security/auth/Operation.java b/core/src/main/scala/kafka/security/auth/Operation.java index cac5b85..ea0701a 100644 --- a/core/src/main/scala/kafka/security/auth/Operation.java +++ b/core/src/main/scala/kafka/security/auth/Operation.java @@ -27,5 +27,19 @@ public enum Operation { EDIT, DESCRIBE, SEND_CONTROL_MSG, - ALL + ALL; + + /** + * method defined for case insensitive check. the default value of method is case sensitive + */ + public static Operation fromString(String operationName) { + if(operationName != null) { + for(Operation operation: Operation.values()) { + if(operationName.equalsIgnoreCase(operation.name())) { + return operation; + } + } + } + throw new IllegalArgumentException("no matching enum value fonund for " + operationName); + } } diff --git a/core/src/main/scala/kafka/security/auth/Resource.scala b/core/src/main/scala/kafka/security/auth/Resource.scala new file mode 100644 index 0000000..a0e91d0 --- /dev/null +++ b/core/src/main/scala/kafka/security/auth/Resource.scala @@ -0,0 +1,9 @@ +package kafka.security.auth + +object Resource { + val clusterResource: Resource = new Resource("KafkaCluster") +} + +case class Resource(val name: String) { + if(name == null || name.isEmpty) throw new IllegalArgumentException("resource name can not be null or empty") +} -- 1.9.5 (Apple Git-50.3) From f204ebd794c5948028e32754ba050cb5ece18986 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 21 Apr 2015 22:09:03 -0700 Subject: [PATCH 9/9] all public entities for pluggable authorizer support in kafka. --- core/src/main/scala/kafka/security/auth/Acl.scala | 23 ++- .../scala/kafka/security/auth/Authorizer.scala | 12 +- .../scala/kafka/security/auth/KafkaPrincipal.scala | 18 ++- .../main/scala/kafka/security/auth/Operation.java | 6 +- .../main/scala/kafka/security/auth/Resource.scala | 9 -- .../kafka/security/auth/SimpleAclAuthorizer.scala | 94 ------------ core/src/main/scala/kafka/server/KafkaApis.scala | 21 +-- core/src/main/scala/kafka/server/KafkaConfig.scala | 8 - core/src/main/scala/kafka/server/KafkaServer.scala | 126 ++++++++-------- .../security/auth/SimpleAclAuthorizerTest.scala | 166 --------------------- .../kafka/server/KafkaConfigConfigDefTest.scala | 1 - 11 files changed, 115 insertions(+), 369 deletions(-) delete mode 100644 core/src/main/scala/kafka/security/auth/Resource.scala delete mode 100644 core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala delete mode 100644 core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala diff --git a/core/src/main/scala/kafka/security/auth/Acl.scala b/core/src/main/scala/kafka/security/auth/Acl.scala index e596e54..e2c9d03 100644 --- a/core/src/main/scala/kafka/security/auth/Acl.scala +++ b/core/src/main/scala/kafka/security/auth/Acl.scala @@ -1,9 +1,23 @@ -package kafka.security.auth +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ -import java.security.Principal +package kafka.security.auth import kafka.utils.Json - import scala.collection.{mutable} object Acl { @@ -18,7 +32,6 @@ object Acl { val CURRENT_VERSION = 1 val ACLS_KEY = "acls" - private[auth] def fromJson(aclJson: String): Set[Acl] = { if(aclJson == null || aclJson.isEmpty) { return collection.immutable.Set.empty[Acl] @@ -35,7 +48,7 @@ object Acl { val permissionType: PermissionType = PermissionType.valueOf(item(PERMISSION_TYPE_KEY).asInstanceOf[String]) val operations: List[Operation] = item(OPERATIONS_KEY).asInstanceOf[List[String]].map(operation => Operation.valueOf(operation)) val hosts: List[String] = item(HOSTS_KEY).asInstanceOf[List[String]] - acls += new Acl(new KafkaPrincipal(principal), permissionType, hosts.toSet, operations.toSet) + acls += new Acl(KafkaPrincipal.fromString(principal), permissionType, hosts.toSet, operations.toSet) }) case None => } diff --git a/core/src/main/scala/kafka/security/auth/Authorizer.scala b/core/src/main/scala/kafka/security/auth/Authorizer.scala index f2a236a..bf8f230 100644 --- a/core/src/main/scala/kafka/security/auth/Authorizer.scala +++ b/core/src/main/scala/kafka/security/auth/Authorizer.scala @@ -34,14 +34,14 @@ trait Authorizer { * Guaranteed to be called before any authorize call is made. */ def initialize(kafkaConfig: KafkaConfig): Unit - + /** * @param session The session being authenticated. * @param operation Type of operation client is trying to perform on resource. * @param resource Resource the client is trying to access. * @return */ - def authorize(session: Session, operation: Operation, resource: Resource): Boolean + def authorize(session: Session, operation: Operation, resource: String): Boolean /** * add the acls to resource, this is an additive operation so existing acls will not be overwritten, instead these new @@ -49,7 +49,7 @@ trait Authorizer { * @param acls set of acls to add to existing acls * @param resource the resource to which these acls should be attached. */ - def addAcls(acls: Set[Acl], resource: Resource): Unit + def addAcls(acls: Set[Acl], resource: String): Unit /** * remove these acls from the resource. @@ -57,21 +57,21 @@ trait Authorizer { * @param resource resource from which the acls should be removed. * @return true if some acl got removed, false if no acl was removed. */ - def removeAcls(acls: Set[Acl], resource: Resource): Boolean + def removeAcls(acls: Set[Acl], resource: String): Boolean /** * remove a resource along with all of its acls from acl store. * @param resource * @return */ - def removeAcls(resource: Resource): Boolean + def removeAcls(resource: String): Boolean /** * get set of acls for this resource * @param resource * @return empty set if no acls are found, otherwise the acls for the resource. */ - def getAcls(resource: Resource): Set[Acl] + def getAcls(resource: String): Set[Acl] /** * get the acls for this principal. diff --git a/core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala b/core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala index 15583a9..d54242c 100644 --- a/core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala +++ b/core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala @@ -1,3 +1,19 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package kafka.security.auth import java.security.Principal @@ -5,7 +21,7 @@ import java.security.Principal object KafkaPrincipal { val seperator: String = ":" - private[auth] def fromString(str: String) : KafkaPrincipal = { + def fromString(str: String) : KafkaPrincipal = { val arr: Array[String] = str.split(seperator) if(arr.length != 2) { diff --git a/core/src/main/scala/kafka/security/auth/Operation.java b/core/src/main/scala/kafka/security/auth/Operation.java index ea0701a..fbbf726 100644 --- a/core/src/main/scala/kafka/security/auth/Operation.java +++ b/core/src/main/scala/kafka/security/auth/Operation.java @@ -26,11 +26,11 @@ public enum Operation { DELETE, EDIT, DESCRIBE, - SEND_CONTROL_MSG, + CLUSTER_ACTION, ALL; /** - * method defined for case insensitive check. the default value of method is case sensitive + * method defined for case insensitive check. the default valueOf() method is case sensitive */ public static Operation fromString(String operationName) { if(operationName != null) { @@ -40,6 +40,6 @@ public enum Operation { } } } - throw new IllegalArgumentException("no matching enum value fonund for " + operationName); + throw new IllegalArgumentException("no matching enum value found for " + operationName); } } diff --git a/core/src/main/scala/kafka/security/auth/Resource.scala b/core/src/main/scala/kafka/security/auth/Resource.scala deleted file mode 100644 index a0e91d0..0000000 --- a/core/src/main/scala/kafka/security/auth/Resource.scala +++ /dev/null @@ -1,9 +0,0 @@ -package kafka.security.auth - -object Resource { - val clusterResource: Resource = new Resource("KafkaCluster") -} - -case class Resource(val name: String) { - if(name == null || name.isEmpty) throw new IllegalArgumentException("resource name can not be null or empty") -} diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala deleted file mode 100644 index d25eb83..0000000 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ /dev/null @@ -1,94 +0,0 @@ -package kafka.security.auth - -import kafka.network.RequestChannel.Session -import kafka.server.{KafkaConfig, TopicConfigCache} -import kafka.utils.Logging - -class SimpleAclAuthorizer extends Authorizer with Logging { - - val topicOperations: Set[Operation] = Set[Operation](Operation.READ, Operation.WRITE, Operation.DESCRIBE, Operation.EDIT) - val supportedOperations: Set[Operation] = topicOperations ++ Set[Operation](Operation.SEND_CONTROL_MSG, Operation.CREATE, Operation.DELETE) - var clusterAclCache: ClusterAclCache = null - var configCache: TopicConfigCache = null - var superUsers: Set[String] = null - - override def authorize(session: Session, operation: Operation, resource: String): Boolean = { - //can we assume session, principal and host will never be null? - if(session == null || session.principal == null || session.host == null) { - debug("session, session.principal and session.host can not be null, , programming error so failing open.") - return true - } - - if(!supportedOperations.contains(operation)) { - debug("SimpleAclAuthorizer only supports " + supportedOperations + " but was invoked with operation = " + operation - + " for session = "+ session + " and resource = " + resource + ", programming error so failing open.") - return true - } - - val principalName: String = session.principal.getName - val remoteAddress: String = session.host - - if(superUsers.contains(principalName)) { - debug("principal = " + principalName + " is a super user, allowing operation without checking acls.") - return true - } - - if(topicOperations.contains(operation) && (resource == null || resource.isEmpty)){ - debug("resource is null or empty for a topic operation " + operation + " for session = "+ session + ", " + - "programming error so failing open.") - return true - } - - val owner: String = if(topicOperations.contains(operation)) configCache.getTopicConfig(resource).owner else null - val acls: Set[Acl] = if(topicOperations.contains(operation)) configCache.getTopicConfig(resource).acls else clusterAclCache.clusterAcl - - if(principalName.equalsIgnoreCase(owner)) { - debug("principal = " + principalName + " is owner of the resource " + resource + ", allowing operation without checking acls.") - return true - } - - if(acls == null || acls.isEmpty) { - debug("No acl found. For backward compatibility when we find no acl we assume access to everyone , authorization failing open.") - return true - } - - //first check if there is any Deny acl that would disallow this operation. - for(acl: Acl <- acls) { - if(acl.permissionType.equals(PermissionType.DENY) - && (acl.principal.equalsIgnoreCase(principalName) || acl.principal.equalsIgnoreCase(Acl.wildCardPrincipal)) - && (acl.operations.contains(operation) || acl.operations.contains(Operation.ALL)) - && (acl.hosts.contains(remoteAddress) || acl.hosts.contains(Acl.wildCardHost))) { - debug("denying operation = " + operation + " on resource = " + resource + " to session = " + session + " based on acl = " + acl) - return false - } - } - - //now check if there is any allow acl that will allow this operation. - for(acl: Acl <- acls) { - if(acl.permissionType.equals(PermissionType.ALLOW) - && (acl.principal.equalsIgnoreCase(principalName) || acl.principal.equalsIgnoreCase(Acl.wildCardPrincipal)) - && (acl.operations.contains(operation) || acl.operations.contains(Operation.ALL)) - && (acl.hosts.contains(remoteAddress) || acl.hosts.contains(Acl.wildCardHost))) { - debug("allowing operation = " + operation + " on resource = " + resource + " to session = " + session + " based on acl = " + acl) - return true - } - } - - //We have some acls defined and they do not specify any allow ACL for the current session, reject request. - debug("principal = " + principalName + " is not allowed to perform operation = " + operation + - " from host = " + remoteAddress + " on resource = " + resource) - return false - } - - /** - * Guaranteed to be called before any authorize call is made. - */ - override def initialize(kafkaConfig: KafkaConfig, topicConfigCache: TopicConfigCache): Unit = { - clusterAclCache = new ClusterAclCache(kafkaConfig.clusterAclJsonFilePath) - superUsers = kafkaConfig.superUser match { - case null => Set.empty[String] - case (str: String) => str.split(",").map(s => s.trim).toSet - } - configCache = topicConfigCache - } -} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 7f5b6a7..dce37f0 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -19,7 +19,7 @@ package kafka.server import kafka.message.{MessageSet} -import kafka.security.auth.{Operation, Authorizer} +import kafka.security.auth. {Operation, Authorizer} import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.requests.{JoinGroupResponse, JoinGroupRequest, HeartbeatRequest, HeartbeatResponse, ResponseHeader} import org.apache.kafka.common.TopicPartition @@ -52,6 +52,7 @@ class KafkaApis(val requestChannel: RequestChannel, val authorizer: Option[Authorizer]) extends Logging { this.logIdent = "[KafkaApi-%d] ".format(brokerId) + val clusterResourceName: String = "kafka-cluster" /** * Top-level method that handles all requests and multiplexes to the right api @@ -102,7 +103,7 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest] - if(authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + if(authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.CLUSTER_ACTION, null)) { val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, Map.empty, ErrorMapping.AuthorizationCode) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse))) return @@ -125,7 +126,7 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest] - if(authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + if(authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.CLUSTER_ACTION, null)) { val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, Map.empty, ErrorMapping.AuthorizationCode) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse))) return @@ -142,7 +143,7 @@ class KafkaApis(val requestChannel: RequestChannel, if(authorizer.isDefined) { val unauthorizedTopicAndPartition = updateMetadataRequest.partitionStateInfos.filterKeys( - topicAndPartition => !authorizer.get.authorize(request.session, Operation.EDIT, topicAndPartition.topic)).keys + topicAndPartition => !authorizer.get.authorize(request.session, Operation.CLUSTER_ACTION, topicAndPartition.topic)).keys //In this case the response does not allow to selectively report success/failure so if authorization fails, we fail the entire request. if (!unauthorizedTopicAndPartition.isEmpty) { val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId, ErrorMapping.AuthorizationCode) @@ -164,7 +165,7 @@ class KafkaApis(val requestChannel: RequestChannel, val controlledShutdownRequest = request.requestObj.asInstanceOf[ControlledShutdownRequest] if(authorizer.isDefined) { - if (!authorizer.get.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + if (!authorizer.get.authorize(request.session, Operation.CLUSTER_ACTION, null)) { val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId, ErrorMapping.AuthorizationCode, Set.empty) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) return @@ -185,7 +186,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetCommitRequest.requestInfo.partition( - mapEntry => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.DESCRIBE, mapEntry._1.topic)) + mapEntry => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.READ, mapEntry._1.topic)) // the callback for sending an offset commit response def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { @@ -354,7 +355,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.requestInfo.partition( - mapEntry => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.READ, mapEntry._1.topic)) + mapEntry => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.DESCRIBE, mapEntry._1.topic)) val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => PartitionOffsetsResponse(ErrorMapping.AuthorizationCode, Nil)) @@ -502,7 +503,7 @@ class KafkaApis(val requestChannel: RequestChannel, val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] val topics = metadataRequest.topics.toSet - val (authorizedTopics, unauthorizedTopics) = topics.partition(topic => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.READ, topic)) + val (authorizedTopics, unauthorizedTopics) = topics.partition(topic => !authorizer.isDefined || authorizer.get.authorize(request.session, Operation.DESCRIBE, topic)) val unauthorizedTopicMetaData = unauthorizedTopics.map(topic => new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.AuthorizationCode)) @@ -583,7 +584,7 @@ class KafkaApis(val requestChannel: RequestChannel, val respHeader = new ResponseHeader(request.header.correlationId) val (authorizedTopics, unauthorizedTopics) = joinGroupRequest.topics().partition( - topic => (!authorizer.isDefined || authorizer.get.authorize(request.session, Operation.DESCRIBE, topic)) + topic => (!authorizer.isDefined || authorizer.get.authorize(request.session, Operation.READ, topic)) ) val unauthorizedTopicPartition = unauthorizedTopics.map(topic => new TopicPartition(topic, -1)) @@ -610,7 +611,7 @@ class KafkaApis(val requestChannel: RequestChannel, val heartbeatRequest = request.body.asInstanceOf[HeartbeatRequest] val respHeader = new ResponseHeader(request.header.correlationId) - if (authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.SEND_CONTROL_MSG, null)) { + if (authorizer.isDefined && !authorizer.get.authorize(request.session, Operation.CLUSTER_ACTION, clusterResourceName)) { val heartbeatResponse = new HeartbeatResponse(ErrorMapping.AuthorizationCode) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(respHeader, heartbeatResponse))) return diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 8f5e005..5e5b642 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -45,7 +45,6 @@ object Defaults { /************* Authorizer Configuration ***********/ val AuthorizerClassName = "" val SuperUser = "" - val ClusterAclJsonFilePath = "" /** ********* Socket Server Configuration ***********/ val Port = 9092 @@ -152,7 +151,6 @@ object KafkaConfig { /************* Authorizer Configuration ***********/ val AuthorizerClassNameProp = "authorizer.class.name" val SuperUserProp = "super.users" - val ClusterAclJsonFilePathProp = "cluster.acl.json.file.path" /** ********* Socket Server Configuration ***********/ val PortProp = "port" @@ -266,8 +264,6 @@ object KafkaConfig { /************* Authorizer Configuration ***********/ val AuthorizerClassNameDoc = "The authorizer class that should be used for authorization" val SuperUserDoc = "Comman seperated list of users that will have super user access to the cluster and all the topics." - val ClusterAclJsonFilePathDoc = "Path to the json file describing cluster's acl. These acls are used to determine which users" + - "have access to cluster actions like CREATE topic." /** ********* Socket Server Configuration ***********/ val PortDoc = "the port to listen and accept connections on" @@ -410,7 +406,6 @@ object KafkaConfig { /************* Authorizer Configuration ***********/ .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, LOW, AuthorizerClassNameDoc) .define(SuperUserProp, STRING, Defaults.SuperUser, LOW, SuperUserDoc) - .define(ClusterAclJsonFilePathProp, STRING, Defaults.ClusterAclJsonFilePath, LOW, ClusterAclJsonFilePathDoc) /** ********* Socket Server Configuration ***********/ .define(PortProp, INT, Defaults.Port, HIGH, PortDoc) @@ -536,7 +531,6 @@ object KafkaConfig { /************* Authorizer Configuration ***********/ authorizerClassName = parsed.get(AuthorizerClassNameProp).asInstanceOf[String], superUser = parsed.get(SuperUserProp).asInstanceOf[String], - clusterAclJsonFilePath = parsed.get(ClusterAclJsonFilePathProp).asInstanceOf[String], /** ********* Socket Server Configuration ***********/ port = parsed.get(PortProp).asInstanceOf[Int], @@ -682,7 +676,6 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ /************* Authorizer Configuration ***********/ val authorizerClassName: String = Defaults.AuthorizerClassName, val superUser: String = Defaults.SuperUser, - val clusterAclJsonFilePath: String = Defaults.ClusterAclJsonFilePath, /** ********* Socket Server Configuration ***********/ val port: Int = Defaults.Port, @@ -911,7 +904,6 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ /************* Authorizer Configuration ***********/ props.put(AuthorizerClassNameProp, authorizerClassName.toString) props.put(SuperUserProp, superUser.toString) - props.put(ClusterAclJsonFilePathProp, clusterAclJsonFilePath.toString) /** ********* Socket Server Configuration ***********/ props.put(PortProp, port.toString) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index bce8507..2fee322 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -63,8 +63,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var topicConfigManager: TopicConfigManager = null - var topicConfigCache: TopicConfigCache = null - var consumerCoordinator: ConsumerCoordinator = null var kafkaController: KafkaController = null @@ -74,6 +72,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var kafkaHealthcheck: KafkaHealthcheck = null val metadataCache: MetadataCache = new MetadataCache(config.brokerId) + + var zkClient: ZkClient = null val correlationId: AtomicInteger = new AtomicInteger(0) val brokerMetaPropsFile = "meta.properties" @@ -119,76 +119,70 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg this.logIdent = "[Kafka Server " + config.brokerId + "], " socketServer = new SocketServer(config.brokerId, - config.listeners, - config.numNetworkThreads, - config.queuedMaxRequests, - config.socketSendBufferBytes, - config.socketReceiveBufferBytes, - config.socketRequestMaxBytes, - config.maxConnectionsPerIp, - config.connectionsMaxIdleMs, - config.maxConnectionsPerIpOverrides) - socketServer.startup() - - /* start replica manager */ - replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) - replicaManager.startup() - - /* start offset manager */ - offsetManager = createOffsetManager() - - /* start kafka controller */ - kafkaController = new KafkaController(config, zkClient, brokerState) - kafkaController.startup() - - /* start kafka coordinator */ - consumerCoordinator = new ConsumerCoordinator(config, zkClient) - consumerCoordinator.startup() - - /*initialize topic config cache*/ - topicConfigCache = new TopicConfigCache(config.brokerId, zkClient, defaultConfig = config) - - /* Get the authorizer and initialize it if one is specified.*/ - val authorizer: Option[Authorizer] = if(config.authorizerClassName != null && !config.authorizerClassName.isEmpty) { - Option(CoreUtils.createObject(config.authorizerClassName)) - } else { - None - } - - if(authorizer.isDefined) { - authorizer.get.initialize(config, topicConfigCache) - } - - /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, - kafkaController, zkClient, config.brokerId, config, metadataCache, authorizer) - requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) - brokerState.newState(RunningAsBroker) + config.listeners, + config.numNetworkThreads, + config.queuedMaxRequests, + config.socketSendBufferBytes, + config.socketReceiveBufferBytes, + config.socketRequestMaxBytes, + config.maxConnectionsPerIp, + config.connectionsMaxIdleMs, + config.maxConnectionsPerIpOverrides) + socketServer.startup() + + /* start replica manager */ + replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) + replicaManager.startup() + + /* start offset manager */ + offsetManager = createOffsetManager() + + /* start kafka controller */ + kafkaController = new KafkaController(config, zkClient, brokerState) + kafkaController.startup() + + /* start kafka coordinator */ + consumerCoordinator = new ConsumerCoordinator(config, zkClient) + consumerCoordinator.startup() + + /* Get the authorizer and initialize it if one is specified.*/ + val authorizer: Option[Authorizer] = if(config.authorizerClassName != null && !config.authorizerClassName.isEmpty) { + val authZ: Authorizer = CoreUtils.createObject(config.authorizerClassName) + authZ.initialize(config) + Option(authZ) + } else { + None + } - Mx4jLoader.maybeLoad() + /* start processing requests */ + apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, + kafkaController, zkClient, config.brokerId, config, metadataCache, authorizer) + requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) + brokerState.newState(RunningAsBroker) - /* start topic config manager */ - topicConfigManager = new TopicConfigManager(zkClient, logManager, topicConfigCache) - topicConfigManager.startup() + Mx4jLoader.maybeLoad() - /* tell everyone we are alive */ - val listeners = config.advertisedListeners.map {case(protocol, endpoint) => - if (endpoint.port == 0) - (protocol, EndPoint(endpoint.host, socketServer.boundPort(), endpoint.protocolType)) - else - (protocol, endpoint) - } + /* start topic config manager */ + topicConfigManager = new TopicConfigManager(zkClient, logManager) + topicConfigManager.startup() - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, config.zkSessionTimeoutMs, zkClient) - kafkaHealthcheck.startup() + /* tell everyone we are alive */ + val listeners = config.advertisedListeners.map {case(protocol, endpoint) => + if (endpoint.port == 0) + (protocol, EndPoint(endpoint.host, socketServer.boundPort(), endpoint.protocolType)) + else + (protocol, endpoint) + } + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, config.zkSessionTimeoutMs, zkClient) + kafkaHealthcheck.startup() - /* register broker metrics */ - registerStats() + /* register broker metrics */ + registerStats() - shutdownLatch = new CountDownLatch(1) - startupComplete.set(true) - isStartingUp.set(false) - info("started") + shutdownLatch = new CountDownLatch(1) + startupComplete.set(true) + isStartingUp.set(false) + info("started") } } catch { diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala deleted file mode 100644 index b6313d8..0000000 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ /dev/null @@ -1,166 +0,0 @@ -package unit.kafka.security.auth - -import java.security.Principal -import java.util.Properties - -import com.sun.security.auth.UserPrincipal -import kafka.network.RequestChannel.Session -import kafka.security.auth._ -import kafka.server.{KafkaConfig, TopicConfig, TopicConfigCache} -import kafka.utils.TestUtils -import kafka.zk.ZooKeeperTestHarness -import org.easymock.EasyMock -import org.junit.Test -import org.scalatest.junit.JUnit3Suite -import org.junit.Assert._ - - -class SimpleAclAuthorizerTest extends JUnit3Suite with ZooKeeperTestHarness { - - val clusterAclCache: ClusterAclCache = EasyMock.createMock(classOf[ClusterAclCache]) - val topicConfigCache: TopicConfigCache = EasyMock.createMock(classOf[TopicConfigCache]) - val simpleAclAuthorizer: SimpleAclAuthorizer = new SimpleAclAuthorizer - val testPrincipal: Principal = new UserPrincipal(Acl.wildCardPrincipal) - val testHostName: String = "test.host.com" - var session: Session = new Session(testPrincipal, testHostName) - val resource: String = "test-topic" - val superUsers: String = "superuser1, superuser2" - - override def setUp() { - super.setUp() - - val props = TestUtils.createBrokerConfig(0, zkConnect) - props.put(KafkaConfig.ClusterAclJsonFilePathProp, "") - props.put(KafkaConfig.SuperUserProp, superUsers) - - val cfg = KafkaConfig.fromProps(props) - simpleAclAuthorizer.initialize(cfg, topicConfigCache) - } - - def testTopicAcl(): Unit = { - val user1: String = "user1" - val host1: String = "host1" - val host2: String = "host2" - - //user1 has READ access from host1 and host2. - val acl1: Acl = new Acl(user1, PermissionType.ALLOW, Set[String](host1, host2), Set[Operation](Operation.READ)) - - //user1 does not have READ access from host1. - val acl2: Acl = new Acl(user1, PermissionType.DENY, Set[String](host1), Set[Operation](Operation.READ)) - - //user1 has Write access from host1 only. - val acl3: Acl = new Acl(user1, PermissionType.ALLOW, Set[String](host1), Set[Operation](Operation.WRITE)) - - //user1 has DESCRIBE access from all hosts. - val acl4: Acl = new Acl(user1, PermissionType.ALLOW, Set[String](Acl.wildCardHost), Set[Operation](Operation.DESCRIBE)) - - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](acl1, acl2, acl3, acl4)) - EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() - EasyMock.replay(clusterAclCache, topicConfigCache) - - val host1Session: Session = new Session(new UserPrincipal(user1), host1) - val host2Session: Session = new Session(new UserPrincipal(user1), host2) - - assertTrue("User1 should have READ access from host2", simpleAclAuthorizer.authorize(host2Session, Operation.READ, resource)) - assertFalse("User1 should not have READ access from host1 due to denyAcl", simpleAclAuthorizer.authorize(host1Session, Operation.READ, resource)) - assertTrue("User1 should have WRITE access from host1", simpleAclAuthorizer.authorize(host1Session, Operation.WRITE, resource)) - assertFalse("User1 should not have WRITE access from host2 as no allow acl is defined", simpleAclAuthorizer.authorize(host2Session, Operation.WRITE, resource)) - assertTrue("User1 should have DESCRIBE access from host1", simpleAclAuthorizer.authorize(host1Session, Operation.DESCRIBE, resource)) - assertTrue("User1 should have DESCRIBE access from host2", simpleAclAuthorizer.authorize(host2Session, Operation.DESCRIBE, resource)) - assertFalse("User1 should not have edit access from host1", simpleAclAuthorizer.authorize(host1Session, Operation.EDIT, resource)) - assertFalse("User1 should not have edit access from host2", simpleAclAuthorizer.authorize(host2Session, Operation.EDIT, resource)) - - EasyMock.verify(clusterAclCache, topicConfigCache) - } - - @Test - def testDenyTakesPrecedence(): Unit = { - val user: String = "random-user" - val host: String = "random-host" - val session: Session = new Session(new UserPrincipal(user), host) - - val allowAll: Acl = Acl.allowAllAcl - val denyAcl: Acl = new Acl(user, PermissionType.DENY, Set[String](host), Set[Operation](Operation.ALL)) - - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](allowAll, denyAcl)) - EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() - EasyMock.replay(clusterAclCache, topicConfigCache) - - assertFalse("deny should take precedence over allow.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) - - EasyMock.verify(clusterAclCache, topicConfigCache) - } - - @Test - def testAllowAllAccess(): Unit = { - val allowAllAcl: Acl = Acl.allowAllAcl - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = "alice", logConfig = null, acls = Set[Acl](Acl.allowAllAcl)) - EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() - EasyMock.replay(clusterAclCache, topicConfigCache) - - val session: Session = new Session(new UserPrincipal("random"), "random.host") - assertTrue("allow all acl should allow access to all.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) - - EasyMock.verify(clusterAclCache, topicConfigCache) - } - - @Test - def testOwnerHasAccess(): Unit = { - val denyAllAcl: Acl = new Acl(Acl.wildCardPrincipal, PermissionType.DENY, Set[String](Acl.wildCardHost), Set[Operation](Operation.ALL)) - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](denyAllAcl)) - EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() - EasyMock.replay(clusterAclCache, topicConfigCache) - - assertTrue("owner always has access, no matter what acls.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) - - EasyMock.verify(clusterAclCache, topicConfigCache) - } - - @Test - def testSuperUserHasAccess(): Unit = { - val denyAllAcl: Acl = new Acl(Acl.wildCardPrincipal, PermissionType.DENY, Set[String](Acl.wildCardHost), Set[Operation](Operation.ALL)) - val topicConfig: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl](denyAllAcl)) - EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig).anyTimes() - EasyMock.replay(clusterAclCache, topicConfigCache) - - val session1: Session = new Session(new UserPrincipal("superuser1"), "random.host") - val session2: Session = new Session(new UserPrincipal("superuser2"), "random.host") - - assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session1, Operation.READ, resource)) - assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session2, Operation.READ, resource)) - - EasyMock.verify(clusterAclCache, topicConfigCache) - } - - - @Test - def testNoAclFound(): Unit = { - val topicConfig1: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = null) - val topicConfig2: TopicConfig = new TopicConfig(version = 1, owner = testPrincipal.getName, logConfig = null, acls = Set[Acl]()) - - EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig1).times(2) - EasyMock.expect(topicConfigCache.getTopicConfig(resource)).andReturn(topicConfig2).times(2) - EasyMock.replay(clusterAclCache, topicConfigCache) - - assertTrue("when acls = null, authorizer should fail open.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) - assertTrue("when acls = [], authorizer should fail open.", simpleAclAuthorizer.authorize(session, Operation.READ, resource)) - - EasyMock.verify(clusterAclCache, topicConfigCache) - } - - @Test - def testFailOpenOnProgrammingErrors(): Unit = { - EasyMock.replay(clusterAclCache, topicConfigCache) - - assertTrue("null session should fail open.", simpleAclAuthorizer.authorize(null, Operation.READ, resource)) - assertTrue("null principal should fail open.", simpleAclAuthorizer.authorize(new Session(null, testHostName), Operation.READ, resource)) - assertTrue("null host should fail open.", simpleAclAuthorizer.authorize(new Session(testPrincipal, null), Operation.READ, resource)) - - assertTrue("null resource should fail open.", simpleAclAuthorizer.authorize(session, Operation.READ, null)) - assertTrue("empty resource should fail open.", simpleAclAuthorizer.authorize(session, Operation.READ, "")) - - assertTrue("unsupported Operation should fail open.", simpleAclAuthorizer.authorize(session, Operation.CREATE, resource)) - - EasyMock.verify(clusterAclCache, topicConfigCache) - } -} diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 3b3cf2e..6749bed 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -275,7 +275,6 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.AuthorizerClassNameProp => // ignore string case KafkaConfig.SuperUserProp => //ignore String - case KafkaConfig.ClusterAclJsonFilePathProp => //ignore String case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.HostNameProp => // ignore string -- 1.9.5 (Apple Git-50.3)