From 9fd2d4ffdba4b45df7807c1a70ff17496b51afa8 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Thu, 1 Mar 2012 21:15:26 +0000 Subject: [PATCH] KAFKA-239 Refactoring code to wire new ZK data structures and making partitions logical; patched by Neha Narkhede; reviewed by Jun Rao git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/branches/0.8@1295861 13f79535-47bb-0310-9956-ffa450edef68 --- .../main/scala/kafka/admin/AdminUtils.scala | 11 +- .../kafka/admin/CreateTopicCommand.scala | 9 +- .../main/scala/kafka/api/TopicMetadata.scala | 4 +- .../kafka/api/TopicMetadataRequest.scala | 12 +- .../main/scala/kafka/cluster/Partition.scala | 35 +- .../NoLeaderForPartitionException.scala | 25 ++ .../scala/kafka/consumer/SimpleConsumer.scala | 31 +- .../consumer/ZookeeperConsumerConnector.scala | 49 ++- .../src/main/scala/kafka/log/LogManager.scala | 63 +-- .../kafka/producer/BrokerPartitionInfo.scala | 84 ++-- .../producer/ConfigBrokerPartitionInfo.scala | 95 ---- .../kafka/producer/KafkaLog4jAppender.scala | 2 +- .../main/scala/kafka/producer/Producer.scala | 16 +- .../scala/kafka/producer/ProducerConfig.scala | 10 +- .../scala/kafka/producer/ProducerPool.scala | 61 ++- .../scala/kafka/producer/SyncProducer.scala | 16 + .../producer/ZKBrokerPartitionInfo.scala | 376 ---------------- .../producer/async/DefaultEventHandler.scala | 62 +-- .../main/scala/kafka/server/KafkaApis.scala | 1 + .../main/scala/kafka/server/KafkaServer.scala | 5 +- .../scala/kafka/server/KafkaZooKeeper.scala | 20 - .../scala/kafka/utils/KafkaScheduler.scala | 37 +- .../scala/kafka/utils/UpdateOffsetsInZK.scala | 32 +- core/src/main/scala/kafka/utils/Utils.scala | 15 + core/src/main/scala/kafka/utils/ZkUtils.scala | 66 ++- .../scala/unit/kafka/admin/AdminTest.scala | 3 +- .../ZookeeperConsumerConnectorTest.scala | 248 +++++++++-- .../integration/AutoOffsetResetTest.scala | 33 +- .../unit/kafka/integration/FetcherTest.scala | 10 +- .../integration/LazyInitProducerTest.scala | 77 ++-- .../kafka/integration/LogCorruptionTest.scala | 8 +- .../kafka/integration/PrimitiveApiTest.scala | 100 +++-- .../ProducerConsumerTestHarness.scala | 18 +- .../ZookeeperConsumerConnectorTest.scala | 17 +- .../integration/PrimitiveApiTest.scala | 415 ------------------ .../scala/unit/kafka/log/LogManagerTest.scala | 35 +- .../scala/unit/kafka/log/LogOffsetTest.scala | 22 +- .../kafka/log4j/KafkaLog4jAppenderTest.scala | 59 +-- .../kafka/producer/AsyncProducerTest.scala | 238 +++++++--- .../unit/kafka/producer/ProducerTest.scala | 153 +++---- .../kafka/producer/SyncProducerTest.scala | 2 +- .../kafka/server/ServerShutdownTest.scala | 56 +-- .../scala/unit/kafka/utils/TestUtils.scala | 43 +- .../unit/kafka/zk/ZKLoadBalanceTest.scala | 126 ------ system_test/broker_failure/bin/run-test.sh | 2 +- 45 files changed, 1039 insertions(+), 1763 deletions(-) create mode 100644 core/src/main/scala/kafka/common/NoLeaderForPartitionException.scala delete mode 100644 core/src/main/scala/kafka/producer/ConfigBrokerPartitionInfo.scala delete mode 100644 core/src/main/scala/kafka/producer/ZKBrokerPartitionInfo.scala delete mode 100644 core/src/test/scala/unit/kafka/javaapi/integration/PrimitiveApiTest.scala delete mode 100644 core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 50c3a065231..07dd95c5a0b 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -80,6 +80,10 @@ object AdminUtils extends Logging { for (i <- 0 until replicaAssignmentList.size) { val zkPath = ZkUtils.getTopicPartitionReplicasPath(topic, i.toString) ZkUtils.updatePersistentPath(zkClient, zkPath, Utils.seqToCSV(replicaAssignmentList(i))) + // TODO: Remove this with leader election patch + // assign leader for the partition i +// ZkUtils.updateEphemeralPath(zkClient, ZkUtils.getTopicPartitionLeaderPath(topic, i.toString), +// replicaAssignmentList(i).head) debug("Updated path %s with %s for replica assignment".format(zkPath, Utils.seqToCSV(replicaAssignmentList(i)))) } } @@ -103,18 +107,19 @@ object AdminUtils extends Logging { for (i <-0 until partitionMetadata.size) { val replicas = ZkUtils.readData(zkClient, ZkUtils.getTopicPartitionReplicasPath(topic, partitions(i).toString)) val inSyncReplicas = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.getTopicPartitionInSyncPath(topic, partitions(i).toString)) - val leader = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.getTopicPartitionLeaderPath(topic, partitions(i).toString)) + val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitions(i)) debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader) partitionMetadata(i) = new PartitionMetadata(partitions(i), - if (leader == null) None else Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(leader.toInt)).head), + leader match { case None => None case Some(l) => Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l.toInt)).head) }, getBrokerInfoFromCache(zkClient, cachedBrokerInfo, Utils.getCSVList(replicas).map(id => id.toInt)), getBrokerInfoFromCache(zkClient, cachedBrokerInfo, Utils.getCSVList(inSyncReplicas).map(id => id.toInt)), None /* Return log segment metadata when getOffsetsBefore will be replaced with this API */) } Some(new TopicMetadata(topic, partitionMetadata)) - } else + } else { None + } } metadataList.toList diff --git a/core/src/main/scala/kafka/admin/CreateTopicCommand.scala b/core/src/main/scala/kafka/admin/CreateTopicCommand.scala index e9b20581671..f66c1068f4c 100644 --- a/core/src/main/scala/kafka/admin/CreateTopicCommand.scala +++ b/core/src/main/scala/kafka/admin/CreateTopicCommand.scala @@ -18,10 +18,10 @@ package kafka.admin import joptsimple.OptionParser -import kafka.utils.{Utils, ZKStringSerializer, ZkUtils} import org.I0Itec.zkclient.ZkClient +import kafka.utils.{Logging, Utils, ZKStringSerializer, ZkUtils} -object CreateTopicCommand { +object CreateTopicCommand extends Logging { def main(args: Array[String]): Unit = { val parser = new OptionParser @@ -91,6 +91,7 @@ object CreateTopicCommand { replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, numPartitions, replicationFactor) else replicaAssignment = getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet) + debug("Replica assignment list for %s is %s".format(topic, replicaAssignment)) AdminUtils.createReplicaAssignmentPathInZK(topic, replicaAssignment, zkClient) } @@ -104,8 +105,8 @@ object CreateTopicCommand { if (brokerList.size != brokerList.toSet.size) throw new AdministrationException("duplicate brokers in replica assignment: " + brokerList) if (!brokerList.toSet.subsetOf(availableBrokerList)) - throw new AdministrationException("some specified brokers not available. specified brokers: " + brokerList + - "available broker:" + availableBrokerList) + throw new AdministrationException("some specified brokers not available. specified brokers: " + brokerList.toString + + "available broker:" + availableBrokerList.toString) ret(i) = brokerList.toList if (ret(i).size != ret(0).size) throw new AdministrationException("partition " + i + " has different replication factor: " + brokerList) diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 93f8d2c0a48..49c108dc93e 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -148,8 +148,8 @@ object PartitionMetadata { } } -case class PartitionMetadata(partitionId: Int, leader: Option[Broker], replicas: Seq[Broker], isr: Seq[Broker], - logMetadata: Option[LogMetadata]) { +case class PartitionMetadata(partitionId: Int, leader: Option[Broker], replicas: Seq[Broker], isr: Seq[Broker] = Seq.empty, + logMetadata: Option[LogMetadata] = None) { def sizeInBytes: Int = { var size: Int = 4 /* partition id */ + 1 /* if leader exists*/ diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 10c7da54e0c..b4b7a1218a3 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -72,6 +72,7 @@ object TopicMetadataRequest { def serializeTopicMetadata(topicMetadata: Seq[TopicMetadata]): ByteBuffer = { val size = topicMetadata.foldLeft(4 /* num topics */)(_ + _.sizeInBytes) val buffer = ByteBuffer.allocate(size) + debug("Allocating buffer of size %d for topic metadata response".format(size)) /* number of topics */ buffer.putInt(topicMetadata.size) /* topic partition_metadata */ @@ -122,13 +123,16 @@ case class TopicMetadataRequest(val topics: Seq[String], } class TopicMetadataSend(topicsMetadata: Seq[TopicMetadata]) extends Send { - private var size: Int = topicsMetadata.foldLeft(0)(_ + _.sizeInBytes) + private var size: Int = topicsMetadata.foldLeft(4)(_ + _.sizeInBytes) private val header = ByteBuffer.allocate(6) - val metadata = TopicMetadataRequest.serializeTopicMetadata(topicsMetadata) header.putInt(size + 2) header.putShort(ErrorMapping.NoError.asInstanceOf[Short]) header.rewind() + val metadata = TopicMetadataRequest.serializeTopicMetadata(topicsMetadata) + metadata.rewind() + + trace("Wrote size %d in header".format(size + 2)) var complete: Boolean = false def writeTo(channel: GatheringByteChannel): Int = { @@ -136,9 +140,13 @@ class TopicMetadataSend(topicsMetadata: Seq[TopicMetadata]) extends Send { var written = 0 if(header.hasRemaining) written += channel.write(header) + trace("Wrote %d bytes for header".format(written)) + if(!header.hasRemaining && metadata.hasRemaining) written += channel.write(metadata) + trace("Wrote %d bytes for header and metadata".format(written)) + if(!metadata.hasRemaining) complete = true written diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 5d79b6c99bc..8b5f88808ff 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -17,41 +17,14 @@ package kafka.cluster -object Partition { - def parse(s: String): Partition = { - val pieces = s.split("-") - if(pieces.length != 2) - throw new IllegalArgumentException("Expected name in the form x-y.") - new Partition(pieces(0).toInt, pieces(1).toInt) - } -} +case class Partition(val brokerId: Int, val partId: Int, val topic: String = "") extends Ordered[Partition] { -class Partition(val brokerId: Int, val partId: Int) extends Ordered[Partition] { - - def this(name: String) = { - this(1, 1) - } - - def name = brokerId + "-" + partId - - override def toString(): String = name + def name = partId def compare(that: Partition) = - if (this.brokerId == that.brokerId) + if (this.topic == that.topic) this.partId - that.partId else - this.brokerId - that.brokerId - - override def equals(other: Any): Boolean = { - other match { - case that: Partition => - (that canEqual this) && brokerId == that.brokerId && partId == that.partId - case _ => false - } - } - - def canEqual(other: Any): Boolean = other.isInstanceOf[Partition] - - override def hashCode: Int = 31 * (17 + brokerId) + partId + this.topic.compareTo(that.topic) } diff --git a/core/src/main/scala/kafka/common/NoLeaderForPartitionException.scala b/core/src/main/scala/kafka/common/NoLeaderForPartitionException.scala new file mode 100644 index 00000000000..bd973bac335 --- /dev/null +++ b/core/src/main/scala/kafka/common/NoLeaderForPartitionException.scala @@ -0,0 +1,25 @@ +/** + * 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.common + +/** + * Thrown when a request is made for partition, but no leader exists for that partition + */ +class NoLeaderForPartitionException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index 554e25f1046..39b01859a8b 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -22,6 +22,7 @@ import java.nio.channels._ import kafka.api._ import kafka.network._ import kafka.utils._ +import kafka.utils.Utils._ /** * A consumer of kafka messages @@ -77,16 +78,16 @@ class SimpleConsumer(val host: String, getOrMakeConnection() var response: Tuple2[Receive,Int] = null try { - sendRequest(request) - response = getResponse + sendRequest(request, channel) + response = getResponse(channel) } catch { case e : java.io.IOException => info("Reconnect in fetch request due to socket error: ", e) // retry once try { channel = connect - sendRequest(request) - response = getResponse + sendRequest(request, channel) + response = getResponse(channel) } catch { case ioe: java.io.IOException => channel = null; throw ioe; } @@ -115,16 +116,16 @@ class SimpleConsumer(val host: String, getOrMakeConnection() var response: Tuple2[Receive,Int] = null try { - sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets)) - response = getResponse + sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets), channel) + response = getResponse(channel) } catch { case e : java.io.IOException => info("Reconnect in get offetset request due to socket error: ", e) // retry once try { channel = connect - sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets)) - response = getResponse + sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets), channel) + response = getResponse(channel) } catch { case ioe: java.io.IOException => channel = null; throw ioe; } @@ -133,20 +134,6 @@ class SimpleConsumer(val host: String, } } - private def sendRequest(request: Request) = { - val send = new BoundedByteBufferSend(request) - send.writeCompletely(channel) - } - - private def getResponse(): Tuple2[Receive,Int] = { - val response = new BoundedByteBufferReceive() - response.readCompletely(channel) - - // this has the side effect of setting the initial position of buffer correctly - val errorCode: Int = response.buffer.getShort - (response, errorCode) - } - private def getOrMakeConnection() { if(channel == null) { channel = connect() diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 83c7bf5ff9f..5d4c5c3c6c0 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -29,9 +29,9 @@ import org.apache.zookeeper.Watcher.Event.KeeperState import kafka.api.OffsetRequest import java.util.UUID import kafka.serializer.Decoder -import kafka.common.{ConsumerRebalanceFailedException, InvalidConfigException} import java.lang.IllegalStateException import kafka.utils.ZkUtils._ +import kafka.common.{NoBrokersForPartitionException, ConsumerRebalanceFailedException, InvalidConfigException} /** * This class handles the consumers interaction with zookeeper @@ -201,6 +201,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, ret } + // this API is used by unit tests only + def getTopicRegistry: Pool[String, Pool[Partition, PartitionTopicInfo]] = topicRegistry + private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = { info("begin registering consumer " + consumerIdString + " in ZK") createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, topicCount.toJsonString) @@ -368,7 +371,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) extends IZkChildListener { private val dirs = new ZKGroupDirs(group) - private var oldPartitionsPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]() + private var oldPartitionsPerTopicMap: mutable.Map[String, Seq[String]] = new mutable.HashMap[String, Seq[String]]() private var oldConsumersPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]() @throws(classOf[Exception]) @@ -379,18 +382,17 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def releasePartitionOwnership()= { info("Releasing partition ownership") for ((topic, infos) <- topicRegistry) { - val topicDirs = new ZKGroupTopicDirs(group, topic) for(partition <- infos.keys) { - val znode = topicDirs.consumerOwnerDir + "/" + partition - deletePath(zkClient, znode) - debug("Consumer " + consumerIdString + " releasing " + znode) + val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition.partId.toString) + deletePath(zkClient, partitionOwnerPath) + debug("Consumer " + consumerIdString + " releasing " + partitionOwnerPath) } } } private def getRelevantTopicMap(myTopicThreadIdsMap: Map[String, Set[String]], - newPartMap: Map[String,List[String]], - oldPartMap: Map[String,List[String]], + newPartMap: Map[String, Seq[String]], + oldPartMap: Map[String, Seq[String]], newConsumerMap: Map[String,List[String]], oldConsumerMap: Map[String,List[String]]): Map[String, Set[String]] = { var relevantTopicThreadIdsMap = new mutable.HashMap[String, Set[String]]() @@ -477,7 +479,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val topicDirs = new ZKGroupTopicDirs(group, topic) val curConsumers = consumersPerTopicMap.get(topic).get - var curPartitions: List[String] = partitionsPerTopicMap.get(topic).get + var curPartitions: Seq[String] = partitionsPerTopicMap.get(topic).get val nPartsPerConsumer = curPartitions.size / curConsumers.size val nConsumersWithExtraPart = curPartitions.size % curConsumers.size @@ -599,8 +601,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val topic = partitionOwner._1._1 val partition = partitionOwner._1._2 val consumerThreadId = partitionOwner._2 - val topicDirs = new ZKGroupTopicDirs(group, topic) - val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition + val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic,partition) try { createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) @@ -618,37 +619,47 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, else true } - private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partitionString: String, + private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partition: String, topic: String, consumerThreadId: String) { - val partition = Partition.parse(partitionString) val partTopicInfoMap = topicRegistry.get(topic) - val znode = topicDirs.consumerOffsetDir + "/" + partition.name + // find the leader for this partition + val leaderOpt = getLeaderForPartition(zkClient, topic, partition.toInt) + leaderOpt match { + case None => throw new NoBrokersForPartitionException("No leader available for partition %s on topic %s". + format(partition, topic)) + case Some(l) => debug("Leader for partition %s for topic %s is %d".format(partition, topic, l)) + } + val leader = leaderOpt.get + + val znode = topicDirs.consumerOffsetDir + "/" + partition val offsetString = readDataMaybeNull(zkClient, znode) // If first time starting a consumer, set the initial offset based on the config var offset : Long = 0L if (offsetString == null) offset = config.autoOffsetReset match { case OffsetRequest.SmallestTimeString => - earliestOrLatestOffset(topic, partition.brokerId, partition.partId, OffsetRequest.EarliestTime) + earliestOrLatestOffset(topic, leader, partition.toInt, OffsetRequest.EarliestTime) case OffsetRequest.LargestTimeString => - earliestOrLatestOffset(topic, partition.brokerId, partition.partId, OffsetRequest.LatestTime) + earliestOrLatestOffset(topic, leader, partition.toInt, OffsetRequest.LatestTime) case _ => throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig") } else offset = offsetString.toLong + + val partitionObject = new Partition(leader, partition.toInt, topic) val queue = queues.get((topic, consumerThreadId)) val consumedOffset = new AtomicLong(offset) val fetchedOffset = new AtomicLong(offset) val partTopicInfo = new PartitionTopicInfo(topic, - partition.brokerId, - partition, + leader, + partitionObject, queue, consumedOffset, fetchedOffset, new AtomicInteger(config.fetchSize)) - partTopicInfoMap.put(partition, partTopicInfo) + partTopicInfoMap.put(partitionObject, partTopicInfo) debug(partTopicInfo + " selected new offset " + offset) } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 409091fe0d9..418a0b1f9f7 100644 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -19,7 +19,6 @@ package kafka.log import java.io._ import kafka.utils._ -import scala.actors.Actor import scala.collection._ import java.util.concurrent.CountDownLatch import kafka.server.{KafkaConfig, KafkaZooKeeper} @@ -32,7 +31,6 @@ import org.I0Itec.zkclient.ZkClient */ @threadsafe private[kafka] class LogManager(val config: KafkaConfig, - private val scheduler: KafkaScheduler, private val time: Time, val logCleanupIntervalMs: Long, val logCleanupDefaultAgeMs: Long, @@ -47,12 +45,12 @@ private[kafka] class LogManager(val config: KafkaConfig, private val topicPartitionsMap = config.topicPartitionsMap private val logCreationLock = new Object private val random = new java.util.Random - private var zkActor: Actor = null private val startupLatch: CountDownLatch = new CountDownLatch(1) private val logFlusherScheduler = new KafkaScheduler(1, "kafka-logflusher-", false) private val logFlushIntervalMap = config.flushIntervalMap private val logRetentionMSMap = getLogRetentionMSMap(config.logRetentionHoursMap) private val logRetentionSize = config.logRetentionSize + private val scheduler = new KafkaScheduler(1, "kafka-logcleaner-", false) /* Initialize a log for each subdirectory of the main log directory */ private val logs = new Pool[String, Pool[Int, Log]]() @@ -78,35 +76,6 @@ private[kafka] class LogManager(val config: KafkaConfig, } } - /* Schedule the cleanup task to delete old logs */ - if(scheduler != null) { - info("starting log cleaner every " + logCleanupIntervalMs + " ms") - scheduler.scheduleWithRate(cleanupLogs, 60 * 1000, logCleanupIntervalMs) - } - - kafkaZookeeper.startup - zkActor = new Actor { - def act() { - loop { - receive { - case topic: String => - try { - kafkaZookeeper.registerTopicInZk(topic) - } - catch { - case e => error(e) // log it and let it go - } - case StopActor => - info("zkActor stopped") - exit - } - } - } - } - zkActor.start - - case object StopActor - private def getLogRetentionMSMap(logRetentionHourMap: Map[String, Int]) : Map[String, Long] = { var ret = new mutable.HashMap[String, Long] for ( (topic, hour) <- logRetentionHourMap ) @@ -118,22 +87,29 @@ private[kafka] class LogManager(val config: KafkaConfig, * Register this broker in ZK for the first time. */ def startup() { + kafkaZookeeper.startup kafkaZookeeper.registerBrokerInZk() - for (topic <- getAllTopics) - kafkaZookeeper.registerTopicInZk(topic) - startupLatch.countDown + + /* Schedule the cleanup task to delete old logs */ + if(scheduler != null) { + if(scheduler.hasShutdown) { + println("Restarting log cleaner scheduler") + scheduler.startUp + } + info("starting log cleaner every " + logCleanupIntervalMs + " ms") + scheduler.scheduleWithRate(cleanupLogs, 60 * 1000, logCleanupIntervalMs) + } + + if(logFlusherScheduler.hasShutdown) logFlusherScheduler.startUp info("Starting log flusher every " + config.flushSchedulerThreadRate + " ms with the following overrides " + logFlushIntervalMap) logFlusherScheduler.scheduleWithRate(flushAllLogs, config.flushSchedulerThreadRate, config.flushSchedulerThreadRate) + startupLatch.countDown } private def awaitStartup() { startupLatch.await } - private def registerNewTopicInZK(topic: String) { - zkActor ! topic - } - /** * Create a log for the given topic and the given partition */ @@ -186,6 +162,10 @@ private[kafka] class LogManager(val config: KafkaConfig, * Create the log if it does not exist, if it exists just return it */ def getOrCreateLog(topic: String, partition: Int): Log = { + // TODO: Change this later + if(!ZkUtils.isPartitionOnBroker(kafkaZookeeper.zkClient, topic, partition, config.brokerId)) + throw new InvalidPartitionException("Broker %d does not host partition %d for topic %s". + format(config.brokerId, partition, topic)) var hasNewTopic = false var parts = getLogPool(topic, partition) if (parts == null) { @@ -196,6 +176,7 @@ private[kafka] class LogManager(val config: KafkaConfig, } var log = parts.get(partition) if(log == null) { + // check if this broker hosts this partition log = createLog(topic, partition) val found = parts.putIfNotExists(partition, log) if(found != null) { @@ -207,8 +188,6 @@ private[kafka] class LogManager(val config: KafkaConfig, info("Created log for '" + topic + "'-" + partition) } - if (hasNewTopic) - registerNewTopicInZK(topic) log } @@ -279,11 +258,11 @@ private[kafka] class LogManager(val config: KafkaConfig, */ def close() { info("Closing log manager") + scheduler.shutdown() logFlusherScheduler.shutdown() val iter = getLogIterator while(iter.hasNext) iter.next.close() - zkActor ! StopActor kafkaZookeeper.close } diff --git a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala index 09f3081e1b1..fd2bc4ebb4f 100644 --- a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala +++ b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala @@ -16,44 +16,70 @@ */ package kafka.producer -import collection.Map -import collection.SortedSet import kafka.cluster.{Broker, Partition} +import collection.mutable.HashMap +import kafka.api.{TopicMetadataRequest, TopicMetadata} +import java.lang.IllegalStateException +import kafka.common.NoLeaderForPartitionException +import kafka.utils.Logging + +class BrokerPartitionInfo(producerPool: ProducerPool) extends Logging { + val topicPartitionInfo = new HashMap[String, TopicMetadata]() + val zkClient = producerPool.getZkClient -trait BrokerPartitionInfo { /** * Return a sequence of (brokerId, numPartitions). * @param topic the topic for which this information is to be returned * @return a sequence of (brokerId, numPartitions). Returns a zero-length * sequence if no brokers are available. */ - def getBrokerPartitionInfo(topic: String = null): SortedSet[Partition] + def getBrokerPartitionInfo(topic: String): Seq[(Partition, Broker)] = { + // check if the cache has metadata for this topic + val topicMetadata = topicPartitionInfo.get(topic) + val metadata: TopicMetadata = + topicMetadata match { + case Some(m) => m + case None => + // refresh the topic metadata cache + info("Fetching metadata for topic %s".format(topic)) + updateInfo(topic) + val topicMetadata = topicPartitionInfo.get(topic) + topicMetadata match { + case Some(m) => m + case None => throw new IllegalStateException("Failed to fetch topic metadata for topic: " + topic) + } + } + val partitionMetadata = metadata.partitionsMetadata + partitionMetadata.map { m => + m.leader match { + case Some(leader) => (new Partition(leader.id, m.partitionId, topic) -> leader) + case None => throw new NoLeaderForPartitionException("No leader for topic %s, partition %d".format(topic, m.partitionId)) + } + }.sortWith((s, t) => s._1.partId < t._1.partId) + } /** - * Generate the host and port information for the broker identified - * by the given broker id - * @param brokerId the broker for which the info is to be returned - * @return host and port of brokerId + * It updates the cache by issuing a get topic metadata request to a random broker. + * @param topic the topic for which the metadata is to be fetched */ - def getBrokerInfo(brokerId: Int): Option[Broker] - - /** - * Generate a mapping from broker id to the host and port for all brokers - * @return mapping from id to host and port of all brokers - */ - def getAllBrokerInfo: Map[Int, Broker] - - /** - * This is relevant to the ZKBrokerPartitionInfo. It updates the ZK cache - * by reading from zookeeper and recreating the data structures. This API - * is invoked by the producer, when it detects that the ZK cache of - * ZKBrokerPartitionInfo is stale. - * - */ - def updateInfo - - /** - * Cleanup - */ - def close + def updateInfo(topic: String = null) = { + val producer = producerPool.getAnyProducer + if(topic != null) { + val topicMetadataRequest = new TopicMetadataRequest(List(topic)) + val topicMetadataList = producer.send(topicMetadataRequest) + val topicMetadata:Option[TopicMetadata] = if(topicMetadataList.size > 0) Some(topicMetadataList.head) else None + topicMetadata match { + case Some(metadata) => + info("Fetched metadata for topics %s".format(topic)) + topicPartitionInfo += (topic -> metadata) + case None => + } + }else { + // refresh cache for all topics + val topics = topicPartitionInfo.keySet.toList + val topicMetadata = producer.send(new TopicMetadataRequest(topics)) + info("Fetched metadata for topics %s".format(topicMetadata.mkString(","))) + topicMetadata.foreach(metadata => topicPartitionInfo += (metadata.topic -> metadata)) + } + } } diff --git a/core/src/main/scala/kafka/producer/ConfigBrokerPartitionInfo.scala b/core/src/main/scala/kafka/producer/ConfigBrokerPartitionInfo.scala deleted file mode 100644 index 206e379cfa2..00000000000 --- a/core/src/main/scala/kafka/producer/ConfigBrokerPartitionInfo.scala +++ /dev/null @@ -1,95 +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.producer - -import collection.mutable.HashMap -import collection.Map -import collection.SortedSet -import kafka.cluster.{Broker, Partition} -import kafka.common.InvalidConfigException -import kafka.api.ProducerRequest - -private[producer] class ConfigBrokerPartitionInfo(config: ProducerConfig) extends BrokerPartitionInfo { - private val brokerPartitions: SortedSet[Partition] = getConfigTopicPartitionInfo - private val allBrokers = getConfigBrokerInfo - - /** - * Return a sequence of (brokerId, numPartitions) - * @param topic this value is null - * @return a sequence of (brokerId, numPartitions) - */ - def getBrokerPartitionInfo(topic: String): SortedSet[Partition] = brokerPartitions - - /** - * Generate the host and port information for the broker identified - * by the given broker id - * @param brokerId the broker for which the info is to be returned - * @return host and port of brokerId - */ - def getBrokerInfo(brokerId: Int): Option[Broker] = { - allBrokers.get(brokerId) - } - - /** - * Generate a mapping from broker id to the host and port for all brokers - * @return mapping from id to host and port of all brokers - */ - def getAllBrokerInfo: Map[Int, Broker] = allBrokers - - def close {} - - def updateInfo = {} - - /** - * Generate a sequence of (brokerId, numPartitions) for all brokers - * specified in the producer configuration - * @return sequence of (brokerId, numPartitions) - */ - private def getConfigTopicPartitionInfo(): SortedSet[Partition] = { - val brokerInfoList = config.brokerList.split(",") - if(brokerInfoList.size == 0) throw new InvalidConfigException("broker.list is empty") - // check if each individual broker info is valid => (brokerId: brokerHost: brokerPort) - brokerInfoList.foreach { bInfo => - val brokerInfo = bInfo.split(":") - if(brokerInfo.size < 3) throw new InvalidConfigException("broker.list has invalid value") - } - val brokerIds = brokerInfoList.map(bInfo => bInfo.split(":").head.toInt) - var brokerParts = SortedSet.empty[Partition] - brokerIds.foreach { bid => - val bidPid = new Partition(bid, ProducerRequest.RandomPartition) - brokerParts += bidPid - } - brokerParts - } - - /** - * Generate the host and port information for for all brokers - * specified in the producer configuration - * @return mapping from brokerId to (host, port) for all brokers - */ - private def getConfigBrokerInfo(): Map[Int, Broker] = { - val brokerInfo = new HashMap[Int, Broker]() - val brokerInfoList = config.brokerList.split(",") - brokerInfoList.foreach{ bInfo => - val brokerIdHostPort = bInfo.split(":") - brokerInfo += (brokerIdHostPort(0).toInt -> new Broker(brokerIdHostPort(0).toInt, brokerIdHostPort(1), - brokerIdHostPort(1), brokerIdHostPort(2).toInt)) - } - brokerInfo - } - -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index cdd3a45e73a..747bbbe498d 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -18,7 +18,7 @@ package kafka.producer import async.MissingConfigException -import org.apache.log4j.spi.{LoggingEvent, ErrorCode} +import org.apache.log4j.spi.LoggingEvent import org.apache.log4j.AppenderSkeleton import org.apache.log4j.helpers.LogLog import kafka.utils.Logging diff --git a/core/src/main/scala/kafka/producer/Producer.scala b/core/src/main/scala/kafka/producer/Producer.scala index 33573500324..e9bbfaed923 100644 --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -22,15 +22,14 @@ import kafka.common.InvalidConfigException import java.util.concurrent.{TimeUnit, LinkedBlockingQueue} import kafka.serializer.Encoder import java.util.concurrent.atomic.{AtomicLong, AtomicBoolean} +import org.I0Itec.zkclient.ZkClient class Producer[K,V](config: ProducerConfig, private val eventHandler: EventHandler[K,V]) // for testing only extends Logging { private val hasShutdown = new AtomicBoolean(false) - if(!Utils.propertyExists(config.zkConnect) && !Utils.propertyExists(config.brokerList)) - throw new InvalidConfigException("At least one of zk.connect or broker.list must be specified") - if (Utils.propertyExists(config.zkConnect) && Utils.propertyExists(config.brokerList)) - throw new InvalidConfigException("Only one of zk.connect and broker.list should be provided") + if(!Utils.propertyExists(config.zkConnect)) + throw new InvalidConfigException("zk.connect property must be specified in the producer") if (config.batchSize > config.queueSize) throw new InvalidConfigException("Batch size can't be larger than queue size.") @@ -52,15 +51,16 @@ extends Logging { * This constructor can be used when all config parameters will be specified through the * ProducerConfig object * @param config Producer Configuration object + * @param zkClient The ZkClient instance use by the producer to connect to zookeeper. used ONLY for testing */ - def this(config: ProducerConfig) = + def this(config: ProducerConfig, zkClient: ZkClient = null) = this(config, new DefaultEventHandler[K,V](config, Utils.getObject[Partitioner[K]](config.partitionerClass), Utils.getObject[Encoder[V]](config.serializerClass), - new ProducerPool(config), - populateProducerPool= true, - brokerPartitionInfo= null)) + new ProducerPool(config, if(zkClient == null) + new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, + config.zkConnectionTimeoutMs, ZKStringSerializer) else zkClient))) /** * Sends the data, partitioned by key to the topic using either the diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index 8e1af37f735..7bc1feb184e 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -29,15 +29,15 @@ class ProducerConfig(val props: Properties) extends ZKConfig(props) * to pass in static broker and per-broker partition information. Format- * * brokerid1:host1:port1, brokerid2:host2:port2*/ val brokerList = Utils.getString(props, "broker.list", null) - if(brokerList != null && Utils.getString(props, "partitioner.class", null) != null) - throw new InvalidConfigException("partitioner.class cannot be used when broker.list is set") + if(brokerList != null) + throw new InvalidConfigException("broker.list is deprecated. Use zk.connect instead") /** If both broker.list and zk.connect options are specified, throw an exception */ - if(brokerList != null && zkConnect != null) - throw new InvalidConfigException("only one of broker.list and zk.connect can be specified") + if(zkConnect == null) + throw new InvalidConfigException("zk.connect property is required") /** the partitioner class for partitioning events amongst sub-topics */ - val partitionerClass = Utils.getString(props, "partitioner.class", null) + val partitionerClass = Utils.getString(props, "partitioner.class", "kafka.producer.DefaultPartitioner") /** this parameter specifies whether the messages are sent asynchronously * * or not. Valid values are - async for asynchronous send * diff --git a/core/src/main/scala/kafka/producer/ProducerPool.scala b/core/src/main/scala/kafka/producer/ProducerPool.scala index 434aa62f5cc..063f713eb4d 100644 --- a/core/src/main/scala/kafka/producer/ProducerPool.scala +++ b/core/src/main/scala/kafka/producer/ProducerPool.scala @@ -13,19 +13,23 @@ * 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.producer -import java.util.Properties import kafka.cluster.Broker -import kafka.utils.Logging -import java.util.concurrent.ConcurrentHashMap +import java.util.Properties +import org.I0Itec.zkclient.ZkClient +import kafka.utils.{ZkUtils, Utils, Logging} +import collection.mutable.HashMap +import java.lang.Object +import kafka.common.{UnavailableProducerException, NoBrokersForPartitionException} -class ProducerPool(private val config: ProducerConfig) extends Logging { - private val syncProducers = new ConcurrentHashMap[Int, SyncProducer] +class ProducerPool(val config: ProducerConfig, val zkClient: ZkClient) extends Logging { + private val syncProducers = new HashMap[Int, SyncProducer] + private val lock = new Object() - def addProducer(broker: Broker) { + private def addProducer(broker: Broker) { val props = new Properties() props.put("host", broker.host) props.put("port", broker.port.toString) @@ -42,17 +46,48 @@ class ProducerPool(private val config: ProducerConfig) extends Logging { syncProducers.put(brokerId, syncProducer) } - def getProducer(brokerId: Int) : SyncProducer = { - syncProducers.get(brokerId) + def addProducers(config: ProducerConfig) { + lock.synchronized { + debug("Connecting to %s for creating sync producers for all brokers in the cluster".format(config.zkConnect)) + val brokers = ZkUtils.getAllBrokersInCluster(zkClient) + brokers.foreach(broker => addProducer(broker)) + } } + def getProducer(brokerId: Int) : SyncProducer = { + lock.synchronized { + val producer = syncProducers.get(brokerId) + producer match { + case Some(p) => p + case None => throw new UnavailableProducerException("Sync producer for broker id %d does not exist".format(brokerId)) + } + } + } + + def getAnyProducer: SyncProducer = { + lock.synchronized { + if(syncProducers.size == 0) { + // refresh the list of brokers from zookeeper + info("No sync producers available. Refreshing the available broker list from ZK and creating sync producers") + addProducers(config) + if(syncProducers.size == 0) + throw new NoBrokersForPartitionException("No brokers available") + } + syncProducers.get(Utils.random.nextInt(syncProducers.size)).get + } + } + + def getZkClient: ZkClient = zkClient + /** * Closes all the producers in the pool */ def close() = { - info("Closing all sync producers") - val iter = syncProducers.values.iterator - while(iter.hasNext) - iter.next.close + lock.synchronized { + info("Closing all sync producers") + val iter = syncProducers.values.iterator + while(iter.hasNext) + iter.next.close + } } } diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index dc1b91448e6..91448dbd1d2 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -26,6 +26,7 @@ import kafka.api._ import scala.math._ import kafka.common.MessageSizeTooLargeException import java.nio.ByteBuffer +import kafka.utils.Utils._ object SyncProducer { val RequestKey: Short = 0 @@ -124,6 +125,21 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { send(new BoundedByteBufferSend(new MultiProducerRequest(produces))) } + def send(request: TopicMetadataRequest): Seq[TopicMetadata] = { + lock synchronized { + getOrMakeConnection() + var response: Tuple2[Receive,Int] = null + try { + sendRequest(request, channel) + response = getResponse(channel) + } catch { + case e : java.io.IOException => error("Failed to write topic metadata request on the socket channel", e) + } + // TODO: handle any errors in the response and throw the relevant exception + TopicMetadataRequest.deserializeTopicsMetadataResponse(response._1.buffer) + } + } + def close() = { lock synchronized { disconnect() diff --git a/core/src/main/scala/kafka/producer/ZKBrokerPartitionInfo.scala b/core/src/main/scala/kafka/producer/ZKBrokerPartitionInfo.scala deleted file mode 100644 index fa72fd37c08..00000000000 --- a/core/src/main/scala/kafka/producer/ZKBrokerPartitionInfo.scala +++ /dev/null @@ -1,376 +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.producer - -import kafka.utils.{ZKStringSerializer, ZkUtils, ZKConfig} -import collection.mutable.HashMap -import collection.immutable.Map -import kafka.utils.Logging -import collection.immutable.TreeSet -import kafka.cluster.{Broker, Partition} -import org.apache.zookeeper.Watcher.Event.KeeperState -import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} -import collection.SortedSet - -private[producer] object ZKBrokerPartitionInfo { - - /** - * Generate a mapping from broker id to (brokerId, numPartitions) for the list of brokers - * specified - * @param topic the topic to which the brokers have registered - * @param brokerList the list of brokers for which the partitions info is to be generated - * @return a sequence of (brokerId, numPartitions) for brokers in brokerList - */ - private def getBrokerPartitions(zkClient: ZkClient, topic: String, brokerList: List[Int]): SortedSet[Partition] = { - val brokerTopicPath = ZkUtils.BrokerTopicsPath + "/" + topic - val numPartitions = brokerList.map(bid => ZkUtils.readData(zkClient, brokerTopicPath + "/" + bid).toInt) - val brokerPartitions = brokerList.zip(numPartitions) - - val sortedBrokerPartitions = brokerPartitions.sortWith((id1, id2) => id1._1 < id2._1) - - var brokerParts = SortedSet.empty[Partition] - sortedBrokerPartitions.foreach { bp => - for(i <- 0 until bp._2) { - val bidPid = new Partition(bp._1, i) - brokerParts = brokerParts + bidPid - } - } - brokerParts - } -} - -/** - * If zookeeper based auto partition discovery is enabled, fetch broker info like - * host, port, number of partitions from zookeeper - */ -private[producer] class ZKBrokerPartitionInfo(config: ZKConfig, producerCbk: (Int, String, Int) => Unit) extends BrokerPartitionInfo with Logging { - private val zkWatcherLock = new Object - private val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) - // maintain a map from topic -> list of (broker, num_partitions) from zookeeper - private var topicBrokerPartitions = getZKTopicPartitionInfo - // maintain a map from broker id to the corresponding Broker object - private var allBrokers = getZKBrokerInfo - - // use just the brokerTopicsListener for all watchers - private val brokerTopicsListener = new BrokerTopicsListener(topicBrokerPartitions, allBrokers) - // register listener for change of topics to keep topicsBrokerPartitions updated - zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, brokerTopicsListener) - - // register listener for change of brokers for each topic to keep topicsBrokerPartitions updated - topicBrokerPartitions.keySet.foreach {topic => - zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath + "/" + topic, brokerTopicsListener) - debug("Registering listener on path: " + ZkUtils.BrokerTopicsPath + "/" + topic) - } - - // register listener for new broker - zkClient.subscribeChildChanges(ZkUtils.BrokerIdsPath, brokerTopicsListener) - - // register listener for session expired event - zkClient.subscribeStateChanges(new ZKSessionExpirationListener(brokerTopicsListener)) - - /** - * Return a sequence of (brokerId, numPartitions) - * @param topic the topic for which this information is to be returned - * @return a sequence of (brokerId, numPartitions). Returns a zero-length - * sequence if no brokers are available. - */ - def getBrokerPartitionInfo(topic: String): SortedSet[Partition] = { - zkWatcherLock synchronized { - val brokerPartitions = topicBrokerPartitions.get(topic) - var numBrokerPartitions = SortedSet.empty[Partition] - brokerPartitions match { - case Some(bp) => - bp.size match { - case 0 => // no brokers currently registered for this topic. Find the list of all brokers in the cluster. - numBrokerPartitions = bootstrapWithExistingBrokers(topic) - topicBrokerPartitions += (topic -> numBrokerPartitions) - case _ => numBrokerPartitions = TreeSet[Partition]() ++ bp - } - case None => // no brokers currently registered for this topic. Find the list of all brokers in the cluster. - numBrokerPartitions = bootstrapWithExistingBrokers(topic) - topicBrokerPartitions += (topic -> numBrokerPartitions) - } - numBrokerPartitions - } - } - - /** - * Generate the host and port information for the broker identified - * by the given broker id - * @param brokerId the broker for which the info is to be returned - * @return host and port of brokerId - */ - def getBrokerInfo(brokerId: Int): Option[Broker] = { - zkWatcherLock synchronized { - allBrokers.get(brokerId) - } - } - - /** - * Generate a mapping from broker id to the host and port for all brokers - * @return mapping from id to host and port of all brokers - */ - def getAllBrokerInfo: Map[Int, Broker] = allBrokers - - def close = zkClient.close - - def updateInfo = { - zkWatcherLock synchronized { - topicBrokerPartitions = getZKTopicPartitionInfo - allBrokers = getZKBrokerInfo - } - } - - private def bootstrapWithExistingBrokers(topic: String): scala.collection.immutable.SortedSet[Partition] = { - debug("Currently, no brokers are registered under topic: " + topic) - debug("Bootstrapping topic: " + topic + " with available brokers in the cluster with default " + - "number of partitions = 1") - val allBrokersIds = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerIdsPath) - trace("List of all brokers currently registered in zookeeper = " + allBrokersIds.toString) - // since we do not have the in formation about number of partitions on these brokers, just assume single partition - // i.e. pick partition 0 from each broker as a candidate - val numBrokerPartitions = TreeSet[Partition]() ++ allBrokersIds.map(b => new Partition(b.toInt, 0)) - // add the rest of the available brokers with default 1 partition for this topic, so all of the brokers - // participate in hosting this topic. - debug("Adding following broker id, partition id for NEW topic: " + topic + "=" + numBrokerPartitions.toString) - numBrokerPartitions - } - - /** - * Generate a sequence of (brokerId, numPartitions) for all topics - * registered in zookeeper - * @return a mapping from topic to sequence of (brokerId, numPartitions) - */ - private def getZKTopicPartitionInfo(): collection.mutable.Map[String, SortedSet[Partition]] = { - val brokerPartitionsPerTopic = new HashMap[String, SortedSet[Partition]]() - ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath) - val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath) - topics.foreach { topic => - // find the number of broker partitions registered for this topic - val brokerTopicPath = ZkUtils.BrokerTopicsPath + "/" + topic - val brokerList = ZkUtils.getChildrenParentMayNotExist(zkClient, brokerTopicPath) - val numPartitions = brokerList.map(bid => ZkUtils.readData(zkClient, brokerTopicPath + "/" + bid).toInt) - val brokerPartitions = brokerList.map(bid => bid.toInt).zip(numPartitions) - val sortedBrokerPartitions = brokerPartitions.sortWith((id1, id2) => id1._1 < id2._1) - debug("Broker ids and # of partitions on each for topic: " + topic + " = " + sortedBrokerPartitions.toString) - - var brokerParts = SortedSet.empty[Partition] - sortedBrokerPartitions.foreach { bp => - for(i <- 0 until bp._2) { - val bidPid = new Partition(bp._1, i) - brokerParts = brokerParts + bidPid - } - } - brokerPartitionsPerTopic += (topic -> brokerParts) - debug("Sorted list of broker ids and partition ids on each for topic: " + topic + " = " + brokerParts.toString) - } - brokerPartitionsPerTopic - } - - /** - * Generate a mapping from broker id to (brokerId, numPartitions) for all brokers - * registered in zookeeper - * @return a mapping from brokerId to (host, port) - */ - private def getZKBrokerInfo(): Map[Int, Broker] = { - val allBrokerIds = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerIdsPath).map(bid => bid.toInt) - val brokers = ZkUtils.getBrokerInfoFromIds(zkClient, allBrokerIds) - allBrokerIds.zip(brokers).toMap - } - - /** - * Listens to new broker registrations under a particular topic, in zookeeper and - * keeps the related data structures updated - */ - class BrokerTopicsListener(val originalBrokerTopicsPartitionsMap: collection.mutable.Map[String, SortedSet[Partition]], - val originalBrokerIdMap: Map[Int, Broker]) extends IZkChildListener with Logging { - private var oldBrokerTopicPartitionsMap = collection.mutable.Map.empty[String, SortedSet[Partition]] ++ - originalBrokerTopicsPartitionsMap - private var oldBrokerIdMap = collection.mutable.Map.empty[Int, Broker] ++ originalBrokerIdMap - - debug("[BrokerTopicsListener] Creating broker topics listener to watch the following paths - \n" + - "/broker/topics, /broker/topics/topic, /broker/ids") - debug("[BrokerTopicsListener] Initialized this broker topics listener with initial mapping of broker id to " + - "partition id per topic with " + oldBrokerTopicPartitionsMap.toString) - - @throws(classOf[Exception]) - def handleChildChange(parentPath : String, currentChildren : java.util.List[String]) { - val curChilds: java.util.List[String] = if(currentChildren != null) currentChildren - else new java.util.ArrayList[String]() - - zkWatcherLock synchronized { - trace("Watcher fired for path: " + parentPath + " with change " + curChilds.toString) - import scala.collection.JavaConversions._ - - parentPath match { - case "/brokers/topics" => // this is a watcher for /broker/topics path - val updatedTopics = asBuffer(curChilds) - debug("[BrokerTopicsListener] List of topics changed at " + parentPath + " Updated topics -> " + - curChilds.toString) - debug("[BrokerTopicsListener] Old list of topics: " + oldBrokerTopicPartitionsMap.keySet.toString) - debug("[BrokerTopicsListener] Updated list of topics: " + updatedTopics.toSet.toString) - val newTopics = updatedTopics.toSet &~ oldBrokerTopicPartitionsMap.keySet - debug("[BrokerTopicsListener] List of newly registered topics: " + newTopics.toString) - newTopics.foreach { topic => - val brokerTopicPath = ZkUtils.BrokerTopicsPath + "/" + topic - val brokerList = ZkUtils.getChildrenParentMayNotExist(zkClient, brokerTopicPath) - processNewBrokerInExistingTopic(topic, brokerList) - zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath + "/" + topic, - brokerTopicsListener) - } - case "/brokers/ids" => // this is a watcher for /broker/ids path - debug("[BrokerTopicsListener] List of brokers changed in the Kafka cluster " + parentPath + - "\t Currently registered list of brokers -> " + curChilds.toString) - processBrokerChange(parentPath, curChilds) - case _ => - val pathSplits = parentPath.split("/") - val topic = pathSplits.last - if(pathSplits.length == 4 && pathSplits(2).equals("topics")) { - debug("[BrokerTopicsListener] List of brokers changed at " + parentPath + "\t Currently registered " + - " list of brokers -> " + curChilds.toString + " for topic -> " + topic) - processNewBrokerInExistingTopic(topic, asBuffer(curChilds)) - } - } - - // update the data structures tracking older state values - oldBrokerTopicPartitionsMap = collection.mutable.Map.empty[String, SortedSet[Partition]] ++ topicBrokerPartitions - oldBrokerIdMap = collection.mutable.Map.empty[Int, Broker] ++ allBrokers - } - } - - def processBrokerChange(parentPath: String, curChilds: Seq[String]) { - if(parentPath.equals(ZkUtils.BrokerIdsPath)) { - import scala.collection.JavaConversions._ - val updatedBrokerList = asBuffer(curChilds).map(bid => bid.toInt) - val newBrokers = updatedBrokerList.toSet &~ oldBrokerIdMap.keySet - debug("[BrokerTopicsListener] List of newly registered brokers: " + newBrokers.toString) - newBrokers.foreach { bid => - val brokerInfo = ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + bid) - val brokerHostPort = brokerInfo.split(":") - allBrokers += (bid -> new Broker(bid, brokerHostPort(1), brokerHostPort(1), brokerHostPort(2).toInt)) - debug("[BrokerTopicsListener] Invoking the callback for broker: " + bid) - producerCbk(bid, brokerHostPort(1), brokerHostPort(2).toInt) - } - // remove dead brokers from the in memory list of live brokers - val deadBrokers = oldBrokerIdMap.keySet &~ updatedBrokerList.toSet - debug("[BrokerTopicsListener] Deleting broker ids for dead brokers: " + deadBrokers.toString) - deadBrokers.foreach {bid => - allBrokers = allBrokers - bid - // also remove this dead broker from particular topics - topicBrokerPartitions.keySet.foreach{ topic => - topicBrokerPartitions.get(topic) match { - case Some(oldBrokerPartitionList) => - val aliveBrokerPartitionList = oldBrokerPartitionList.filter(bp => bp.brokerId != bid) - topicBrokerPartitions += (topic -> aliveBrokerPartitionList) - debug("[BrokerTopicsListener] Removing dead broker ids for topic: " + topic + "\t " + - "Updated list of broker id, partition id = " + aliveBrokerPartitionList.toString) - case None => - } - } - } - } - } - - /** - * Generate the updated mapping of (brokerId, numPartitions) for the new list of brokers - * registered under some topic - * @param parentPath the path of the topic under which the brokers have changed - * @param curChilds the list of changed brokers - */ - def processNewBrokerInExistingTopic(topic: String, curChilds: Seq[String]) = { - // find the old list of brokers for this topic - oldBrokerTopicPartitionsMap.get(topic) match { - case Some(brokersParts) => - debug("[BrokerTopicsListener] Old list of brokers: " + brokersParts.map(bp => bp.brokerId).toString) - case None => - } - - val updatedBrokerList = curChilds.map(b => b.toInt) - import ZKBrokerPartitionInfo._ - val updatedBrokerParts:SortedSet[Partition] = getBrokerPartitions(zkClient, topic, updatedBrokerList.toList) - debug("[BrokerTopicsListener] Currently registered list of brokers for topic: " + topic + " are " + - curChilds.toString) - // update the number of partitions on existing brokers - var mergedBrokerParts: SortedSet[Partition] = TreeSet[Partition]() ++ updatedBrokerParts - topicBrokerPartitions.get(topic) match { - case Some(oldBrokerParts) => - debug("[BrokerTopicsListener] Unregistered list of brokers for topic: " + topic + " are " + - oldBrokerParts.toString) - mergedBrokerParts = oldBrokerParts ++ updatedBrokerParts - case None => - } - // keep only brokers that are alive - mergedBrokerParts = mergedBrokerParts.filter(bp => allBrokers.contains(bp.brokerId)) - topicBrokerPartitions += (topic -> mergedBrokerParts) - debug("[BrokerTopicsListener] List of broker partitions for topic: " + topic + " are " + - mergedBrokerParts.toString) - } - - def resetState = { - trace("[BrokerTopicsListener] Before reseting broker topic partitions state " + - oldBrokerTopicPartitionsMap.toString) - oldBrokerTopicPartitionsMap = collection.mutable.Map.empty[String, SortedSet[Partition]] ++ topicBrokerPartitions - debug("[BrokerTopicsListener] After reseting broker topic partitions state " + - oldBrokerTopicPartitionsMap.toString) - trace("[BrokerTopicsListener] Before reseting broker id map state " + oldBrokerIdMap.toString) - oldBrokerIdMap = collection.mutable.Map.empty[Int, Broker] ++ allBrokers - debug("[BrokerTopicsListener] After reseting broker id map state " + oldBrokerIdMap.toString) - } - } - - /** - * Handles the session expiration event in zookeeper - */ - class ZKSessionExpirationListener(val brokerTopicsListener: BrokerTopicsListener) - extends IZkStateListener { - - @throws(classOf[Exception]) - def handleStateChanged(state: KeeperState) { - // do nothing, since zkclient will do reconnect for us. - } - - /** - * Called after the zookeeper session has expired and a new session has been created. You would have to re-create - * any ephemeral nodes here. - * - * @throws Exception - * On any error. - */ - @throws(classOf[Exception]) - def handleNewSession() { - /** - * When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has reestablished a - * connection for us. - */ - info("ZK expired; release old list of broker partitions for topics ") - topicBrokerPartitions = getZKTopicPartitionInfo - allBrokers = getZKBrokerInfo - brokerTopicsListener.resetState - - // register listener for change of brokers for each topic to keep topicsBrokerPartitions updated - // NOTE: this is probably not required here. Since when we read from getZKTopicPartitionInfo() above, - // it automatically recreates the watchers there itself - topicBrokerPartitions.keySet.foreach(topic => zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath + "/" + topic, - brokerTopicsListener)) - // there is no need to re-register other listeners as they are listening on the child changes of - // permanent nodes - } - - } - -} diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index e26d8fa2061..b524e6a3e86 100644 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -19,56 +19,26 @@ package kafka.producer.async import kafka.api.ProducerRequest import kafka.serializer.Encoder -import java.util.Properties import kafka.producer._ -import kafka.utils.{ZKConfig, Utils, Logging} import kafka.cluster.{Partition, Broker} import collection.mutable.{ListBuffer, HashMap} import scala.collection.Map import kafka.common.{FailedToSendMessageException, InvalidPartitionException, NoBrokersForPartitionException} import kafka.message.{Message, NoCompressionCodec, ByteBufferMessageSet} +import kafka.utils.{Utils, Logging} class DefaultEventHandler[K,V](config: ProducerConfig, // this api is for testing private val partitioner: Partitioner[K], // use the other constructor private val encoder: Encoder[V], - private val producerPool: ProducerPool, - private val populateProducerPool: Boolean, - private var brokerPartitionInfo: BrokerPartitionInfo) + private val producerPool: ProducerPool) extends EventHandler[K,V] with Logging { + val brokerPartitionInfo = new BrokerPartitionInfo(producerPool) + + // add producers to the producer pool + producerPool.addProducers(config) + private val lock = new Object() - private val zkEnabled = Utils.propertyExists(config.zkConnect) - if(brokerPartitionInfo == null) { - zkEnabled match { - case true => - val zkProps = new Properties() - zkProps.put("zk.connect", config.zkConnect) - zkProps.put("zk.sessiontimeout.ms", config.zkSessionTimeoutMs.toString) - zkProps.put("zk.connectiontimeout.ms", config.zkConnectionTimeoutMs.toString) - zkProps.put("zk.synctime.ms", config.zkSyncTimeMs.toString) - brokerPartitionInfo = new ZKBrokerPartitionInfo(new ZKConfig(zkProps), producerCbk) - case false => - brokerPartitionInfo = new ConfigBrokerPartitionInfo(config) - } - } - - // pool of producers, one per broker - if(populateProducerPool) { - val allBrokers = brokerPartitionInfo.getAllBrokerInfo - allBrokers.foreach(b => producerPool.addProducer(new Broker(b._1, b._2.host, b._2.host, b._2.port))) - } - - /** - * Callback to add a new producer to the producer pool. Used by ZKBrokerPartitionInfo - * on registration of new broker in zookeeper - * @param bid the id of the broker - * @param host the hostname of the broker - * @param port the port of the broker - */ - private def producerCbk(bid: Int, host: String, port: Int) = { - if(populateProducerPool) producerPool.addProducer(new Broker(bid, host, host, port)) - else debug("Skipping the callback since populateProducerPool = false") - } def handle(events: Seq[ProducerData[K,V]]) { lock synchronized { @@ -81,7 +51,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, val partitionedData = partitionAndCollate(messages) for ( (brokerid, eventsPerBrokerMap) <- partitionedData) { if (logger.isTraceEnabled) - eventsPerBrokerMap.foreach(partitionAndEvent => trace("Handling event for Topic: %s, Broker: %d, Partition: %d" + eventsPerBrokerMap.foreach(partitionAndEvent => trace("Handling event for Topic: %s, Broker: %d, Partitions: %s" .format(partitionAndEvent._1, brokerid, partitionAndEvent._2))) val messageSetPerBroker = groupMessagesToSet(eventsPerBrokerMap) @@ -98,7 +68,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, numRetries +=1 Thread.sleep(config.producerRetryBackoffMs) try { - brokerPartitionInfo.updateInfo + brokerPartitionInfo.updateInfo() handleSerializedData(eventsPerBroker, 0) return } @@ -125,15 +95,15 @@ class DefaultEventHandler[K,V](config: ProducerConfig, val brokerPartition = topicPartitionsList(partitionIndex) var dataPerBroker: HashMap[(String, Int), Seq[ProducerData[K,Message]]] = null - ret.get(brokerPartition.brokerId) match { + ret.get(brokerPartition._2.id) match { case Some(element) => dataPerBroker = element.asInstanceOf[HashMap[(String, Int), Seq[ProducerData[K,Message]]]] case None => dataPerBroker = new HashMap[(String, Int), Seq[ProducerData[K,Message]]] - ret.put(brokerPartition.brokerId, dataPerBroker) + ret.put(brokerPartition._2.id, dataPerBroker) } - val topicAndPartition = (event.getTopic, brokerPartition.partId) + val topicAndPartition = (event.getTopic, brokerPartition._1.partId) var dataPerTopicPartition: ListBuffer[ProducerData[K,Message]] = null dataPerBroker.get(topicAndPartition) match { case Some(element) => @@ -147,9 +117,9 @@ class DefaultEventHandler[K,V](config: ProducerConfig, ret } - private def getPartitionListForTopic(pd: ProducerData[K,Message]): Seq[Partition] = { + private def getPartitionListForTopic(pd: ProducerData[K,Message]): Seq[(Partition, Broker)] = { debug("Getting the number of broker partitions registered for topic: " + pd.getTopic) - val topicPartitionsList = brokerPartitionInfo.getBrokerPartitionInfo(pd.getTopic).toSeq + val topicPartitionsList = brokerPartitionInfo.getBrokerPartitionInfo(pd.getTopic) debug("Broker partitions registered for topic: " + pd.getTopic + " = " + topicPartitionsList) val totalNumPartitions = topicPartitionsList.length if(totalNumPartitions == 0) throw new NoBrokersForPartitionException("Partition = " + pd.getKey) @@ -168,7 +138,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, throw new InvalidPartitionException("Invalid number of partitions: " + numPartitions + "\n Valid values are > 0") val partition = if(key == null) Utils.getNextRandomInt(numPartitions) - else partitioner.partition(key , numPartitions) + else partitioner.partition(key, numPartitions) if(partition < 0 || partition >= numPartitions) throw new InvalidPartitionException("Invalid partition id : " + partition + "\n Valid values are in the range inclusive [0, " + (numPartitions-1) + "]") @@ -235,7 +205,5 @@ class DefaultEventHandler[K,V](config: ProducerConfig, def close() { if (producerPool != null) producerPool.close - if (brokerPartitionInfo != null) - brokerPartitionInfo.close } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6deca6e327b..2a29c6925e3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -173,6 +173,7 @@ class KafkaApis(val logManager: LogManager) extends Logging { } } } + info("Sending response for topic metadata request") Some(new TopicMetadataSend(topicsMetadata)) } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index c7902f3c0a2..9c18a3efdd0 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -20,7 +20,7 @@ package kafka.server import java.util.concurrent._ import java.util.concurrent.atomic._ import java.io.File -import kafka.utils.{Mx4jLoader, Utils, SystemTime, KafkaScheduler, Logging} +import kafka.utils.{Mx4jLoader, Utils, SystemTime, Logging} import kafka.network.{SocketServerStats, SocketServer} import kafka.log.LogManager @@ -36,7 +36,6 @@ class KafkaServer(val config: KafkaConfig) extends Logging { private val statsMBeanName = "kafka:type=kafka.SocketServerStats" var socketServer: SocketServer = null var requestHandlerPool: KafkaRequestHandlerPool = null - val scheduler = new KafkaScheduler(1, "kafka-logcleaner-", false) private var logManager: LogManager = null /** @@ -52,7 +51,6 @@ class KafkaServer(val config: KafkaConfig) extends Logging { cleanShutDownFile.delete } logManager = new LogManager(config, - scheduler, SystemTime, 1000L * 60 * config.logCleanupIntervalMinutes, 1000L * 60 * 60 * config.logRetentionHours, @@ -85,7 +83,6 @@ class KafkaServer(val config: KafkaConfig) extends Logging { val canShutdown = isShuttingDown.compareAndSet(false, true); if (canShutdown) { info("Shutting down Kafka server") - scheduler.shutdown() if (socketServer != null) socketServer.shutdown() if(requestHandlerPool != null) diff --git a/core/src/main/scala/kafka/server/KafkaZooKeeper.scala b/core/src/main/scala/kafka/server/KafkaZooKeeper.scala index b545c2fe83b..4f53b3296c2 100644 --- a/core/src/main/scala/kafka/server/KafkaZooKeeper.scala +++ b/core/src/main/scala/kafka/server/KafkaZooKeeper.scala @@ -51,21 +51,6 @@ class KafkaZooKeeper(config: KafkaConfig, logManager: LogManager) extends Loggin ZkUtils.registerBrokerInZk(zkClient, config.brokerId, hostName, creatorId, config.port) } - def registerTopicInZk(topic: String) { - registerTopicInZkInternal(topic) - lock synchronized { - topics ::= topic - } - } - - def registerTopicInZkInternal(topic: String) { - val brokerTopicPath = ZkUtils.BrokerTopicsPath + "/" + topic + "/" + config.brokerId - val numParts = logManager.getTopicPartitionsMap.getOrElse(topic, config.numPartitions) - info("Begin registering broker topic " + brokerTopicPath + " with " + numParts.toString + " partitions") - ZkUtils.createEphemeralPathExpectConflict(zkClient, brokerTopicPath, numParts.toString) - info("End registering broker topic " + brokerTopicPath) - } - /** * When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has reestablished a * connection for us. We need to re-register this broker in the broker registry. @@ -87,11 +72,6 @@ class KafkaZooKeeper(config: KafkaConfig, logManager: LogManager) extends Loggin def handleNewSession() { info("re-registering broker info in ZK for broker " + config.brokerId) registerBrokerInZk() - lock synchronized { - info("re-registering broker topics in ZK for broker " + config.brokerId) - for (topic <- topics) - registerTopicInZkInternal(topic) - } info("done re-registering broker") } } diff --git a/core/src/main/scala/kafka/utils/KafkaScheduler.scala b/core/src/main/scala/kafka/utils/KafkaScheduler.scala index 07b999445d6..d5a5d3d946b 100644 --- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala +++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala @@ -19,6 +19,7 @@ package kafka.utils import java.util.concurrent._ import java.util.concurrent.atomic._ +import java.lang.IllegalStateException /** * A scheduler for running jobs in the background @@ -26,25 +27,41 @@ import java.util.concurrent.atomic._ */ class KafkaScheduler(val numThreads: Int, val baseThreadName: String, isDaemon: Boolean) extends Logging { private val threadId = new AtomicLong(0) - private val executor = new ScheduledThreadPoolExecutor(numThreads, new ThreadFactory() { - def newThread(runnable: Runnable): Thread = { - val t = new Thread(runnable, baseThreadName + threadId.getAndIncrement) - t.setDaemon(isDaemon) - t - } - }) - executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false) - executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false) + private var executor:ScheduledThreadPoolExecutor = null + startUp - def scheduleWithRate(fun: () => Unit, delayMs: Long, periodMs: Long) = + def startUp = { + executor = new ScheduledThreadPoolExecutor(numThreads, new ThreadFactory() { + def newThread(runnable: Runnable): Thread = { + val t = new Thread(runnable, baseThreadName + threadId.getAndIncrement) + t.setDaemon(isDaemon) + t + } + }) + executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false) + executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false) + } + + def hasShutdown: Boolean = executor.isShutdown + + private def checkIfExecutorHasStarted = { + if(executor == null) + throw new IllegalStateException("Kafka scheduler has not been started") + } + + def scheduleWithRate(fun: () => Unit, delayMs: Long, periodMs: Long) = { + checkIfExecutorHasStarted executor.scheduleAtFixedRate(Utils.loggedRunnable(fun), delayMs, periodMs, TimeUnit.MILLISECONDS) + } def shutdownNow() { + checkIfExecutorHasStarted executor.shutdownNow() info("force shutdown scheduler " + baseThreadName) } def shutdown() { + checkIfExecutorHasStarted executor.shutdown() info("shutdown scheduler " + baseThreadName) } diff --git a/core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala index ae0d86e5de0..06518ae27a9 100644 --- a/core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala @@ -19,12 +19,11 @@ package kafka.utils import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} -import kafka.cluster.Partition import kafka.api.OffsetRequest import java.lang.IllegalStateException /** - * A utility that updates the offset of every broker partition to the offset of latest log segment file, in ZK. + * A utility that updates the offset of every broker partition to the offset of earliest or latest log segment file, in ZK. */ object UpdateOffsetsInZK { val Earliest = "earliest" @@ -46,7 +45,7 @@ object UpdateOffsetsInZK { private def getAndSetOffsets(zkClient: ZkClient, offsetOption: Long, config: ConsumerConfig, topic: String): Unit = { val cluster = ZkUtils.getCluster(zkClient) val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, List(topic).iterator) - var partitions: List[String] = Nil + var partitions: Seq[String] = Nil partitionsPerTopicMap.get(topic) match { case Some(l) => partitions = l.sortWith((s,t) => s < t) @@ -54,22 +53,29 @@ object UpdateOffsetsInZK { } var numParts = 0 - for (partString <- partitions) { - val part = Partition.parse(partString) - val broker = cluster.getBroker(part.brokerId) match { + for (partition <- partitions) { + val brokerHostingPartition = ZkUtils.getLeaderForPartition(zkClient, topic, partition.toInt) + + val broker = brokerHostingPartition match { case Some(b) => b - case None => throw new IllegalStateException("Broker " + part.brokerId + " is unavailable. Cannot issue " + + case None => throw new IllegalStateException("Broker " + brokerHostingPartition + " is unavailable. Cannot issue " + "getOffsetsBefore request") } - val consumer = new SimpleConsumer(broker.host, broker.port, 10000, 100 * 1024) - val offsets = consumer.getOffsetsBefore(topic, part.partId, offsetOption, 1) + + val brokerInfos = ZkUtils.getBrokerInfoFromIds(zkClient, List(broker)) + if(brokerInfos.size == 0) + throw new IllegalStateException("Broker information for broker id %d does not exist in ZK".format(broker)) + + val brokerInfo = brokerInfos.head + val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024) + val offsets = consumer.getOffsetsBefore(topic, partition.toInt, offsetOption, 1) val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) - - println("updating partition " + part.name + " with new offset: " + offsets(0)) - ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + part.name, offsets(0).toString) + + println("updating partition " + partition + " with new offset: " + offsets(0)) + ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + partition, offsets(0).toString) numParts += 1 } - println("updated the offset for " + numParts + " partitions") + println("updated the offset for " + numParts + " partitions") } private def usage() = { diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 43ec48855a5..cb69914dae6 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -29,6 +29,7 @@ import scala.collection.mutable import kafka.message.{NoCompressionCodec, CompressionCodec} import org.I0Itec.zkclient.ZkClient import java.util.{Random, Properties} +import kafka.network.{BoundedByteBufferReceive, Receive, BoundedByteBufferSend, Request} /** * Helper functions! @@ -669,6 +670,20 @@ object Utils extends Logging { case _ => // swallow } } + + def sendRequest(request: Request, channel: SocketChannel) = { + val send = new BoundedByteBufferSend(request) + send.writeCompletely(channel) + } + + def getResponse(channel: SocketChannel): Tuple2[Receive,Int] = { + val response = new BoundedByteBufferReceive() + response.readCompletely(channel) + + // this has the side effect of setting the initial position of buffer correctly + val errorCode: Int = response.buffer.getShort + (response, errorCode) + } } class SnapshotStats(private val monitorDurationNs: Long = 600L * 1000L * 1000L * 1000L) { diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 9599c1580f2..46cb8b1b9ad 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -35,7 +35,7 @@ object ZkUtils extends Logging { } def getTopicPartitionsPath(topic: String): String ={ - getTopicPath(topic) + "/" + "partitions" + getTopicPath(topic) + "/partitions" } def getTopicPartitionPath(topic: String, partitionId: String): String ={ @@ -62,6 +62,38 @@ object ZkUtils extends Logging { ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted } + def getAllBrokersInCluster(zkClient: ZkClient): Seq[Broker] = { + val brokerIds = ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted + getBrokerInfoFromIds(zkClient, brokerIds.map(b => b.toInt)) + } + + def getLeaderForPartition(zkClient: ZkClient, topic: String, partition: Int): Option[Int] = { + // TODO: When leader election is implemented, change this method to return the leader as follows + // until then, assume the first replica as the leader +// val leader = readDataMaybeNull(zkClient, getTopicPartitionLeaderPath(topic, partition.toString)) + val replicaListString = readDataMaybeNull(zkClient, getTopicPartitionReplicasPath(topic, partition.toString)) + val replicas = Utils.getCSVList(replicaListString) + replicas.size match { + case 0 => None + case _ => Some(replicas.head.toInt) + } + } + + def getReplicasForPartition(zkClient: ZkClient, topic: String, partition: Int): Seq[String] = { + val replicaListString = readDataMaybeNull(zkClient, getTopicPartitionReplicasPath(topic, partition.toString)) + if(replicaListString == null) + Seq.empty[String] + else { + Utils.getCSVList(replicaListString) + } + } + + def isPartitionOnBroker(zkClient: ZkClient, topic: String, partition: Int, brokerId: Int): Boolean = { + val replicas = getReplicasForPartition(zkClient, topic, partition) + debug("The list of replicas for topic %s, partition %d is %s".format(topic, partition, replicas)) + replicas.contains(brokerId.toString) + } + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, creator: String, port: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val broker = new Broker(id, creator, host, port) @@ -77,6 +109,11 @@ object ZkUtils extends Logging { info("Registering broker " + brokerIdPath + " succeeded with " + broker) } + def getConsumerPartitionOwnerPath(group: String, topic: String, partition: String): String = { + val topicDirs = new ZKGroupTopicDirs(group, topic) + topicDirs.consumerOwnerDir + "/" + partition + } + /** * make sure a persistent path exists in ZK. Create the path if not exist. */ @@ -269,30 +306,17 @@ object ZkUtils extends Logging { cluster } - def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, List[String]] = { - val ret = new mutable.HashMap[String, List[String]]() - for (topic <- topics) { - var partList: List[String] = Nil - val brokers = getChildrenParentMayNotExist(zkClient, BrokerTopicsPath + "/" + topic) - for (broker <- brokers) { - val nParts = readData(zkClient, BrokerTopicsPath + "/" + topic + "/" + broker).toInt - for (part <- 0 until nParts) - partList ::= broker + "-" + part - } - partList = partList.sortWith((s,t) => s < t) - ret += (topic -> partList) + def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, Seq[String]] = { + val ret = new mutable.HashMap[String, Seq[String]]() + topics.foreach { topic => + // get the partitions that exist for topic + val partitions = getChildrenParentMayNotExist(zkClient, getTopicPartitionsPath(topic)) + debug("children of /brokers/topics/%s are %s".format(topic, partitions)) + ret += (topic -> partitions.sortWith((s,t) => s < t)) } ret } - def setupPartition(zkClient : ZkClient, brokerId: Int, host: String, port: Int, topic: String, nParts: Int) { - val brokerIdPath = BrokerIdsPath + "/" + brokerId - val broker = new Broker(brokerId, brokerId.toString, host, port) - createEphemeralPathExpectConflict(zkClient, brokerIdPath, broker.getZKString) - val brokerPartTopicPath = BrokerTopicsPath + "/" + topic + "/" + brokerId - createEphemeralPathExpectConflict(zkClient, brokerPartTopicPath, nParts.toString) - } - def deletePartition(zkClient : ZkClient, brokerId: Int, topic: String) { val brokerIdPath = BrokerIdsPath + "/" + brokerId zkClient.delete(brokerIdPath) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index b8b0500ae7b..236bf014b17 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -169,7 +169,8 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness { case Some(metadata) => assertEquals(topic, metadata.topic) assertNotNull("partition metadata list cannot be null", metadata.partitionsMetadata) assertEquals("partition metadata list length should be 2", 2, metadata.partitionsMetadata.size) - assertNull("leader should not be assigned for now", metadata.partitionsMetadata.head.leader.getOrElse(null)) + assertEquals("leader of partition 0 should be 0", 0, metadata.partitionsMetadata.head.leader.get.id) + assertEquals("leader of partition 1 should be 1", 1, metadata.partitionsMetadata.last.leader.get.id) val actualReplicaAssignment = metadata.partitionsMetadata.map(p => p.replicas) val actualReplicaList = actualReplicaAssignment.map(r => r.map(b => b.id.toString).toList).toList assertEquals(expectedReplicaAssignment.toList, actualReplicaList) diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index ff0c0798b7c..b42a8b3baff 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -22,15 +22,19 @@ import junit.framework.Assert._ import kafka.integration.KafkaServerTestHarness import kafka.server._ import scala.collection._ -import kafka.utils.{Utils, Logging} -import kafka.utils.{TestZKUtils, TestUtils} import org.scalatest.junit.JUnit3Suite import org.apache.log4j.{Level, Logger} import kafka.message._ import kafka.serializer.StringDecoder +import kafka.admin.CreateTopicCommand +import org.I0Itec.zkclient.ZkClient +import kafka.utils._ +import kafka.producer.{ProducerConfig, ProducerData, Producer} +import java.util.{Collections, Properties} class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { + var dirs : ZKGroupTopicDirs = null val zookeeperConnect = TestZKUtils.zookeeperConnect val numNodes = 2 val numParts = 2 @@ -48,25 +52,28 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val consumer3 = "consumer3" val nMessages = 2 + override def setUp() { + super.setUp() + dirs = new ZKGroupTopicDirs(group, topic) + } + def testBasic() { val requestHandlerLogger = Logger.getLogger(classOf[KafkaApis]) requestHandlerLogger.setLevel(Level.FATAL) - var actualMessages: List[Message] = Nil - // test consumer timeout logic val consumerConfig0 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { override val consumerTimeoutMs = 200 } val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true) - val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) + val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> 1)) // no messages to consume, we should hit timeout; // also the iterator should support re-entrant, so loop it twice - for (i <- 0 until 2) { + for (i <- 0 until 2) { try { - getMessagesSortedByChecksum(nMessages*2, topicMessageStreams0) + getMessages(nMessages*2, topicMessageStreams0) fail("should get an exception") } catch { @@ -78,14 +85,26 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar zkConsumerConnector0.shutdown // send some messages to each broker - val sentMessages1 = sendMessages(nMessages, "batch1") + val sentMessages1_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) + val sentMessages1_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages1 = (sentMessages1_1 ++ sentMessages1_2).sortWith((s,t) => s.checksum < t.checksum) + // create a consumer val consumerConfig1 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) - val receivedMessages1 = getMessagesSortedByChecksum(nMessages*2, topicMessageStreams1) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) + val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) + assertEquals(sentMessages1.size, receivedMessages1.size) assertEquals(sentMessages1, receivedMessages1) + + // also check partition ownership + val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_1 = List( ("0", "group1_consumer1-0"), + ("1", "group1_consumer1-0")) +// assertEquals(expected_1, actual_1) + assertEquals(expected_1, actual_1) + // commit consumed offsets zkConsumerConnector1.commitOffsets @@ -93,15 +112,25 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val consumerConfig2 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer2)) val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) - val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) + val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> 1)) // send some messages to each broker - val sentMessages2 = sendMessages(nMessages, "batch2") + val sentMessages2_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) + val sentMessages2_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages2 = (sentMessages2_1 ++ sentMessages2_2).sortWith((s,t) => s.checksum < t.checksum) + Thread.sleep(200) - val receivedMessages2_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1) - val receivedMessages2_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2) + + val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2) val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum) assertEquals(sentMessages2, receivedMessages2) + // also check partition ownership + val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_2 = List( ("0", "group1_consumer1-0"), + ("1", "group1_consumer2-0")) + assertEquals(expected_2, actual_2) + // create a consumer with empty map val consumerConfig3 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer3)) @@ -109,13 +138,20 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) // send some messages to each broker Thread.sleep(200) - val sentMessages3 = sendMessages(nMessages, "batch3") + val sentMessages3_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) + val sentMessages3_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages3 = (sentMessages3_1 ++ sentMessages3_2).sortWith((s,t) => s.checksum < t.checksum) Thread.sleep(200) - val receivedMessages3_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1) - val receivedMessages3_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2) + val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2) val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum) + assertEquals(sentMessages3.size, receivedMessages3.size) assertEquals(sentMessages3, receivedMessages3) + // also check partition ownership + val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir) + assertEquals(expected_2, actual_3) + zkConsumerConnector1.shutdown zkConsumerConnector2.shutdown zkConsumerConnector3.shutdown @@ -127,48 +163,73 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) - println("Sending messages for 1st consumer") // send some messages to each broker - val sentMessages1 = sendMessages(nMessages, "batch1", DefaultCompressionCodec) + val sentMessages1_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) + val sentMessages1_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages1 = (sentMessages1_1 ++ sentMessages1_2).sortWith((s,t) => s.checksum < t.checksum) + // create a consumer val consumerConfig1 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) - val receivedMessages1 = getMessagesSortedByChecksum(nMessages*2, topicMessageStreams1) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) + val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) + assertEquals(sentMessages1.size, receivedMessages1.size) assertEquals(sentMessages1, receivedMessages1) + + // also check partition ownership + val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_1 = List( ("0", "group1_consumer1-0"), + ("1", "group1_consumer1-0")) + assertEquals(expected_1, actual_1) + // commit consumed offsets zkConsumerConnector1.commitOffsets - println("Sending more messages for 2nd consumer") // create a consumer val consumerConfig2 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer2)) val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) - val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) + val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> 1)) // send some messages to each broker - val sentMessages2 = sendMessages(nMessages, "batch2", DefaultCompressionCodec) + val sentMessages2_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) + val sentMessages2_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages2 = (sentMessages2_1 ++ sentMessages2_2).sortWith((s,t) => s.checksum < t.checksum) + Thread.sleep(200) - val receivedMessages2_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1) - val receivedMessages2_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2) + + val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2) val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum) assertEquals(sentMessages2, receivedMessages2) + // also check partition ownership + val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_2 = List( ("0", "group1_consumer1-0"), + ("1", "group1_consumer2-0")) + assertEquals(expected_2, actual_2) + // create a consumer with empty map - println("Sending more messages for 3rd consumer") val consumerConfig3 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer3)) val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) // send some messages to each broker Thread.sleep(200) - val sentMessages3 = sendMessages(nMessages, "batch3", DefaultCompressionCodec) + val sentMessages3_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) + val sentMessages3_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages3 = (sentMessages3_1 ++ sentMessages3_2).sortWith((s,t) => s.checksum < t.checksum) Thread.sleep(200) - val receivedMessages3_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1) - val receivedMessages3_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2) + val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2) val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum) + assertEquals(sentMessages3.size, receivedMessages3.size) assertEquals(sentMessages3, receivedMessages3) + // also check partition ownership + val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir) + assertEquals(expected_2, actual_3) + zkConsumerConnector1.shutdown zkConsumerConnector2.shutdown zkConsumerConnector3.shutdown @@ -187,7 +248,10 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar Thread.sleep(500) // send some messages to each broker - val sentMessages = sendMessages(configs.head, 200, "batch1", DefaultCompressionCodec) + val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, 200, DefaultCompressionCodec) + val sentMessages2 = sendMessagesToBrokerPartition(configs.last, topic, 1, 200, DefaultCompressionCodec) + val sentMessages = (sentMessages1 ++ sentMessages2).sortWith((s,t) => s.checksum < t.checksum) + // test consumer timeout logic val consumerConfig0 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { @@ -195,16 +259,30 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar } val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true) val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> 1)) - getMessagesSortedByChecksum(100, topicMessageStreams0) + getMessages(100, topicMessageStreams0) + + // also check partition ownership + val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_1 = List( ("0", "group1_consumer0-0"), + ("1", "group1_consumer0-0")) + assertEquals(expected_1, actual_1) + zkConsumerConnector0.shutdown // at this point, only some part of the message set was consumed. So consumed offset should still be 0 // also fetched offset should be 0 val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig0, true) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) - val receivedMessages = getMessagesSortedByChecksum(400, topicMessageStreams1) + val receivedMessages = getMessages(400, topicMessageStreams1) val sortedReceivedMessages = receivedMessages.sortWith((s,t) => s.checksum < t.checksum) val sortedSentMessages = sentMessages.sortWith((s,t) => s.checksum < t.checksum) assertEquals(sortedSentMessages, sortedReceivedMessages) + + // also check partition ownership + val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_2 = List( ("0", "group1_consumer0-0"), + ("1", "group1_consumer0-0")) + assertEquals(expected_2, actual_2) + zkConsumerConnector1.shutdown requestHandlerLogger.setLevel(Level.ERROR) @@ -214,17 +292,18 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) - val sentMessages = sendMessages(nMessages, "batch1", NoCompressionCodec). - map(m => Utils.toString(m.payload, "UTF-8")). + // send some messages to each broker + val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, NoCompressionCodec) + val sentMessages2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, NoCompressionCodec) + val sentMessages = (sentMessages1 ++ sentMessages2).map(m => Utils.toString(m.payload, "UTF-8")). sortWith((s, t) => s.compare(t) == -1) - val consumerConfig = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer1)) + + val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector = new ZookeeperConsumerConnector(consumerConfig, true) val topicMessageStreams = - zkConsumerConnector.createMessageStreams( - Predef.Map(topic -> numNodes*numParts/2), new StringDecoder) + zkConsumerConnector.createMessageStreams(Predef.Map(topic -> 1), new StringDecoder) var receivedMessages: List[String] = Nil for ((topic, messageStreams) <- topicMessageStreams) { @@ -245,31 +324,106 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.ERROR) } - def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compression: CompressionCodec): List[Message]= { + def testLeaderSelectionForPartition() { + val zkClient = new ZkClient(zookeeperConnect, 6000, 30000, ZKStringSerializer) + + // create topic topic1 with 1 partition on broker 0 + CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") + + // send some messages to each broker + val sentMessages1 = sendMessages(configs.head, nMessages, "batch1", NoCompressionCodec, 1) + + // create a consumer + val consumerConfig1 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer1)) + val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) + val topicRegistry = zkConsumerConnector1.getTopicRegistry + assertEquals(1, topicRegistry.map(r => r._1).size) + assertEquals(topic, topicRegistry.map(r => r._1).head) + val topicsAndPartitionsInRegistry = topicRegistry.map(r => (r._1, r._2.map(p => p._1))) + val brokerPartition = topicsAndPartitionsInRegistry.head._2.head + assertEquals(0, brokerPartition.brokerId) + assertEquals(0, brokerPartition.partId) + + // also check partition ownership + val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_1 = List( ("0", "group1_consumer1-0")) + assertEquals(expected_1, actual_1) + + val receivedMessages1 = getMessages(nMessages, topicMessageStreams1) + assertEquals(nMessages, receivedMessages1.size) + assertEquals(sentMessages1, receivedMessages1) + } + + def sendMessagesToBrokerPartition(config: KafkaConfig, topic: String, partition: Int, numMessages: Int, + compression: CompressionCodec = NoCompressionCodec): List[Message] = { + val header = "test-%d-%d".format(config.brokerId, partition) + val props = new Properties() + props.put("zk.connect", zkConnect) + props.put("partitioner.class", "kafka.utils.FixedValuePartitioner") + props.put("compression.codec", compression.codec.toString) + val producer: Producer[Int, Message] = new Producer[Int, Message](new ProducerConfig(props)) + val ms = 0.until(numMessages).map(x => + new Message((header + config.brokerId + "-" + partition + "-" + x).getBytes)).toArray + producer.send(new ProducerData[Int, Message](topic, partition, ms)) + debug("Sent %d messages to broker %d for topic %s and partition %d".format(ms.size, config.brokerId, topic, partition)) + producer + ms.toList + } + + def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compression: CompressionCodec, numParts: Int): List[Message]= { var messages: List[Message] = Nil - val producer = TestUtils.createProducer("localhost", conf.port) + val props = new Properties() + props.put("zk.connect", zkConnect) + props.put("partitioner.class", "kafka.utils.FixedValuePartitioner") + val producer: Producer[Int, Message] = new Producer[Int, Message](new ProducerConfig(props)) + for (partition <- 0 until numParts) { val ms = 0.until(messagesPerNode).map(x => new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray - val mSet = new ByteBufferMessageSet(compressionCodec = compression, messages = ms: _*) for (message <- ms) messages ::= message - producer.send(topic, partition, mSet) + producer.send(new ProducerData[Int, Message](topic, partition, ms)) + debug("Sent %d messages to broker %d for topic %s and partition %d".format(ms.size, conf.brokerId, topic, partition)) } producer.close() - messages + messages.reverse } def sendMessages(messagesPerNode: Int, header: String, compression: CompressionCodec = NoCompressionCodec): List[Message]= { var messages: List[Message] = Nil for(conf <- configs) { - messages ++= sendMessages(conf, messagesPerNode, header, compression) + messages ++= sendMessages(conf, messagesPerNode, header, compression, numParts) } messages.sortWith((s,t) => s.checksum < t.checksum) } - def getMessagesSortedByChecksum(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[Message]]]): List[Message]= { - val messages = TestUtils.getConsumedMessages(nMessagesPerThread, topicMessageStreams) + def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[Message]]]): List[Message]= { + var messages: List[Message] = Nil + for ((topic, messageStreams) <- topicMessageStreams) { + for (messageStream <- messageStreams) { + val iterator = messageStream.iterator + for (i <- 0 until nMessagesPerThread) { + assertTrue(iterator.hasNext) + val message = iterator.next + messages ::= message + debug("received message: " + Utils.toString(message.payload, "UTF-8")) + } + } + } messages.sortWith((s,t) => s.checksum < t.checksum) } + + def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { + import scala.collection.JavaConversions + val children = zookeeper.client.getChildren(path) + Collections.sort(children) + val childrenAsSeq : Seq[java.lang.String] = JavaConversions.asBuffer(children) + childrenAsSeq.map(partition => + (partition, zookeeper.client.readData(path + "/" + partition).asInstanceOf[String])) + } + } + + diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala index 6033fcc511e..bc613466883 100644 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala @@ -18,7 +18,6 @@ package kafka.integration import junit.framework.Assert._ -import kafka.zk.ZooKeeperTestHarness import java.nio.channels.ClosedByInterruptException import java.util.concurrent.atomic.AtomicInteger import kafka.utils.{ZKGroupTopicDirs, Logging} @@ -27,15 +26,16 @@ import kafka.server._ import org.apache.log4j.{Level, Logger} import org.scalatest.junit.JUnit3Suite import kafka.utils.TestUtils +import kafka.message.Message +import kafka.producer.{Producer, ProducerData} -class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { +class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with Logging { val topic = "test_topic" val group = "default_group" val testConsumer = "consumer" val brokerPort = 9892 - val kafkaConfig = new KafkaConfig(TestUtils.createBrokerConfig(0, brokerPort)) - var kafkaServer : KafkaServer = null + val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0, brokerPort))) val numMessages = 10 val largeOffset = 10000 val smallOffset = -1 @@ -44,7 +44,6 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log override def setUp() { super.setUp() - kafkaServer = TestUtils.createServer(kafkaConfig) // temporarily set request handler logger to a higher level requestHandlerLogger.setLevel(Level.FATAL) @@ -53,15 +52,14 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log override def tearDown() { // restore set request handler logger to a higher level requestHandlerLogger.setLevel(Level.ERROR) - kafkaServer.shutdown super.tearDown } def testEarliestOffsetResetForward() = { - val producer = TestUtils.createProducer("localhost", brokerPort) + val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect) for(i <- 0 until numMessages) { - producer.send(topic, TestUtils.singleMessageSet("test".getBytes())) + producer.send(new ProducerData[String, Message](topic, topic, new Message("test".getBytes()))) } // update offset in zookeeper for consumer to jump "forward" in time @@ -71,7 +69,7 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log consumerProps.put("consumer.timeout.ms", "2000") val consumerConfig = new ConsumerConfig(consumerProps) - TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", largeOffset) + TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0", largeOffset) info("Updated consumer offset to " + largeOffset) Thread.sleep(500) @@ -112,10 +110,10 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log } def testEarliestOffsetResetBackward() = { - val producer = TestUtils.createProducer("localhost", brokerPort) + val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect) for(i <- 0 until numMessages) { - producer.send(topic, TestUtils.singleMessageSet("test".getBytes())) + producer.send(new ProducerData[String, Message](topic, topic, new Message("test".getBytes()))) } // update offset in zookeeper for consumer to jump "forward" in time @@ -125,7 +123,7 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log consumerProps.put("consumer.timeout.ms", "2000") val consumerConfig = new ConsumerConfig(consumerProps) - TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", smallOffset) + TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0", smallOffset) info("Updated consumer offset to " + smallOffset) @@ -145,7 +143,7 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log } } catch { - case _: InterruptedException => + case _: InterruptedException => case _: ClosedByInterruptException => case e => throw e } @@ -159,16 +157,15 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log threadList(0).join(2000) - info("Asserting...") assertEquals(numMessages, nMessages.get) consumerConnector.shutdown } def testLatestOffsetResetForward() = { - val producer = TestUtils.createProducer("localhost", brokerPort) + val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect) for(i <- 0 until numMessages) { - producer.send(topic, TestUtils.singleMessageSet("test".getBytes())) + producer.send(new ProducerData[String, Message](topic, topic, new Message("test".getBytes()))) } // update offset in zookeeper for consumer to jump "forward" in time @@ -178,7 +175,7 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log consumerProps.put("consumer.timeout.ms", "2000") val consumerConfig = new ConsumerConfig(consumerProps) - TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", largeOffset) + TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0", largeOffset) info("Updated consumer offset to " + largeOffset) @@ -198,7 +195,7 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log } } catch { - case _: InterruptedException => + case _: InterruptedException => case _: ClosedByInterruptException => case e => throw e } diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index b089d6184b8..6e9ad216bbb 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -28,6 +28,7 @@ import kafka.server._ import org.scalatest.junit.JUnit3Suite import kafka.integration.KafkaServerTestHarness import kafka.utils.TestUtils +import kafka.producer.{ProducerData, Producer} class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { @@ -35,7 +36,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { val configs = for(props <- TestUtils.createBrokerConfigs(numNodes)) yield new KafkaConfig(props) - val messages = new mutable.HashMap[Int, ByteBufferMessageSet] + val messages = new mutable.HashMap[Int, Seq[Message]] val topic = "topic" val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.brokerId.toString, "localhost", c.port))) val shutdown = ZookeeperConsumerConnector.shutdownCommand @@ -79,11 +80,10 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { def sendMessages(messagesPerNode: Int): Int = { var count = 0 for(conf <- configs) { - val producer = TestUtils.createProducer("localhost", conf.port) + val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect) val ms = 0.until(messagesPerNode).map(x => new Message((conf.brokerId * 5 + x).toString.getBytes)).toArray - val mSet = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = ms: _*) - messages += conf.brokerId -> mSet - producer.send(topic, mSet) + messages += conf.brokerId -> ms + producer.send(new ProducerData[String, Message](topic, topic, ms)) producer.close() count += ms.size } diff --git a/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala b/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala index ee1c8a9e052..11020a4c0de 100644 --- a/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala +++ b/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala @@ -17,33 +17,31 @@ package kafka.integration -import kafka.api.{FetchRequestBuilder, ProducerRequest} +import kafka.api.FetchRequestBuilder import kafka.common.OffsetOutOfRangeException -import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} -import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig} -import kafka.utils.{TestUtils, Utils} -import kafka.zk.ZooKeeperTestHarness +import kafka.message.{Message, ByteBufferMessageSet} +import kafka.server.{KafkaRequestHandler, KafkaConfig} +import kafka.utils.TestUtils import org.apache.log4j.{Level, Logger} import org.scalatest.junit.JUnit3Suite import scala.collection._ +import kafka.producer.ProducerData /** * End to end tests of the primitive apis against a local server */ -class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness { +class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness { val port = TestUtils.choosePort val props = TestUtils.createBrokerConfig(0, port) val config = new KafkaConfig(props) val configs = List(config) - var servers: List[KafkaServer] = null val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) override def setUp() { super.setUp if(configs.size <= 0) throw new IllegalArgumentException("Must suply at least one server config.") - servers = configs.map(TestUtils.createServer(_)) // temporarily set request handler logger to a higher level requestHandlerLogger.setLevel(Level.FATAL) @@ -54,24 +52,21 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness requestHandlerLogger.setLevel(Level.ERROR) super.tearDown - servers.map(server => server.shutdown()) - servers.map(server => Utils.rm(server.config.logDir)) } def testProduceAndFetch() { // send some messages val topic = "test" - val sent = new ByteBufferMessageSet(NoCompressionCodec, - new Message("hello".getBytes()), new Message("there".getBytes())) - producer.send(topic, sent) - sent.getBuffer.rewind + val sentMessages = List(new Message("hello".getBytes()), new Message("there".getBytes())) + val producerData = new ProducerData[String, Message](topic, topic, sentMessages) + producer.send(producerData) var fetchedMessage: ByteBufferMessageSet = null while(fetchedMessage == null || fetchedMessage.validBytes == 0) { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) fetchedMessage = fetched.messageSet(topic, 0) } - TestUtils.checkEquals(sent.iterator, fetchedMessage.iterator) + TestUtils.checkEquals(sentMessages.iterator, fetchedMessage.map(m => m.message).iterator) // send an invalid offset try { @@ -87,14 +82,12 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness // send some messages, with non-ordered topics val topicOffsets = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); { - val messages = new mutable.HashMap[String, ByteBufferMessageSet] + val messages = new mutable.HashMap[String, Seq[Message]] val builder = new FetchRequestBuilder() for( (topic, offset) <- topicOffsets) { - val set = new ByteBufferMessageSet(NoCompressionCodec, - new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) - producer.send(topic, set) - set.getBuffer.rewind - messages += topic -> set + val producedData = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) + messages += topic -> producedData + producer.send(new ProducerData[String, Message](topic, topic, producedData)) builder.addFetch(topic, offset, 0, 10000) } @@ -104,7 +97,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness val response = consumer.fetch(request) for( (topic, offset) <- topicOffsets) { val fetched = response.messageSet(topic, offset) - TestUtils.checkEquals(messages(topic).iterator, fetched.iterator) + TestUtils.checkEquals(messages(topic).iterator, fetched.map(m => m.message).iterator) } } @@ -121,7 +114,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness responses.messageSet(topic, offset).iterator fail("Expected an OffsetOutOfRangeException exception to be thrown") } catch { - case e: OffsetOutOfRangeException => + case e: OffsetOutOfRangeException => } } } @@ -130,20 +123,16 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness def testMultiProduce() { // send some messages val topics = List("test1", "test2", "test3"); - val messages = new mutable.HashMap[String, ByteBufferMessageSet] + val messages = new mutable.HashMap[String, Seq[Message]] val builder = new FetchRequestBuilder() - var produceList: List[ProducerRequest] = Nil + var produceList: List[ProducerData[String, Message]] = Nil for(topic <- topics) { - val set = new ByteBufferMessageSet(NoCompressionCodec, - new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) + val set = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) messages += topic -> set - produceList ::= new ProducerRequest(topic, 0, set) + produceList ::= new ProducerData[String, Message](topic, topic, set) builder.addFetch(topic, 0, 0, 10000) } - producer.multiSend(produceList.toArray) - - for (messageSet <- messages.values) - messageSet.getBuffer.rewind + producer.send(produceList: _*) // wait a bit for produced message to be available Thread.sleep(200) @@ -151,39 +140,33 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness val response = consumer.fetch(request) for(topic <- topics) { val fetched = response.messageSet(topic, 0) - TestUtils.checkEquals(messages(topic).iterator, fetched.iterator) + TestUtils.checkEquals(messages(topic).iterator, fetched.map(m => m.message).iterator) } } def testMultiProduceResend() { // send some messages val topics = List("test1", "test2", "test3"); - val messages = new mutable.HashMap[String, ByteBufferMessageSet] + val messages = new mutable.HashMap[String, Seq[Message]] val builder = new FetchRequestBuilder() - var produceList: List[ProducerRequest] = Nil + var produceList: List[ProducerData[String, Message]] = Nil for(topic <- topics) { - val set = new ByteBufferMessageSet(NoCompressionCodec, - new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) + val set = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) messages += topic -> set - produceList ::= new ProducerRequest(topic, 0, set) + produceList ::= new ProducerData[String, Message](topic, topic, set) builder.addFetch(topic, 0, 0, 10000) } - producer.multiSend(produceList.toArray) - - // resend the same multisend - producer.multiSend(produceList.toArray) - - for (messageSet <- messages.values) - messageSet.getBuffer.rewind + producer.send(produceList: _*) + producer.send(produceList: _*) // wait a bit for produced message to be available Thread.sleep(750) val request = builder.build() val response = consumer.fetch(request) for(topic <- topics) { val topicMessages = response.messageSet(topic, 0) - TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).map(m => m.message).iterator, - messages(topic).map(m => m.message).iterator), + TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).iterator, + messages(topic).iterator), topicMessages.iterator.map(_.message)) } } diff --git a/core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala b/core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala index 68f7c868dee..dda798b4f50 100644 --- a/core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala @@ -24,10 +24,11 @@ import kafka.api.FetchRequestBuilder import kafka.common.InvalidMessageSizeException import kafka.consumer.{ZookeeperConsumerConnector, ConsumerConfig} import kafka.integration.{KafkaServerTestHarness, ProducerConsumerTestHarness} -import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} +import kafka.message.Message import kafka.utils.{Utils, TestUtils} import org.scalatest.junit.JUnit3Suite import org.apache.log4j.{Logger, Level} +import kafka.producer.ProducerData class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness { val port = TestUtils.choosePort @@ -47,8 +48,9 @@ class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness wit fetcherLogger.setLevel(Level.FATAL) // send some messages - val sent1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message("hello".getBytes())) - producer.send(topic, sent1) + val producerData = new ProducerData[String, Message](topic, topic, List(new Message("hello".getBytes()))) + + producer.send(producerData) Thread.sleep(200) // corrupt the file on disk diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index 64a6293c2f4..67fbcc17961 100644 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -17,25 +17,25 @@ package kafka.integration -import java.io.File import java.nio.ByteBuffer -import java.util.Properties import junit.framework.Assert._ -import kafka.api.{OffsetDetail, FetchRequest, FetchRequestBuilder, ProducerRequest} +import kafka.api.{OffsetDetail, FetchRequest, FetchRequestBuilder} import kafka.common.{FetchRequestFormatException, OffsetOutOfRangeException, InvalidPartitionException} -import kafka.message.{DefaultCompressionCodec, NoCompressionCodec, Message, ByteBufferMessageSet} -import kafka.producer.{ProducerData, Producer, ProducerConfig} -import kafka.serializer.StringDecoder import kafka.server.{KafkaRequestHandler, KafkaConfig} -import kafka.utils.TestUtils import org.apache.log4j.{Level, Logger} import org.scalatest.junit.JUnit3Suite +import java.util.Properties +import kafka.producer.{ProducerData, Producer, ProducerConfig} +import kafka.serializer.StringDecoder +import kafka.message.Message +import java.io.File +import kafka.utils.{TestZKUtils, TestUtils} import scala.collection._ /** * End to end tests of the primitive apis against a local server */ -class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness { +class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness { val port = TestUtils.choosePort val props = TestUtils.createBrokerConfig(0, port) @@ -45,6 +45,20 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val configs = List(config) val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) +//<<<<<<< .mine + override def setUp() { + super.setUp + // temporarily set request handler logger to a higher level + requestHandlerLogger.setLevel(Level.FATAL) + } + + override def tearDown() { + // restore set request handler logger to a higher level + requestHandlerLogger.setLevel(Level.ERROR) + + super.tearDown + } + def testFetchRequestCanProperlySerialize() { val request = new FetchRequestBuilder() .correlationId(100) @@ -83,7 +97,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val topic = "test-topic" val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("broker.list", "0:localhost:" + port) + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new ProducerConfig(props) val stringProducer1 = new Producer[String, String](config) @@ -111,7 +125,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val topic = "test-topic" val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("broker.list", "0:localhost:" + port) + props.put("zk.connect", TestZKUtils.zookeeperConnect) props.put("compression", "true") val config = new ProducerConfig(props) @@ -133,14 +147,13 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with // send some messages val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); { - val messages = new mutable.HashMap[String, ByteBufferMessageSet] + val messages = new mutable.HashMap[String, Seq[Message]] val builder = new FetchRequestBuilder() for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(NoCompressionCodec, - new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) - messages += topic -> set - producer.send(topic, set) - set.getBuffer.rewind + val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) + val producerData = new ProducerData[String, Message](topic, topic, messageList) + messages += topic -> messageList + producer.send(producerData) builder.addFetch(topic, partition, 0, 10000) } @@ -150,7 +163,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val response = consumer.fetch(request) for( (topic, partition) <- topics) { val fetched = response.messageSet(topic, partition) - TestUtils.checkEquals(messages(topic).iterator, fetched.iterator) + TestUtils.checkEquals(messages(topic).iterator, fetched.map(messageAndOffset => messageAndOffset.message).iterator) } } @@ -172,7 +185,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with } catch { case e: OffsetOutOfRangeException => "this is good" } - } + } { // send some invalid partitions @@ -199,14 +212,13 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with // send some messages val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); { - val messages = new mutable.HashMap[String, ByteBufferMessageSet] + val messages = new mutable.HashMap[String, Seq[Message]] val builder = new FetchRequestBuilder() for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(DefaultCompressionCodec, - new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) - messages += topic -> set - producer.send(topic, set) - set.getBuffer.rewind + val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) + val producerData = new ProducerData[String, Message](topic, topic, messageList) + messages += topic -> messageList + producer.send(producerData) builder.addFetch(topic, partition, 0, 10000) } @@ -216,7 +228,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val response = consumer.fetch(request) for( (topic, partition) <- topics) { val fetched = response.messageSet(topic, partition) - TestUtils.checkEquals(messages(topic).iterator, fetched.iterator) + TestUtils.checkEquals(messages(topic).iterator, fetched.map(messageAndOffset => messageAndOffset.message).iterator) } } @@ -264,48 +276,42 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with def testMultiProduce() { // send some messages val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - val messages = new mutable.HashMap[String, ByteBufferMessageSet] + val messages = new mutable.HashMap[String, Seq[Message]] val builder = new FetchRequestBuilder() - var produceList: List[ProducerRequest] = Nil + var produceList: List[ProducerData[String, Message]] = Nil for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(NoCompressionCodec, - new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) - messages += topic -> set - produceList ::= new ProducerRequest(topic, 0, set) + val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) + val producerData = new ProducerData[String, Message](topic, topic, messageList) + messages += topic -> messageList + producer.send(producerData) builder.addFetch(topic, partition, 0, 10000) } - producer.multiSend(produceList.toArray) + producer.send(produceList: _*) - for (messageSet <- messages.values) - messageSet.getBuffer.rewind - // wait a bit for produced message to be available Thread.sleep(200) val request = builder.build() val response = consumer.fetch(request) for( (topic, partition) <- topics) { val fetched = response.messageSet(topic, partition) - TestUtils.checkEquals(messages(topic).iterator, fetched.iterator) + TestUtils.checkEquals(messages(topic).iterator, fetched.map(messageAndOffset => messageAndOffset.message).iterator) } } def testMultiProduceWithCompression() { // send some messages val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - val messages = new mutable.HashMap[String, ByteBufferMessageSet] + val messages = new mutable.HashMap[String, Seq[Message]] val builder = new FetchRequestBuilder() - var produceList: List[ProducerRequest] = Nil + var produceList: List[ProducerData[String, Message]] = Nil for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(DefaultCompressionCodec, - new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) - messages += topic -> set - produceList ::= new ProducerRequest(topic, 0, set) + val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) + val producerData = new ProducerData[String, Message](topic, topic, messageList) + messages += topic -> messageList + producer.send(producerData) builder.addFetch(topic, partition, 0, 10000) } - producer.multiSend(produceList.toArray) - - for (messageSet <- messages.values) - messageSet.getBuffer.rewind + producer.send(produceList: _*) // wait a bit for produced message to be available Thread.sleep(200) @@ -313,7 +319,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val response = consumer.fetch(request) for( (topic, partition) <- topics) { val fetched = response.messageSet(topic, 0) - TestUtils.checkEquals(messages(topic).iterator, fetched.iterator) + TestUtils.checkEquals(messages(topic).iterator, fetched.map(messageAndOffset => messageAndOffset.message).iterator) } } diff --git a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala index 76ae0b1acd3..c7a24f457fd 100644 --- a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala @@ -20,33 +20,35 @@ package kafka.integration import kafka.consumer.SimpleConsumer import org.scalatest.junit.JUnit3Suite import java.util.Properties -import kafka.producer.{SyncProducerConfig, SyncProducer} +import kafka.utils.TestZKUtils +import kafka.producer.{ProducerConfig, Producer} +import kafka.message.Message -trait ProducerConsumerTestHarness extends JUnit3Suite { +trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarness { val port: Int val host = "localhost" - var producer: SyncProducer = null + var producer: Producer[String, Message] = null var consumer: SimpleConsumer = null override def setUp() { + super.setUp val props = new Properties() - props.put("host", host) - props.put("port", port.toString) + props.put("partitioner.class", "kafka.utils.StaticPartitioner") + props.put("zk.connect", TestZKUtils.zookeeperConnect) props.put("buffer.size", "65536") props.put("connect.timeout.ms", "100000") props.put("reconnect.interval", "10000") - producer = new SyncProducer(new SyncProducerConfig(props)) + producer = new Producer(new ProducerConfig(props)) consumer = new SimpleConsumer(host, port, 1000000, 64*1024) - super.setUp } override def tearDown() { - super.tearDown producer.close() consumer.close() + super.tearDown } } diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala index 5284e370efd..d9995d0b064 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -24,10 +24,10 @@ import kafka.utils.{Utils, Logging} import kafka.utils.TestUtils import org.scalatest.junit.JUnit3Suite import scala.collection.JavaConversions._ -import kafka.javaapi.message.ByteBufferMessageSet import kafka.consumer.{ConsumerConfig, KafkaMessageStream} import org.apache.log4j.{Level, Logger} import kafka.message._ +import kafka.javaapi.producer.{ProducerData, Producer} class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { @@ -67,16 +67,17 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compressed: CompressionCodec): List[Message]= { var messages: List[Message] = Nil - val producer = new kafka.javaapi.producer.SyncProducer(TestUtils.createProducer("localhost", conf.port)) + val producer: kafka.producer.Producer[Int, Message] = TestUtils.createProducer(zkConnect) + val javaProducer: Producer[Int, Message] = new kafka.javaapi.producer.Producer(producer) for (partition <- 0 until numParts) { val ms = 0.until(messagesPerNode).map(x => new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray - val mSet = new ByteBufferMessageSet(compressionCodec = compressed, messages = getMessageList(ms: _*)) for (message <- ms) messages ::= message - producer.send(topic, partition, mSet) + import scala.collection.JavaConversions._ + javaProducer.send(new ProducerData[Int, Message](topic, partition, asList(ms))) } - producer.close() + javaProducer.close messages } @@ -106,12 +107,6 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar messages.sortWith((s,t) => s.checksum < t.checksum) } - private def getMessageList(messages: Message*): java.util.List[Message] = { - val messageList = new java.util.ArrayList[Message]() - messages.foreach(m => messageList.add(m)) - messageList - } - private def toJavaMap(scalaMap: Map[String, Int]): java.util.Map[String, java.lang.Integer] = { val javaMap = new java.util.HashMap[String, java.lang.Integer]() scalaMap.foreach(m => javaMap.put(m._1, m._2.asInstanceOf[java.lang.Integer])) diff --git a/core/src/test/scala/unit/kafka/javaapi/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/javaapi/integration/PrimitiveApiTest.scala deleted file mode 100644 index ad261201f84..00000000000 --- a/core/src/test/scala/unit/kafka/javaapi/integration/PrimitiveApiTest.scala +++ /dev/null @@ -1,415 +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.javaapi.integration - -import scala.collection._ -import kafka.api.FetchRequestBuilder -import kafka.common.{InvalidPartitionException, OffsetOutOfRangeException} -import kafka.javaapi.ProducerRequest -import kafka.javaapi.message.ByteBufferMessageSet -import kafka.message.{DefaultCompressionCodec, NoCompressionCodec, Message} -import kafka.server.{KafkaRequestHandler, KafkaConfig} -import kafka.utils.TestUtils -import org.apache.log4j.{Level, Logger} -import org.scalatest.junit.JUnit3Suite - -/** - * End to end tests of the primitive apis against a local server - */ -class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with kafka.integration.KafkaServerTestHarness { - - val port = 9999 - val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) - val configs = List(config) - val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) - - def testProduceAndFetch() { - // send some messages - val topic = "test" - - // send an empty messageset first - val sent2 = new ByteBufferMessageSet(NoCompressionCodec, getMessageList(Seq.empty[Message]: _*)) - producer.send(topic, sent2) - - Thread.sleep(200) - sent2.getBuffer.rewind - val fetched2 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) - val fetchedMessage2 = fetched2.messageSet(topic, 0) - TestUtils.checkEquals(sent2.iterator, fetchedMessage2.iterator) - - - // send some messages - val sent3 = new ByteBufferMessageSet(NoCompressionCodec, - getMessageList( - new Message("hello".getBytes()),new Message("there".getBytes()))) - producer.send(topic, sent3) - - Thread.sleep(200) - sent3.getBuffer.rewind - var messageSet: ByteBufferMessageSet = null - while(messageSet == null || messageSet.validBytes == 0) { - val fetched3 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) - messageSet = fetched3.messageSet(topic, 0).asInstanceOf[ByteBufferMessageSet] - } - TestUtils.checkEquals(sent3.iterator, messageSet.iterator) - - // temporarily set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.FATAL) - - // send an invalid offset - try { - val fetchedWithError = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, -1, 10000).build()) - val messageWithError = fetchedWithError.messageSet(topic, 0) - messageWithError.iterator - fail("Fetch with invalid offset should throw an exception when iterating over response") - } catch { - case e: OffsetOutOfRangeException => "this is good" - } - - // restore set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.ERROR) - } - - def testProduceAndFetchWithCompression() { - // send some messages - val topic = "test" - - // send an empty messageset first - val sent2 = new ByteBufferMessageSet(DefaultCompressionCodec, getMessageList(Seq.empty[Message]: _*)) - producer.send(topic, sent2) - - Thread.sleep(200) - sent2.getBuffer.rewind - val fetched2 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) - val message2 = fetched2.messageSet(topic, 0) - TestUtils.checkEquals(sent2.iterator, message2.iterator) - - - // send some messages - val sent3 = new ByteBufferMessageSet( DefaultCompressionCodec, - getMessageList( - new Message("hello".getBytes()),new Message("there".getBytes()))) - producer.send(topic, sent3) - - Thread.sleep(200) - sent3.getBuffer.rewind - var fetchedMessage: ByteBufferMessageSet = null - while(fetchedMessage == null || fetchedMessage.validBytes == 0) { - val fetched3 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) - fetchedMessage = fetched3.messageSet(topic, 0).asInstanceOf[ByteBufferMessageSet] - } - TestUtils.checkEquals(sent3.iterator, fetchedMessage.iterator) - - // temporarily set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.FATAL) - - // send an invalid offset - try { - val fetchedWithError = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, -1, 10000).build()) - val messageWithError = fetchedWithError.messageSet(topic, 0) - messageWithError.iterator - fail("Fetch with invalid offset should throw an exception when iterating over response") - } catch { - case e: OffsetOutOfRangeException => "this is good" - } - - // restore set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.ERROR) - } - - def testProduceAndMultiFetch() { - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - { - val messages = new mutable.HashMap[String, ByteBufferMessageSet] - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, - messages = getMessageList(new Message(("a_" + topic).getBytes), - new Message(("b_" + topic).getBytes))) - messages += topic -> set - producer.send(topic, set) - set.getBuffer.rewind - builder.addFetch(topic, partition, 0, 10000) - } - - // wait a bit for produced message to be available - Thread.sleep(200) - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val messageSet = response.messageSet(topic, partition) - TestUtils.checkEquals(messages(topic).iterator, messageSet.iterator) - } - } - - // temporarily set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.FATAL) - - { - // send some invalid offsets - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) - builder.addFetch(topic, partition, -1, 10000) - - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - try { - val iter = response.messageSet(topic, partition).iterator - while (iter.hasNext) - iter.next - fail("MessageSet for invalid offset should throw exception") - } catch { - case e: OffsetOutOfRangeException => "this is good" - } - } - } - - { - // send some invalid partitions - val builder = new FetchRequestBuilder() - for( (topic, _) <- topics) - builder.addFetch(topic, -1, 0, 10000) - - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, _) <- topics) { - try { - val iter = response.messageSet(topic, -1).iterator - while (iter.hasNext) - iter.next - fail("MessageSet for invalid partition should throw exception") - } catch { - case e: InvalidPartitionException => "this is good" - } - } - } - - // restore set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.ERROR) - } - - def testProduceAndMultiFetchWithCompression() { - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - { - val messages = new mutable.HashMap[String, ByteBufferMessageSet] - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, - messages = getMessageList(new Message(("a_" + topic).getBytes), - new Message(("b_" + topic).getBytes))) - messages += topic -> set - producer.send(topic, set) - set.getBuffer.rewind - builder.addFetch(topic, partition, 0, 10000) - } - - // wait a bit for produced message to be available - Thread.sleep(200) - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val iter = response.messageSet(topic, partition).iterator - if (iter.hasNext) { - TestUtils.checkEquals(messages(topic).iterator, iter) - } else { - fail("fewer responses than expected") - } - } - } - - // temporarily set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.FATAL) - - { - // send some invalid offsets - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) - builder.addFetch(topic, partition, -1, 10000) - - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - try { - val iter = response.messageSet(topic, partition).iterator - while (iter.hasNext) - iter.next - fail("Expected exception when fetching invalid offset") - } catch { - case e: OffsetOutOfRangeException => "this is good" - } - } - } - - { - // send some invalid partitions - val builder = new FetchRequestBuilder() - for( (topic, _) <- topics) - builder.addFetch(topic, -1, 0, 10000) - - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, _) <- topics) { - try { - val iter = response.messageSet(topic, -1).iterator - while (iter.hasNext) - iter.next - fail("Expected exception when fetching invalid partition") - } catch { - case e: InvalidPartitionException => "this is good" - } - } - } - - // restore set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.ERROR) - } - - def testProduceAndMultiFetchJava() { - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - { - val messages = new mutable.HashMap[String, ByteBufferMessageSet] - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, - messages = getMessageList(new Message(("a_" + topic).getBytes), - new Message(("b_" + topic).getBytes))) - messages += topic -> set - producer.send(topic, set) - set.getBuffer.rewind - builder.addFetch(topic, partition, 0, 10000) - } - - // wait a bit for produced message to be available - Thread.sleep(200) - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val iter = response.messageSet(topic, partition).iterator - if (iter.hasNext) { - TestUtils.checkEquals(messages(topic).iterator, iter) - } else { - fail("fewer responses than expected") - } - } - } - } - - def testProduceAndMultiFetchJavaWithCompression() { - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - { - val messages = new mutable.HashMap[String, ByteBufferMessageSet] - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, - messages = getMessageList(new Message(("a_" + topic).getBytes), - new Message(("b_" + topic).getBytes))) - messages += topic -> set - producer.send(topic, set) - set.getBuffer.rewind - builder.addFetch(topic, partition, 0, 10000) - } - - // wait a bit for produced message to be available - Thread.sleep(200) - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val iter = response.messageSet(topic, partition).iterator - TestUtils.checkEquals(messages(topic).iterator, iter) - } - } - } - - def testMultiProduce() { - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - val messages = new mutable.HashMap[String, ByteBufferMessageSet] - val builder = new FetchRequestBuilder() - var produceList: List[ProducerRequest] = Nil - for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, - messages = getMessageList(new Message(("a_" + topic).getBytes), - new Message(("b_" + topic).getBytes))) - messages += topic -> set - produceList ::= new ProducerRequest(topic, 0, set) - builder.addFetch(topic, partition, 0, 10000) - } - producer.multiSend(produceList.toArray) - - for (messageSet <- messages.values) - messageSet.getBuffer.rewind - - // wait a bit for produced message to be available - Thread.sleep(200) - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val iter = response.messageSet(topic, partition).iterator - if (iter.hasNext) { - TestUtils.checkEquals(messages(topic).iterator, iter) - } else { - fail("fewer responses than expected") - } - } - } - - def testMultiProduceWithCompression() { - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - val messages = new mutable.HashMap[String, ByteBufferMessageSet] - val builder = new FetchRequestBuilder() - var produceList: List[ProducerRequest] = Nil - for( (topic, partition) <- topics) { - val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, - messages = getMessageList(new Message(("a_" + topic).getBytes), - new Message(("b_" + topic).getBytes))) - messages += topic -> set - produceList ::= new ProducerRequest(topic, 0, set) - builder.addFetch(topic, partition, 0, 10000) - } - producer.multiSend(produceList.toArray) - - for (messageSet <- messages.values) - messageSet.getBuffer.rewind - - // wait a bit for produced message to be available - Thread.sleep(200) - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val iter = response.messageSet(topic, partition).iterator - if (iter.hasNext) { - TestUtils.checkEquals(messages(topic).iterator, iter) - } else { - fail("fewer responses than expected") - } - } - } - - private def getMessageList(messages: Message*): java.util.List[Message] = { - val messageList = new java.util.ArrayList[Message]() - messages.foreach(m => messageList.add(m)) - messageList - } -} diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 16074c88a60..a0b3e2c11da 100644 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -25,6 +25,7 @@ import kafka.common.OffsetOutOfRangeException import kafka.zk.ZooKeeperTestHarness import kafka.utils.{TestZKUtils, Utils, MockTime, TestUtils} import org.scalatest.junit.JUnit3Suite +import kafka.admin.CreateTopicCommand class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -34,6 +35,8 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { var logManager: LogManager = null var config:KafkaConfig = null val zookeeperConnect = TestZKUtils.zookeeperConnect + val name = "kafka" + val veryLargeLogFlushInterval = 10000000L override def setUp() { super.setUp() @@ -41,9 +44,13 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { config = new KafkaConfig(props) { override val logFileSize = 1024 } - logManager = new LogManager(config, null, time, -1, maxLogAge, false) + logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false) logManager.startup logDir = logManager.logDir + + // setup brokers in zookeeper as owners of partitions for this test + CreateTopicCommand.createTopic(zookeeper.client, name, 3, 1, "0,0,0") + } override def tearDown() { @@ -55,7 +62,6 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testCreateLog() { - val name = "kafka" val log = logManager.getOrCreateLog(name, 0) val logFile = new File(config.logDir, name + "-0") assertTrue(logFile.exists) @@ -64,7 +70,6 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testGetLog() { - val name = "kafka" val log = logManager.getLog(name, 0) val logFile = new File(config.logDir, name + "-0") assertTrue(!logFile.exists) @@ -72,7 +77,7 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testCleanupExpiredSegments() { - val log = logManager.getOrCreateLog("cleanup", 0) + val log = logManager.getOrCreateLog(name, 0) var offset = 0L for(i <- 0 until 1000) { var set = TestUtils.singleMessageSet("test".getBytes()) @@ -111,11 +116,11 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Int] // keep exactly 6 segments + 1 roll over override val logRetentionHours = retentionHours } - logManager = new LogManager(config, null, time, -1, retentionMs, false) + logManager = new LogManager(config, time, veryLargeLogFlushInterval, retentionMs, false) logManager.startup // create a log - val log = logManager.getOrCreateLog("cleanup", 0) + val log = logManager.getOrCreateLog(name, 0) var offset = 0L // add a bunch of messages that should be larger than the retentionSize @@ -151,14 +156,14 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { logManager.close Thread.sleep(100) config = new KafkaConfig(props) { - override val logFileSize = 1024 *1024 *1024 + override val logFileSize = 1024 *1024 *1024 override val flushSchedulerThreadRate = 50 override val flushInterval = Int.MaxValue override val flushIntervalMap = Utils.getTopicFlushIntervals("timebasedflush:100") } - logManager = new LogManager(config, null, time, -1, maxLogAge, false) + logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false) logManager.startup - val log = logManager.getOrCreateLog("timebasedflush", 0) + val log = logManager.getOrCreateLog(name, 0) for(i <- 0 until 200) { var set = TestUtils.singleMessageSet("test".getBytes()) log.append(set) @@ -177,12 +182,12 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { override val logFileSize = 256 override val topicPartitionsMap = Utils.getTopicPartitions("testPartition:2") } - - logManager = new LogManager(config, null, time, -1, maxLogAge, false) + + logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false) logManager.startup - - for(i <- 0 until 2) { - val log = logManager.getOrCreateLog("testPartition", i) + + for(i <- 0 until 1) { + val log = logManager.getOrCreateLog(name, i) for(i <- 0 until 250) { var set = TestUtils.singleMessageSet("test".getBytes()) log.append(set) @@ -191,7 +196,7 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { try { - val log = logManager.getOrCreateLog("testPartition", 2) + val log = logManager.getOrCreateLog(name, 2) assertTrue("Should not come here", log != null) } catch { case _ => diff --git a/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala index 6bc92b95ad6..ecde6f46119 100644 --- a/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala @@ -29,6 +29,7 @@ import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} import org.apache.log4j._ import kafka.zk.ZooKeeperTestHarness import org.scalatest.junit.JUnit3Suite +import kafka.admin.CreateTopicCommand import kafka.api.{FetchRequestBuilder, OffsetRequest} object LogOffsetTest { @@ -51,7 +52,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val config: Properties = createBrokerConfig(1, brokerPort) val logDirPath = config.getProperty("log.dir") logDir = new File(logDirPath) - + server = TestUtils.createServer(new KafkaConfig(config)) simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024) } @@ -94,10 +95,12 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testGetOffsetsBeforeLatestTime() { val topicPartition = "kafka-" + 0 - val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition val topic = topicPartition.split("-").head val part = Integer.valueOf(topicPartition.split("-").last).intValue + // setup brokers in zookeeper as owners of partitions for this test + CreateTopicCommand.createTopic(zookeeper.client, topic, 1, 1, "1") + val logManager = server.getLogManager val log = logManager.getOrCreateLog(topic, part) @@ -133,6 +136,9 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val topic = topicPartition.split("-").head val part = Integer.valueOf(topicPartition.split("-").last).intValue + // setup brokers in zookeeper as owners of partitions for this test + CreateTopicCommand.createTopic(zookeeper.client, topic, 1, 1, "1") + var offsetChanged = false for(i <- 1 to 14) { val consumerOffsets = simpleConsumer.getOffsetsBefore(topic, part, @@ -147,11 +153,13 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testGetOffsetsBeforeNow() { - val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(10) - val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition + val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(3) val topic = topicPartition.split("-").head val part = Integer.valueOf(topicPartition.split("-").last).intValue + // setup brokers in zookeeper as owners of partitions for this test + CreateTopicCommand.createTopic(zookeeper.client, topic, 3, 1, "1,1,1") + val logManager = server.getLogManager val log = logManager.getOrCreateLog(topic, part) val message = new Message(Integer.toString(42).getBytes()) @@ -172,11 +180,13 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testGetOffsetsBeforeEarliestTime() { - val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(10) - val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition + val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(3) val topic = topicPartition.split("-").head val part = Integer.valueOf(topicPartition.split("-").last).intValue + // setup brokers in zookeeper as owners of partitions for this test + CreateTopicCommand.createTopic(zookeeper.client, topic, 3, 1, "1,1,1") + val logManager = server.getLogManager val log = logManager.getOrCreateLog(topic, part) val message = new Message(Integer.toString(42).getBytes()) diff --git a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala index 290caf99cdf..1892359d1e4 100644 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala @@ -26,18 +26,17 @@ import kafka.message.Message import kafka.producer.async.MissingConfigException import kafka.serializer.Encoder import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.{TestUtils, TestZKUtils, Utils, Logging} import kafka.zk.{EmbeddedZookeeper, ZooKeeperTestHarness} import org.apache.log4j.spi.LoggingEvent import org.apache.log4j.{PropertyConfigurator, Logger} import org.junit.{After, Before, Test} import org.scalatest.junit.JUnit3Suite +import kafka.utils._ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { var logDirZk: File = null var logDirBl: File = null - // var topicLogDir: File = null var serverBl: KafkaServer = null var serverZk: KafkaServer = null @@ -63,7 +62,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with logDirZk = new File(logDirZkPath) serverZk = TestUtils.createServer(new KafkaConfig(propsZk)); - val propsBl: Properties = createBrokerConfig(brokerBl, portBl) + val propsBl: Properties = TestUtils.createBrokerConfig(brokerBl, portBl) val logDirBlPath = propsBl.getProperty("log.dir") logDirBl = new File(logDirBlPath) serverBl = TestUtils.createServer(new KafkaConfig(propsBl)) @@ -85,8 +84,6 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Utils.rm(logDirBl) Thread.sleep(500) -// zkServer.shutdown -// Thread.sleep(500) super.tearDown() } @@ -132,7 +129,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") props.put("log4j.appender.KAFKA.SerializerClass", "kafka.log4j.AppenderStringEncoder") - props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString) + props.put("log4j.appender.KAFKA.ZkConnect", TestZKUtils.zookeeperConnect) props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") // topic missing @@ -148,7 +145,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString) + props.put("log4j.appender.KAFKA.ZkConnect", TestZKUtils.zookeeperConnect) props.put("log4j.appender.KAFKA.Topic", "test-topic") props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") @@ -160,27 +157,6 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with } } - @Test - def testBrokerListLog4jAppends() { - PropertyConfigurator.configure(getLog4jConfigWithBrokerList) - - for(i <- 1 to 5) - info("test") - - Thread.sleep(2500) - - var offset = 0L - val response = simpleConsumerBl.fetch(new FetchRequestBuilder().addFetch("test-topic", 0, offset, 1024*1024).build()) - val fetchedMessage = response.messageSet("test-topic", 0) - var count = 0 - for(message <- fetchedMessage) { - count = count + 1 - offset += message.offset - } - - assertEquals(5, count) - } - @Test def testZkConnectLog4jAppends() { PropertyConfigurator.configure(getLog4jConfigWithZkConnect) @@ -208,18 +184,6 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with assertEquals(5, count) } - private def getLog4jConfigWithBrokerList: Properties = { - var props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString) - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.logger.kafka.log4j", "INFO,KAFKA") - props - } - private def getLog4jConfigWithZkConnect: Properties = { var props = new Properties() props.put("log4j.rootLogger", "INFO") @@ -232,21 +196,6 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with props } - private def createBrokerConfig(nodeId: Int, port: Int): Properties = { - val props = new Properties - props.put("brokerid", nodeId.toString) - props.put("port", port.toString) - props.put("log.dir", getLogDir.getAbsolutePath) - props.put("log.flush.interval", "1") - props.put("enable.zookeeper", "false") - props.put("num.partitions", "1") - props.put("log.retention.hours", "10") - props.put("log.cleanup.interval.mins", "5") - props.put("log.file.size", "1000") - props.put("zk.connect", zkConnect.toString) - props - } - private def getLogDir(): File = { val dir = TestUtils.tempDir() dir diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index 6303844075f..ed3bfad2630 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -18,23 +18,41 @@ package kafka.producer import org.easymock.EasyMock -import kafka.api.ProducerRequest import org.junit.Test -import org.scalatest.junit.JUnitSuite import kafka.producer.async._ import java.util.concurrent.LinkedBlockingQueue import junit.framework.Assert._ -import collection.SortedSet -import kafka.cluster.{Broker, Partition} -import collection.mutable.{HashMap, ListBuffer} +import kafka.cluster.Broker +import collection.mutable.ListBuffer import collection.Map import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} import kafka.serializer.{StringEncoder, StringDecoder, Encoder} import java.util.{LinkedList, Properties} -import kafka.utils.{TestZKUtils, TestUtils} import kafka.common.{InvalidConfigException, NoBrokersForPartitionException, InvalidPartitionException} +import kafka.api.{PartitionMetadata, TopicMetadata, TopicMetadataRequest, ProducerRequest} +import kafka.utils.{NegativePartitioner, TestZKUtils, TestUtils} +import kafka.zk.ZooKeeperTestHarness +import org.scalatest.junit.JUnit3Suite +import kafka.utils.TestUtils._ +import kafka.server.KafkaConfig +import org.I0Itec.zkclient.ZkClient -class AsyncProducerTest extends JUnitSuite { +class AsyncProducerTest extends JUnit3Suite with ZooKeeperTestHarness { + val props = createBrokerConfigs(1) + val configs = props.map(p => new KafkaConfig(p) { override val flushInterval = 1}) + var zkClient: ZkClient = null + var brokers: Seq[Broker] = null + + override def setUp() { + super.setUp() + zkClient = zookeeper.client + // create brokers in zookeeper + brokers = TestUtils.createBrokersInZk(zkClient, configs.map(config => config.brokerId)) + } + + override def tearDown() { + super.tearDown() + } @Test def testProducerQueueSize() { @@ -50,7 +68,7 @@ class AsyncProducerTest extends JUnitSuite { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("broker.list", "0:localhost:9092") + props.put("zk.connect", TestZKUtils.zookeeperConnect) props.put("producer.type", "async") props.put("queue.size", "10") props.put("batch.size", "1") @@ -72,13 +90,13 @@ class AsyncProducerTest extends JUnitSuite { def testProduceAfterClosed() { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("broker.list", "0:localhost:9092") + props.put("zk.connect", TestZKUtils.zookeeperConnect) props.put("producer.type", "async") props.put("batch.size", "1") val config = new ProducerConfig(props) val produceData = getProduceData(10) - val producer = new Producer[String, String](config) + val producer = new Producer[String, String](config, zkClient) producer.close try { @@ -157,18 +175,35 @@ class AsyncProducerTest extends JUnitSuite { producerDataList.append(new ProducerData[Int,Message]("topic2", 4, new Message("msg5".getBytes))) val props = new Properties() - props.put("broker.list", "0:localhost:9092,1:localhost:9092") + props.put("zk.connect", zkConnect) + val broker1 = new Broker(0, "localhost", "localhost", 9092) + val broker2 = new Broker(1, "localhost", "localhost", 9093) + // form expected partitions metadata + val partition1Metadata = new PartitionMetadata(0, Some(broker1), List(broker1, broker2)) + val partition2Metadata = new PartitionMetadata(1, Some(broker2), List(broker1, broker2)) + val topic1Metadata = new TopicMetadata("topic1", List(partition1Metadata, partition2Metadata)) + val topic2Metadata = new TopicMetadata("topic2", List(partition1Metadata, partition2Metadata)) val intPartitioner = new Partitioner[Int] { def partition(key: Int, numPartitions: Int): Int = key % numPartitions } val config = new ProducerConfig(props) + + val syncProducer = getSyncProducer(List("topic1", "topic2"), List(topic1Metadata, topic2Metadata)) + + val producerPool = EasyMock.createMock(classOf[ProducerPool]) + producerPool.getZkClient + EasyMock.expectLastCall().andReturn(zkClient) + producerPool.addProducers(config) + EasyMock.expectLastCall() + producerPool.getAnyProducer + EasyMock.expectLastCall().andReturn(syncProducer).times(2) + EasyMock.replay(producerPool) val handler = new DefaultEventHandler[Int,String](config, partitioner = intPartitioner, encoder = null.asInstanceOf[Encoder[String]], - producerPool = null, - populateProducerPool = false, - brokerPartitionInfo = null) + producerPool) + val topic1Broker1Data = new ListBuffer[ProducerData[Int,Message]] topic1Broker1Data.appendAll(List(new ProducerData[Int,Message]("topic1", 0, new Message("msg1".getBytes)), @@ -181,29 +216,34 @@ class AsyncProducerTest extends JUnitSuite { topic2Broker2Data.appendAll(List(new ProducerData[Int,Message]("topic2", 1, new Message("msg2".getBytes)))) val expectedResult = Map( 0 -> Map( - ("topic1", -1) -> topic1Broker1Data, - ("topic2", -1) -> topic2Broker1Data), + ("topic1", 0) -> topic1Broker1Data, + ("topic2", 0) -> topic2Broker1Data), 1 -> Map( - ("topic1", -1) -> topic1Broker2Data, - ("topic2", -1) -> topic2Broker2Data) + ("topic1", 1) -> topic1Broker2Data, + ("topic2", 1) -> topic2Broker2Data) ) val actualResult = handler.partitionAndCollate(producerDataList) assertEquals(expectedResult, actualResult) + EasyMock.verify(syncProducer) + EasyMock.verify(producerPool) } @Test def testSerializeEvents() { val produceData = TestUtils.getMsgStrings(5).map(m => new ProducerData[String,String]("topic1",m)) val props = new Properties() - props.put("broker.list", "0:localhost:9092,1:localhost:9092") + props.put("zk.connect", zkConnect) val config = new ProducerConfig(props) + // form expected partitions metadata + val topic1Metadata = getTopicMetadata("topic1", 0, "localhost", 9092) + + val syncProducer = getSyncProducer(List("topic1"), List(topic1Metadata)) + val producerPool = getMockProducerPool(config, syncProducer) val handler = new DefaultEventHandler[String,String](config, partitioner = null.asInstanceOf[Partitioner[String]], encoder = new StringEncoder, - producerPool = null, - populateProducerPool = false, - brokerPartitionInfo = null) + producerPool) val serializedData = handler.serialize(produceData) val decoder = new StringDecoder @@ -216,14 +256,20 @@ class AsyncProducerTest extends JUnitSuite { val producerDataList = new ListBuffer[ProducerData[String,Message]] producerDataList.append(new ProducerData[String,Message]("topic1", "key1", new Message("msg1".getBytes))) val props = new Properties() - props.put("broker.list", "0:localhost:9092,1:localhost:9092") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new ProducerConfig(props) + + // form expected partitions metadata + val topic1Metadata = getTopicMetadata("topic1", 0, "localhost", 9092) + + val syncProducer = getSyncProducer(List("topic1"), List(topic1Metadata)) + + val producerPool = getMockProducerPool(config, syncProducer) + val handler = new DefaultEventHandler[String,String](config, partitioner = new NegativePartitioner, encoder = null.asInstanceOf[Encoder[String]], - producerPool = null, - populateProducerPool = false, - brokerPartitionInfo = null) + producerPool) try { handler.partitionAndCollate(producerDataList) fail("Should fail with InvalidPartitionException") @@ -231,34 +277,29 @@ class AsyncProducerTest extends JUnitSuite { catch { case e: InvalidPartitionException => // expected, do nothing } - } - - private def getMockBrokerPartitionInfo(): BrokerPartitionInfo ={ - new BrokerPartitionInfo { - def getBrokerPartitionInfo(topic: String = null): SortedSet[Partition] = SortedSet.empty[Partition] - - def getBrokerInfo(brokerId: Int): Option[Broker] = None - - def getAllBrokerInfo: Map[Int, Broker] = new HashMap[Int, Broker] - - def updateInfo = {} - - def close = {} - } + EasyMock.verify(syncProducer) + EasyMock.verify(producerPool) } @Test def testNoBroker() { + val props = new Properties() + props.put("zk.connect", zkConnect) + + val config = new ProducerConfig(props) + // create topic metadata with 0 partitions + val topic1Metadata = new TopicMetadata("topic1", Seq.empty) + + val syncProducer = getSyncProducer(List("topic1"), List(topic1Metadata)) + + val producerPool = getMockProducerPool(config, syncProducer) + val producerDataList = new ListBuffer[ProducerData[String,String]] producerDataList.append(new ProducerData[String,String]("topic1", "msg1")) - val props = new Properties() - val config = new ProducerConfig(props) val handler = new DefaultEventHandler[String,String](config, partitioner = null.asInstanceOf[Partitioner[String]], encoder = new StringEncoder, - producerPool = null, - populateProducerPool = false, - brokerPartitionInfo = getMockBrokerPartitionInfo) + producerPool) try { handler.handle(producerDataList) fail("Should fail with NoBrokersForPartitionException") @@ -266,12 +307,14 @@ class AsyncProducerTest extends JUnitSuite { catch { case e: NoBrokersForPartitionException => // expected, do nothing } + EasyMock.verify(syncProducer) + EasyMock.verify(producerPool) } @Test def testIncompatibleEncoder() { val props = new Properties() - props.put("broker.list", "0:localhost:9092,1:localhost:9092") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new ProducerConfig(props) val producer=new Producer[String, String](config) @@ -286,14 +329,28 @@ class AsyncProducerTest extends JUnitSuite { @Test def testRandomPartitioner() { val props = new Properties() - props.put("broker.list", "0:localhost:9092,1:localhost:9092") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new ProducerConfig(props) + + // create topic metadata with 0 partitions + val topic1Metadata = getTopicMetadata("topic1", 0, "localhost", 9092) + val topic2Metadata = getTopicMetadata("topic2", 0, "localhost", 9092) + + val syncProducer = getSyncProducer(List("topic1", "topic2"), List(topic1Metadata, topic2Metadata)) + + val producerPool = EasyMock.createMock(classOf[ProducerPool]) + producerPool.getZkClient + EasyMock.expectLastCall().andReturn(zkClient) + producerPool.addProducers(config) + EasyMock.expectLastCall() + producerPool.getAnyProducer + EasyMock.expectLastCall().andReturn(syncProducer).times(2) + EasyMock.replay(producerPool) + val handler = new DefaultEventHandler[String,String](config, partitioner = null.asInstanceOf[Partitioner[String]], encoder = null.asInstanceOf[Encoder[String]], - producerPool = null, - populateProducerPool = false, - brokerPartitionInfo = null) + producerPool) val producerDataList = new ListBuffer[ProducerData[String,Message]] producerDataList.append(new ProducerData[String,Message]("topic1", new Message("msg1".getBytes))) producerDataList.append(new ProducerData[String,Message]("topic2", new Message("msg2".getBytes))) @@ -302,41 +359,51 @@ class AsyncProducerTest extends JUnitSuite { val partitionedData = handler.partitionAndCollate(producerDataList) for ((brokerId, dataPerBroker) <- partitionedData) { for ( ((topic, partitionId), dataPerTopic) <- dataPerBroker) - assertTrue(partitionId == ProducerRequest.RandomPartition) + assertTrue(partitionId == 0) } + EasyMock.verify(producerPool) } @Test def testBrokerListAndAsync() { - val topic = "topic1" - val msgs = TestUtils.getMsgStrings(10) - val mockSyncProducer = EasyMock.createMock(classOf[SyncProducer]) - mockSyncProducer.multiSend(EasyMock.aryEq(Array(new ProducerRequest(topic, ProducerRequest.RandomPartition, - messagesToSet(msgs.take(5)))))) - EasyMock.expectLastCall - mockSyncProducer.multiSend(EasyMock.aryEq(Array(new ProducerRequest(topic, ProducerRequest.RandomPartition, - messagesToSet(msgs.takeRight(5)))))) - EasyMock.expectLastCall - mockSyncProducer.close - EasyMock.expectLastCall - EasyMock.replay(mockSyncProducer) - val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") props.put("producer.type", "async") props.put("batch.size", "5") - props.put("broker.list", "0:localhost:9092") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new ProducerConfig(props) - val producerPool = new ProducerPool(config) - producerPool.addProducer(0, mockSyncProducer) + + val topic = "topic1" + val topic1Metadata = getTopicMetadata(topic, 0, "localhost", 9092) + + val msgs = TestUtils.getMsgStrings(10) + val mockSyncProducer = EasyMock.createMock(classOf[SyncProducer]) + mockSyncProducer.send(new TopicMetadataRequest(List(topic))) + EasyMock.expectLastCall().andReturn(List(topic1Metadata)) + mockSyncProducer.multiSend(EasyMock.aryEq(Array(new ProducerRequest(topic, 0, messagesToSet(msgs.take(5)))))) + EasyMock.expectLastCall + mockSyncProducer.multiSend(EasyMock.aryEq(Array(new ProducerRequest(topic, 0, messagesToSet(msgs.takeRight(5)))))) + EasyMock.expectLastCall + EasyMock.replay(mockSyncProducer) + + val producerPool = EasyMock.createMock(classOf[ProducerPool]) + producerPool.getZkClient + EasyMock.expectLastCall().andReturn(zkClient) + producerPool.addProducers(config) + EasyMock.expectLastCall() + producerPool.getAnyProducer + EasyMock.expectLastCall().andReturn(mockSyncProducer) + producerPool.getProducer(0) + EasyMock.expectLastCall().andReturn(mockSyncProducer).times(2) + producerPool.close() + EasyMock.expectLastCall() + EasyMock.replay(producerPool) val handler = new DefaultEventHandler[String,String](config, partitioner = null.asInstanceOf[Partitioner[String]], encoder = new StringEncoder, - producerPool = producerPool, - populateProducerPool = false, - brokerPartitionInfo = null) + producerPool = producerPool) val producer = new Producer[String, String](config, handler) try { @@ -349,6 +416,7 @@ class AsyncProducerTest extends JUnitSuite { } EasyMock.verify(mockSyncProducer) + EasyMock.verify(producerPool) } @Test @@ -380,7 +448,7 @@ class AsyncProducerTest extends JUnitSuite { def testInvalidConfiguration() { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("broker.list", "0:localhost:9092") + props.put("broker.list", TestZKUtils.zookeeperConnect) props.put("zk.connect", TestZKUtils.zookeeperConnect) props.put("producer.type", "async") @@ -398,6 +466,34 @@ class AsyncProducerTest extends JUnitSuite { new ByteBufferMessageSet(NoCompressionCodec, messages.map(m => encoder.toMessage(m)): _*) } + private def getSyncProducer(topic: Seq[String], topicMetadata: Seq[TopicMetadata]): SyncProducer = { + val syncProducer = EasyMock.createMock(classOf[SyncProducer]) + topic.zip(topicMetadata).foreach { topicAndMetadata => + syncProducer.send(new TopicMetadataRequest(List(topicAndMetadata._1))) + EasyMock.expectLastCall().andReturn(List(topicAndMetadata._2)) + } + EasyMock.replay(syncProducer) + syncProducer + } + + private def getMockProducerPool(config: ProducerConfig, syncProducer: SyncProducer): ProducerPool = { + val producerPool = EasyMock.createMock(classOf[ProducerPool]) + producerPool.getZkClient + EasyMock.expectLastCall().andReturn(zkClient) + producerPool.addProducers(config) + EasyMock.expectLastCall() + producerPool.getAnyProducer + EasyMock.expectLastCall().andReturn(syncProducer) + EasyMock.replay(producerPool) + producerPool + } + + private def getTopicMetadata(topic: String, brokerId: Int, brokerHost: String, brokerPort: Int): TopicMetadata = { + val broker1 = new Broker(brokerId, brokerHost, brokerHost, brokerPort) + val partition1Metadata = new PartitionMetadata(brokerId, Some(broker1), List(broker1)) + new TopicMetadata(topic, List(partition1Metadata)) + } + class MockProducer(override val config: SyncProducerConfig) extends SyncProducer(config) { override def send(topic: String, messages: ByteBufferMessageSet): Unit = { Thread.sleep(1000) diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 86b55c36d45..1887b480857 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -17,20 +17,21 @@ package kafka.producer -import junit.framework.Assert._ -import java.util.Properties -import kafka.api.FetchRequestBuilder +import org.scalatest.junit.JUnit3Suite +import kafka.zk.ZooKeeperTestHarness import kafka.consumer.SimpleConsumer +import org.I0Itec.zkclient.ZkClient +import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} +import java.util.Properties +import org.apache.log4j.{Level, Logger} +import org.junit.Test +import kafka.utils.{TestZKUtils, Utils, TestUtils} import kafka.message.Message -import kafka.serializer.Encoder -import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig} -import kafka.utils.{TestUtils, TestZKUtils, Utils} -import kafka.zk.EmbeddedZookeeper -import org.apache.log4j.{Logger, Level} -import org.junit.{After, Before, Test} -import org.scalatest.junit.JUnitSuite +import kafka.admin.CreateTopicCommand +import kafka.api.FetchRequestBuilder +import org.junit.Assert._ -class ProducerTest extends JUnitSuite { +class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness { private val topic = "test-topic" private val brokerId1 = 0 private val brokerId2 = 1 @@ -40,13 +41,13 @@ class ProducerTest extends JUnitSuite { private var server2: KafkaServer = null private var consumer1: SimpleConsumer = null private var consumer2: SimpleConsumer = null - private var zkServer:EmbeddedZookeeper = null private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) + private var zkClient: ZkClient = null - @Before - def setUp() { + override def setUp() { + super.setUp() // set up 2 brokers with 4 partitions each - zkServer = new EmbeddedZookeeper(TestZKUtils.zookeeperConnect) + zkClient = zookeeper.client val props1 = TestUtils.createBrokerConfig(brokerId1, port1) val config1 = new KafkaConfig(props1) { @@ -73,8 +74,7 @@ class ProducerTest extends JUnitSuite { Thread.sleep(500) } - @After - def tearDown() { + override def tearDown() { // restore set request handler logger to a higher level requestHandlerLogger.setLevel(Level.ERROR) server1.shutdown @@ -82,38 +82,43 @@ class ProducerTest extends JUnitSuite { Utils.rm(server1.config.logDir) Utils.rm(server2.config.logDir) Thread.sleep(500) - zkServer.shutdown - Thread.sleep(500) + super.tearDown() } @Test def testZKSendToNewTopic() { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("partitioner.class", "kafka.producer.StaticPartitioner") + props.put("partitioner.class", "kafka.utils.StaticPartitioner") props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new ProducerConfig(props) val producer = new Producer[String, String](config) try { - // Available broker id, partition id at this stage should be (0,0), (1,0) - // this should send the message to broker 0 on partition 0 + // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0, but + // since partition 0 can exist on any of the two brokers, we need to fetch from both brokers producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1"))) Thread.sleep(100) - // Available broker id, partition id at this stage should be (0,0), (0,1), (0,2), (0,3), (1,0) - // Since 4 % 5 = 4, this should send the message to broker 1 on partition 0 producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1"))) - Thread.sleep(100) - // cross check if brokers got the messages + Thread.sleep(1000) + // cross check if one of the brokers got the messages val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) - val messageSet1 = response1.messageSet("new-topic", 0) - assertTrue("Message set should have 1 message", messageSet1.iterator.hasNext) - assertEquals(new Message("test1".getBytes), messageSet1.head.message) + val messageSet1 = response1.messageSet("new-topic", 0).iterator val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) - val messageSet2 = response2.messageSet("new-topic", 0) - assertTrue("Message set should have 1 message", messageSet2.iterator.hasNext) - assertEquals(new Message("test1".getBytes), messageSet2.head.message) + val messageSet2 = response2.messageSet("new-topic", 0).iterator + assertTrue("Message set should have 1 message", (messageSet1.hasNext || messageSet2.hasNext)) + + if(messageSet1.hasNext) { + assertEquals(new Message("test1".getBytes), messageSet1.next.message) + assertTrue("Message set should have 1 message", messageSet1.hasNext) + assertEquals(new Message("test1".getBytes), messageSet1.next.message) + } + else { + assertEquals(new Message("test1".getBytes), messageSet2.next.message) + assertTrue("Message set should have 1 message", messageSet2.hasNext) + assertEquals(new Message("test1".getBytes), messageSet2.next.message) + } } catch { case e: Exception => fail("Not expected", e) } @@ -124,34 +129,40 @@ class ProducerTest extends JUnitSuite { def testZKSendWithDeadBroker() { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("partitioner.class", "kafka.producer.StaticPartitioner") + props.put("partitioner.class", "kafka.utils.StaticPartitioner") props.put("zk.connect", TestZKUtils.zookeeperConnect) + // create topic + CreateTopicCommand.createTopic(zkClient, "new-topic", 4, 2, "0:1,0:1,0:1,0:1") + val config = new ProducerConfig(props) val producer = new Producer[String, String](config) try { - // Available broker id, partition id at this stage should be (0,0), (1,0) - // Hence, this should send the message to broker 0 on partition 0 + // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0 and + // all partitions have broker 0 as the leader. producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1"))) Thread.sleep(100) // kill 2nd broker - server2.shutdown + server1.shutdown Thread.sleep(100) - // Available broker id, partition id at this stage should be (0,0), (0,1), (0,2), (0,3), (1,0) - // Since 4 % 5 = 4, in a normal case, it would send to broker 1 on partition 0. But since broker 1 is down, - // 4 % 4 = 0, So it should send the message to broker 0 on partition 0 + + // Since all partitions are unavailable, this request will be dropped producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1"))) Thread.sleep(100) + + // restart server 1 + server1.startup() + Thread.sleep(100) + // cross check if brokers got the messages val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) - val messageSet1Iter = response1.messageSet("new-topic", 0).iterator - assertTrue("Message set should have 1 message", messageSet1Iter.hasNext) - assertEquals(new Message("test1".getBytes), messageSet1Iter.next.message) - assertTrue("Message set should have another message", messageSet1Iter.hasNext) - assertEquals(new Message("test1".getBytes), messageSet1Iter.next.message) + val messageSet1 = response1.messageSet("new-topic", 0).iterator + assertTrue("Message set should have 1 message", messageSet1.hasNext) + assertEquals(new Message("test1".getBytes), messageSet1.next.message) + assertFalse("Message set should have another message", messageSet1.hasNext) } catch { - case e: Exception => fail("Not expected") + case e: Exception => fail("Not expected", e) } producer.close } @@ -160,7 +171,7 @@ class ProducerTest extends JUnitSuite { def testZKSendToExistingTopicWithNoBrokers() { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("partitioner.class", "kafka.producer.StaticPartitioner") + props.put("partitioner.class", "kafka.utils.StaticPartitioner") props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new ProducerConfig(props) @@ -168,19 +179,19 @@ class ProducerTest extends JUnitSuite { val producer = new Producer[String, String](config) var server: KafkaServer = null + // create topic + CreateTopicCommand.createTopic(zkClient, "new-topic", 4, 2, "0:1,0:1,0:1,0:1") + try { - // shutdown server1 - server1.shutdown - Thread.sleep(100) - // Available broker id, partition id at this stage should be (1,0) - // this should send the message to broker 1 on partition 0 + // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0 and + // all partitions have broker 0 as the leader. producer.send(new ProducerData[String, String]("new-topic", "test", Array("test"))) Thread.sleep(100) // cross check if brokers got the messages - val response1 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) - val messageSet1 = response1.messageSet("new-topic", 0) - assertTrue("Message set should have 1 message", messageSet1.iterator.hasNext) - assertEquals(new Message("test".getBytes), messageSet1.head.message) + val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) + val messageSet1 = response1.messageSet("new-topic", 0).iterator + assertTrue("Message set should have 1 message", messageSet1.hasNext) + assertEquals(new Message("test".getBytes), messageSet1.next.message) // shutdown server2 server2.shutdown @@ -189,7 +200,7 @@ class ProducerTest extends JUnitSuite { Utils.rm(server2.config.logDir) Thread.sleep(100) // start it up again. So broker 2 exists under /broker/ids, but nothing exists under /broker/topics/new-topic - val props2 = TestUtils.createBrokerConfig(brokerId1, port1) + val props2 = TestUtils.createBrokerConfig(brokerId2, port2) val config2 = new KafkaConfig(props2) { override val numPartitions = 4 } @@ -202,9 +213,9 @@ class ProducerTest extends JUnitSuite { // cross check if brokers got the messages val response2 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) - val messageSet2 = response2.messageSet("new-topic", 0) - assertTrue("Message set should have 1 message", messageSet2.iterator.hasNext) - assertEquals(new Message("test".getBytes), messageSet2.head.message) + val messageSet2 = response1.messageSet("new-topic", 0).iterator + assertTrue("Message set should have 1 message", messageSet2.hasNext) + assertEquals(new Message("test".getBytes), messageSet2.next.message) } catch { case e: Exception => fail("Not expected", e) @@ -213,29 +224,5 @@ class ProducerTest extends JUnitSuite { producer.close } } - } -class StringSerializer extends Encoder[String] { - def toEvent(message: Message):String = message.toString - def toMessage(event: String):Message = new Message(event.getBytes) - def getTopic(event: String): String = event.concat("-topic") -} - -class NegativePartitioner extends Partitioner[String] { - def partition(data: String, numPartitions: Int): Int = { - -1 - } -} - -class StaticPartitioner extends Partitioner[String] { - def partition(data: String, numPartitions: Int): Int = { - (data.length % numPartitions) - } -} - -class HashPartitioner extends Partitioner[String] { - def partition(data: String, numPartitions: Int): Int = { - (data.hashCode % numPartitions) - } -} diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index b2b7eec4835..fc628e941f9 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -21,11 +21,11 @@ import junit.framework.Assert import kafka.server.KafkaConfig import kafka.common.MessageSizeTooLargeException import java.util.Properties -import kafka.api.ProducerRequest import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.integration.KafkaServerTestHarness import kafka.utils.{TestZKUtils, SystemTime, TestUtils} import org.scalatest.junit.JUnit3Suite +import kafka.api.ProducerRequest class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { private var messageBytes = new Array[Byte](2); diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 93a30614abd..4db60b79a77 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -17,16 +17,17 @@ package kafka.server import java.io.File -import kafka.producer.{SyncProducer, SyncProducerConfig} import kafka.consumer.SimpleConsumer import java.util.Properties import org.junit.Test import junit.framework.Assert._ -import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} +import kafka.message.{Message, ByteBufferMessageSet} import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import kafka.utils.{TestUtils, Utils} -import kafka.api.{FetchResponse, FetchRequestBuilder, FetchRequest} +import kafka.producer._ +import kafka.admin.CreateTopicCommand +import kafka.api.FetchRequestBuilder class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val port = TestUtils.choosePort @@ -38,26 +39,20 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val host = "localhost" val topic = "test" - val sent1 = new ByteBufferMessageSet(NoCompressionCodec, new Message("hello".getBytes()), new Message("there".getBytes())) - val sent2 = new ByteBufferMessageSet(NoCompressionCodec, new Message("more".getBytes()), new Message("messages".getBytes())) + val sent1 = List(new Message("hello".getBytes()), new Message("there".getBytes())) + val sent2 = List( new Message("more".getBytes()), new Message("messages".getBytes())) { - val producer = new SyncProducer(getProducerConfig(host, - port, - 64*1024, - 100000, - 10000)) - val consumer = new SimpleConsumer(host, - port, - 1000000, - 64*1024) - val server = new KafkaServer(config) server.startup() + // create topic + CreateTopicCommand.createTopic(zookeeper.client, topic, 1, 1, "0") + + val producer = new Producer[Int, Message](getProducerConfig(64*1024, 100000, 10000)) + // send some messages - producer.send(topic, sent1) - sent1.getBuffer.rewind + producer.send(new ProducerData[Int, Message](topic, 0, sent1)) Thread.sleep(200) // do a clean shutdown @@ -68,11 +63,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { { - val producer = new SyncProducer(getProducerConfig(host, - port, - 64*1024, - 100000, - 10000)) + val producer = new Producer[Int, Message](getProducerConfig(64*1024, 100000, 10000)) val consumer = new SimpleConsumer(host, port, 1000000, @@ -81,18 +72,19 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val server = new KafkaServer(config) server.startup() - // bring the server back again and read the messages + Thread.sleep(100) + var fetchedMessage: ByteBufferMessageSet = null while(fetchedMessage == null || fetchedMessage.validBytes == 0) { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) fetchedMessage = fetched.messageSet(topic, 0) } - TestUtils.checkEquals(sent1.iterator, fetchedMessage.iterator) + TestUtils.checkEquals(sent1.iterator, fetchedMessage.map(m => m.message).iterator) val newOffset = fetchedMessage.validBytes // send some more messages - producer.send(topic, sent2) - sent2.getBuffer.rewind + println("Sending messages to topic " + topic) + producer.send(new ProducerData[Int, Message](topic, 0, sent2)) Thread.sleep(200) @@ -101,7 +93,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, newOffset, 10000).build()) fetchedMessage = fetched.messageSet(topic, 0) } - TestUtils.checkEquals(sent2.map(m => m.message).iterator, fetchedMessage.map(m => m.message).iterator) + TestUtils.checkEquals(sent2.iterator, fetchedMessage.map(m => m.message).iterator) server.shutdown() Utils.rm(server.config.logDir) @@ -109,14 +101,14 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { } - private def getProducerConfig(host: String, port: Int, bufferSize: Int, connectTimeout: Int, - reconnectInterval: Int): SyncProducerConfig = { + private def getProducerConfig(bufferSize: Int, connectTimeout: Int, + reconnectInterval: Int): ProducerConfig = { val props = new Properties() - props.put("host", host) - props.put("port", port.toString) + props.put("zk.connect", zkConnect) + props.put("partitioner.class", "kafka.utils.FixedValuePartitioner") props.put("buffer.size", bufferSize.toString) props.put("connect.timeout.ms", connectTimeout.toString) props.put("reconnect.interval", reconnectInterval.toString) - new SyncProducerConfig(props) + new ProducerConfig(props) } } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index ba87c798cd6..51f85402d8c 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -32,6 +32,7 @@ import kafka.cluster.Broker import collection.mutable.ListBuffer import kafka.consumer.{KafkaMessageStream, ConsumerConfig} import scala.collection.Map +import kafka.serializer.Encoder /** * Utility functions to help with testing @@ -134,6 +135,7 @@ object TestUtils { props.put("zk.sessiontimeout.ms", "400") props.put("zk.synctime.ms", "200") props.put("autocommit.interval.ms", "1000") + props.put("rebalance.retries.max", "4") props } @@ -275,14 +277,13 @@ object TestUtils { /** * Create a producer for the given host and port */ - def createProducer(host: String, port: Int): SyncProducer = { + def createProducer[K, V](zkConnect: String): Producer[K, V] = { val props = new Properties() - props.put("host", host) - props.put("port", port.toString) + props.put("zk.connect", zkConnect) props.put("buffer.size", "65536") props.put("connect.timeout.ms", "100000") props.put("reconnect.interval", "10000") - return new SyncProducer(new SyncProducerConfig(props)) + new Producer[K, V](new ProducerConfig(props)) } def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = { @@ -308,6 +309,12 @@ object TestUtils { brokers } + def deleteBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { + val brokers = ids.map(id => new Broker(id, "localhost" + System.currentTimeMillis(), "localhost", 6667)) + brokers.foreach(b => ZkUtils.deletePath(zkClient, ZkUtils.BrokerIdsPath + "/" + b)) + brokers + } + def getConsumedMessages[T](nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[T]]]): List[T]= { var messages: List[T] = Nil for ((topic, messageStreams) <- topicMessageStreams) { @@ -335,3 +342,31 @@ object TestUtils { object TestZKUtils { val zookeeperConnect = "127.0.0.1:2182" } + +class StringSerializer extends Encoder[String] { + def toEvent(message: Message):String = message.toString + def toMessage(event: String):Message = new Message(event.getBytes) + def getTopic(event: String): String = event.concat("-topic") +} + +class NegativePartitioner extends Partitioner[String] { + def partition(data: String, numPartitions: Int): Int = { + -1 + } +} + +class StaticPartitioner extends Partitioner[String] { + def partition(data: String, numPartitions: Int): Int = { + (data.length % numPartitions) + } +} + +class HashPartitioner extends Partitioner[String] { + def partition(data: String, numPartitions: Int): Int = { + (data.hashCode % numPartitions) + } +} + +class FixedValuePartitioner extends Partitioner[Int] { + def partition(data: Int, numPartitions: Int): Int = data +} diff --git a/core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala b/core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala deleted file mode 100644 index 374182c3c8f..00000000000 --- a/core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala +++ /dev/null @@ -1,126 +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.zk - -import junit.framework.Assert._ -import java.util.Collections -import kafka.consumer.{ConsumerConfig, ZookeeperConsumerConnector} -import java.lang.Thread -import org.scalatest.junit.JUnit3Suite -import kafka.utils.{TestUtils, ZkUtils, ZKGroupTopicDirs} - -class ZKLoadBalanceTest extends JUnit3Suite with ZooKeeperTestHarness { - var dirs : ZKGroupTopicDirs = null - val topic = "topic1" - val group = "group1" - val firstConsumer = "consumer1" - val secondConsumer = "consumer2" - - override def setUp() { - super.setUp() - - dirs = new ZKGroupTopicDirs(group, topic) - } - - def testLoadBalance() { - // create the first partition - ZkUtils.setupPartition(zookeeper.client, 400, "broker1", 1111, "topic1", 1) - // add the first consumer - val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, firstConsumer)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, false) - zkConsumerConnector1.createMessageStreams(Map(topic -> 1)) - - { - // check Partition Owner Registry - val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_1 = List( ("400-0", "group1_consumer1-0") ) - checkSetEqual(actual_1, expected_1) - } - - // add a second consumer - val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, secondConsumer)) - val ZKConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, false) - ZKConsumerConnector2.createMessageStreams(Map(topic -> 1)) - // wait a bit to make sure rebalancing logic is triggered - Thread.sleep(200) - - { - // check Partition Owner Registry - val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_2 = List( ("400-0", "group1_consumer1-0") ) - checkSetEqual(actual_2, expected_2) - } - - { - // add a few more partitions - val brokers = List( - (200, "broker2", 1111, "topic1", 2), - (300, "broker3", 1111, "topic1", 2) ) - - for ((brokerID, host, port, topic, nParts) <- brokers) - ZkUtils.setupPartition(zookeeper.client, brokerID, host, port, topic, nParts) - - - // wait a bit to make sure rebalancing logic is triggered - Thread.sleep(1500) - // check Partition Owner Registry - val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_3 = List( ("200-0", "group1_consumer1-0"), - ("200-1", "group1_consumer1-0"), - ("300-0", "group1_consumer1-0"), - ("300-1", "group1_consumer2-0"), - ("400-0", "group1_consumer2-0") ) - checkSetEqual(actual_3, expected_3) - } - - { - // now delete a partition - ZkUtils.deletePartition(zookeeper.client, 400, "topic1") - - // wait a bit to make sure rebalancing logic is triggered - Thread.sleep(500) - // check Partition Owner Registry - val actual_4 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_4 = List( ("200-0", "group1_consumer1-0"), - ("200-1", "group1_consumer1-0"), - ("300-0", "group1_consumer2-0"), - ("300-1", "group1_consumer2-0") ) - checkSetEqual(actual_4, expected_4) - } - - zkConsumerConnector1.shutdown - ZKConsumerConnector2.shutdown - } - - private def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { - import scala.collection.JavaConversions - val children = zookeeper.client.getChildren(path) - Collections.sort(children) - val childrenAsSeq : Seq[java.lang.String] = JavaConversions.asBuffer(children) - childrenAsSeq.map(partition => - (partition, zookeeper.client.readData(path + "/" + partition).asInstanceOf[String])) - } - - private def checkSetEqual(actual : Seq[Tuple2[String,String]], expected : Seq[Tuple2[String,String]]) { - assertEquals(expected.length, actual.length) - for (i <- 0 until expected.length) { - assertEquals(expected(i)._1, actual(i)._1) - assertEquals(expected(i)._2, actual(i)._2) - } - } -} diff --git a/system_test/broker_failure/bin/run-test.sh b/system_test/broker_failure/bin/run-test.sh index b3284e341fb..96be3e332b2 100755 --- a/system_test/broker_failure/bin/run-test.sh +++ b/system_test/broker_failure/bin/run-test.sh @@ -68,7 +68,7 @@ readonly test_start_time="$(date +%s)" readonly num_msg_per_batch=500 readonly batches_per_iteration=5 -readonly num_iterations=12 +readonly num_iterations=5 readonly zk_source_port=2181 readonly zk_mirror_port=2182