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
This commit is contained in:
Neha Narkhede 2012-03-01 21:15:26 +00:00
parent 6e36047fae
commit 9fd2d4ffdb
45 changed files with 1039 additions and 1763 deletions

View File

@ -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

View File

@ -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)

View File

@ -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*/

View File

@ -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

View File

@ -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)
}

View File

@ -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)
}

View File

@ -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()

View File

@ -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)
}
}

View File

@ -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
}

View File

@ -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))
}
}
}

View File

@ -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
}
}

View File

@ -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

View File

@ -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

View File

@ -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 *

View File

@ -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
}
}
}

View File

@ -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()

View File

@ -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
}
}
}

View File

@ -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
}
}

View File

@ -173,6 +173,7 @@ class KafkaApis(val logManager: LogManager) extends Logging {
}
}
}
info("Sending response for topic metadata request")
Some(new TopicMetadataSend(topicsMetadata))
}
}

View File

@ -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)

View File

@ -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")
}
}

View File

@ -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)
}

View File

@ -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() = {

View File

@ -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) {

View File

@ -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)

View File

@ -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)

View File

@ -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]))
}
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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))
}
}

View File

@ -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

View File

@ -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)
}
}

View File

@ -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
}
}

View File

@ -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]))

View File

@ -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
}
}

View File

@ -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 _ =>

View File

@ -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())

View File

@ -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

View File

@ -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)

View File

@ -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)
}
}

View File

@ -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);

View File

@ -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)
}
}

View File

@ -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
}

View File

@ -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)
}
}
}

View File

@ -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