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) { for (i <- 0 until replicaAssignmentList.size) {
val zkPath = ZkUtils.getTopicPartitionReplicasPath(topic, i.toString) val zkPath = ZkUtils.getTopicPartitionReplicasPath(topic, i.toString)
ZkUtils.updatePersistentPath(zkClient, zkPath, Utils.seqToCSV(replicaAssignmentList(i))) 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)))) 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) { for (i <-0 until partitionMetadata.size) {
val replicas = ZkUtils.readData(zkClient, ZkUtils.getTopicPartitionReplicasPath(topic, partitions(i).toString)) val replicas = ZkUtils.readData(zkClient, ZkUtils.getTopicPartitionReplicasPath(topic, partitions(i).toString))
val inSyncReplicas = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.getTopicPartitionInSyncPath(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) debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader)
partitionMetadata(i) = new PartitionMetadata(partitions(i), 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(replicas).map(id => id.toInt)),
getBrokerInfoFromCache(zkClient, cachedBrokerInfo, Utils.getCSVList(inSyncReplicas).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 */) None /* Return log segment metadata when getOffsetsBefore will be replaced with this API */)
} }
Some(new TopicMetadata(topic, partitionMetadata)) Some(new TopicMetadata(topic, partitionMetadata))
} else } else {
None None
}
} }
metadataList.toList metadataList.toList

View File

@ -18,10 +18,10 @@
package kafka.admin package kafka.admin
import joptsimple.OptionParser import joptsimple.OptionParser
import kafka.utils.{Utils, ZKStringSerializer, ZkUtils}
import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.ZkClient
import kafka.utils.{Logging, Utils, ZKStringSerializer, ZkUtils}
object CreateTopicCommand { object CreateTopicCommand extends Logging {
def main(args: Array[String]): Unit = { def main(args: Array[String]): Unit = {
val parser = new OptionParser val parser = new OptionParser
@ -91,6 +91,7 @@ object CreateTopicCommand {
replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, numPartitions, replicationFactor) replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, numPartitions, replicationFactor)
else else
replicaAssignment = getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet) replicaAssignment = getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet)
debug("Replica assignment list for %s is %s".format(topic, replicaAssignment))
AdminUtils.createReplicaAssignmentPathInZK(topic, replicaAssignment, zkClient) AdminUtils.createReplicaAssignmentPathInZK(topic, replicaAssignment, zkClient)
} }
@ -104,8 +105,8 @@ object CreateTopicCommand {
if (brokerList.size != brokerList.toSet.size) if (brokerList.size != brokerList.toSet.size)
throw new AdministrationException("duplicate brokers in replica assignment: " + brokerList) throw new AdministrationException("duplicate brokers in replica assignment: " + brokerList)
if (!brokerList.toSet.subsetOf(availableBrokerList)) if (!brokerList.toSet.subsetOf(availableBrokerList))
throw new AdministrationException("some specified brokers not available. specified brokers: " + brokerList + throw new AdministrationException("some specified brokers not available. specified brokers: " + brokerList.toString +
"available broker:" + availableBrokerList) "available broker:" + availableBrokerList.toString)
ret(i) = brokerList.toList ret(i) = brokerList.toList
if (ret(i).size != ret(0).size) if (ret(i).size != ret(0).size)
throw new AdministrationException("partition " + i + " has different replication factor: " + brokerList) 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], case class PartitionMetadata(partitionId: Int, leader: Option[Broker], replicas: Seq[Broker], isr: Seq[Broker] = Seq.empty,
logMetadata: Option[LogMetadata]) { logMetadata: Option[LogMetadata] = None) {
def sizeInBytes: Int = { def sizeInBytes: Int = {
var size: Int = 4 /* partition id */ + 1 /* if leader exists*/ var size: Int = 4 /* partition id */ + 1 /* if leader exists*/

View File

@ -72,6 +72,7 @@ object TopicMetadataRequest {
def serializeTopicMetadata(topicMetadata: Seq[TopicMetadata]): ByteBuffer = { def serializeTopicMetadata(topicMetadata: Seq[TopicMetadata]): ByteBuffer = {
val size = topicMetadata.foldLeft(4 /* num topics */)(_ + _.sizeInBytes) val size = topicMetadata.foldLeft(4 /* num topics */)(_ + _.sizeInBytes)
val buffer = ByteBuffer.allocate(size) val buffer = ByteBuffer.allocate(size)
debug("Allocating buffer of size %d for topic metadata response".format(size))
/* number of topics */ /* number of topics */
buffer.putInt(topicMetadata.size) buffer.putInt(topicMetadata.size)
/* topic partition_metadata */ /* topic partition_metadata */
@ -122,13 +123,16 @@ case class TopicMetadataRequest(val topics: Seq[String],
} }
class TopicMetadataSend(topicsMetadata: Seq[TopicMetadata]) extends Send { 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) private val header = ByteBuffer.allocate(6)
val metadata = TopicMetadataRequest.serializeTopicMetadata(topicsMetadata)
header.putInt(size + 2) header.putInt(size + 2)
header.putShort(ErrorMapping.NoError.asInstanceOf[Short]) header.putShort(ErrorMapping.NoError.asInstanceOf[Short])
header.rewind() header.rewind()
val metadata = TopicMetadataRequest.serializeTopicMetadata(topicsMetadata)
metadata.rewind()
trace("Wrote size %d in header".format(size + 2))
var complete: Boolean = false var complete: Boolean = false
def writeTo(channel: GatheringByteChannel): Int = { def writeTo(channel: GatheringByteChannel): Int = {
@ -136,9 +140,13 @@ class TopicMetadataSend(topicsMetadata: Seq[TopicMetadata]) extends Send {
var written = 0 var written = 0
if(header.hasRemaining) if(header.hasRemaining)
written += channel.write(header) written += channel.write(header)
trace("Wrote %d bytes for header".format(written))
if(!header.hasRemaining && metadata.hasRemaining) if(!header.hasRemaining && metadata.hasRemaining)
written += channel.write(metadata) written += channel.write(metadata)
trace("Wrote %d bytes for header and metadata".format(written))
if(!metadata.hasRemaining) if(!metadata.hasRemaining)
complete = true complete = true
written written

View File

@ -17,41 +17,14 @@
package kafka.cluster package kafka.cluster
object Partition { case class Partition(val brokerId: Int, val partId: Int, val topic: String = "") extends Ordered[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)
}
}
class Partition(val brokerId: Int, val partId: Int) extends Ordered[Partition] { def name = partId
def this(name: String) = {
this(1, 1)
}
def name = brokerId + "-" + partId
override def toString(): String = name
def compare(that: Partition) = def compare(that: Partition) =
if (this.brokerId == that.brokerId) if (this.topic == that.topic)
this.partId - that.partId this.partId - that.partId
else else
this.brokerId - that.brokerId this.topic.compareTo(that.topic)
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
} }

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.api._
import kafka.network._ import kafka.network._
import kafka.utils._ import kafka.utils._
import kafka.utils.Utils._
/** /**
* A consumer of kafka messages * A consumer of kafka messages
@ -77,16 +78,16 @@ class SimpleConsumer(val host: String,
getOrMakeConnection() getOrMakeConnection()
var response: Tuple2[Receive,Int] = null var response: Tuple2[Receive,Int] = null
try { try {
sendRequest(request) sendRequest(request, channel)
response = getResponse response = getResponse(channel)
} catch { } catch {
case e : java.io.IOException => case e : java.io.IOException =>
info("Reconnect in fetch request due to socket error: ", e) info("Reconnect in fetch request due to socket error: ", e)
// retry once // retry once
try { try {
channel = connect channel = connect
sendRequest(request) sendRequest(request, channel)
response = getResponse response = getResponse(channel)
} catch { } catch {
case ioe: java.io.IOException => channel = null; throw ioe; case ioe: java.io.IOException => channel = null; throw ioe;
} }
@ -115,16 +116,16 @@ class SimpleConsumer(val host: String,
getOrMakeConnection() getOrMakeConnection()
var response: Tuple2[Receive,Int] = null var response: Tuple2[Receive,Int] = null
try { try {
sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets)) sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets), channel)
response = getResponse response = getResponse(channel)
} catch { } catch {
case e : java.io.IOException => case e : java.io.IOException =>
info("Reconnect in get offetset request due to socket error: ", e) info("Reconnect in get offetset request due to socket error: ", e)
// retry once // retry once
try { try {
channel = connect channel = connect
sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets)) sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets), channel)
response = getResponse response = getResponse(channel)
} catch { } catch {
case ioe: java.io.IOException => channel = null; throw ioe; 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() { private def getOrMakeConnection() {
if(channel == null) { if(channel == null) {
channel = connect() channel = connect()

View File

@ -29,9 +29,9 @@ import org.apache.zookeeper.Watcher.Event.KeeperState
import kafka.api.OffsetRequest import kafka.api.OffsetRequest
import java.util.UUID import java.util.UUID
import kafka.serializer.Decoder import kafka.serializer.Decoder
import kafka.common.{ConsumerRebalanceFailedException, InvalidConfigException}
import java.lang.IllegalStateException import java.lang.IllegalStateException
import kafka.utils.ZkUtils._ import kafka.utils.ZkUtils._
import kafka.common.{NoBrokersForPartitionException, ConsumerRebalanceFailedException, InvalidConfigException}
/** /**
* This class handles the consumers interaction with zookeeper * This class handles the consumers interaction with zookeeper
@ -201,6 +201,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
ret 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) = { private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = {
info("begin registering consumer " + consumerIdString + " in ZK") info("begin registering consumer " + consumerIdString + " in ZK")
createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, topicCount.toJsonString) createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, topicCount.toJsonString)
@ -368,7 +371,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]])
extends IZkChildListener { extends IZkChildListener {
private val dirs = new ZKGroupDirs(group) 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]]() private var oldConsumersPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]()
@throws(classOf[Exception]) @throws(classOf[Exception])
@ -379,18 +382,17 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private def releasePartitionOwnership()= { private def releasePartitionOwnership()= {
info("Releasing partition ownership") info("Releasing partition ownership")
for ((topic, infos) <- topicRegistry) { for ((topic, infos) <- topicRegistry) {
val topicDirs = new ZKGroupTopicDirs(group, topic)
for(partition <- infos.keys) { for(partition <- infos.keys) {
val znode = topicDirs.consumerOwnerDir + "/" + partition val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition.partId.toString)
deletePath(zkClient, znode) deletePath(zkClient, partitionOwnerPath)
debug("Consumer " + consumerIdString + " releasing " + znode) debug("Consumer " + consumerIdString + " releasing " + partitionOwnerPath)
} }
} }
} }
private def getRelevantTopicMap(myTopicThreadIdsMap: Map[String, Set[String]], private def getRelevantTopicMap(myTopicThreadIdsMap: Map[String, Set[String]],
newPartMap: Map[String,List[String]], newPartMap: Map[String, Seq[String]],
oldPartMap: Map[String,List[String]], oldPartMap: Map[String, Seq[String]],
newConsumerMap: Map[String,List[String]], newConsumerMap: Map[String,List[String]],
oldConsumerMap: Map[String,List[String]]): Map[String, Set[String]] = { oldConsumerMap: Map[String,List[String]]): Map[String, Set[String]] = {
var relevantTopicThreadIdsMap = new mutable.HashMap[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 topicDirs = new ZKGroupTopicDirs(group, topic)
val curConsumers = consumersPerTopicMap.get(topic).get 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 nPartsPerConsumer = curPartitions.size / curConsumers.size
val nConsumersWithExtraPart = 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 topic = partitionOwner._1._1
val partition = partitionOwner._1._2 val partition = partitionOwner._1._2
val consumerThreadId = partitionOwner._2 val consumerThreadId = partitionOwner._2
val topicDirs = new ZKGroupTopicDirs(group, topic) val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic,partition)
val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition
try { try {
createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId)
info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic)
@ -618,37 +619,47 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
else true else true
} }
private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partitionString: String, private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partition: String,
topic: String, consumerThreadId: String) { topic: String, consumerThreadId: String) {
val partition = Partition.parse(partitionString)
val partTopicInfoMap = topicRegistry.get(topic) 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) val offsetString = readDataMaybeNull(zkClient, znode)
// If first time starting a consumer, set the initial offset based on the config // If first time starting a consumer, set the initial offset based on the config
var offset : Long = 0L var offset : Long = 0L
if (offsetString == null) if (offsetString == null)
offset = config.autoOffsetReset match { offset = config.autoOffsetReset match {
case OffsetRequest.SmallestTimeString => case OffsetRequest.SmallestTimeString =>
earliestOrLatestOffset(topic, partition.brokerId, partition.partId, OffsetRequest.EarliestTime) earliestOrLatestOffset(topic, leader, partition.toInt, OffsetRequest.EarliestTime)
case OffsetRequest.LargestTimeString => case OffsetRequest.LargestTimeString =>
earliestOrLatestOffset(topic, partition.brokerId, partition.partId, OffsetRequest.LatestTime) earliestOrLatestOffset(topic, leader, partition.toInt, OffsetRequest.LatestTime)
case _ => case _ =>
throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig") throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig")
} }
else else
offset = offsetString.toLong offset = offsetString.toLong
val partitionObject = new Partition(leader, partition.toInt, topic)
val queue = queues.get((topic, consumerThreadId)) val queue = queues.get((topic, consumerThreadId))
val consumedOffset = new AtomicLong(offset) val consumedOffset = new AtomicLong(offset)
val fetchedOffset = new AtomicLong(offset) val fetchedOffset = new AtomicLong(offset)
val partTopicInfo = new PartitionTopicInfo(topic, val partTopicInfo = new PartitionTopicInfo(topic,
partition.brokerId, leader,
partition, partitionObject,
queue, queue,
consumedOffset, consumedOffset,
fetchedOffset, fetchedOffset,
new AtomicInteger(config.fetchSize)) new AtomicInteger(config.fetchSize))
partTopicInfoMap.put(partition, partTopicInfo) partTopicInfoMap.put(partitionObject, partTopicInfo)
debug(partTopicInfo + " selected new offset " + offset) debug(partTopicInfo + " selected new offset " + offset)
} }
} }

View File

@ -19,7 +19,6 @@ package kafka.log
import java.io._ import java.io._
import kafka.utils._ import kafka.utils._
import scala.actors.Actor
import scala.collection._ import scala.collection._
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import kafka.server.{KafkaConfig, KafkaZooKeeper} import kafka.server.{KafkaConfig, KafkaZooKeeper}
@ -32,7 +31,6 @@ import org.I0Itec.zkclient.ZkClient
*/ */
@threadsafe @threadsafe
private[kafka] class LogManager(val config: KafkaConfig, private[kafka] class LogManager(val config: KafkaConfig,
private val scheduler: KafkaScheduler,
private val time: Time, private val time: Time,
val logCleanupIntervalMs: Long, val logCleanupIntervalMs: Long,
val logCleanupDefaultAgeMs: Long, val logCleanupDefaultAgeMs: Long,
@ -47,12 +45,12 @@ private[kafka] class LogManager(val config: KafkaConfig,
private val topicPartitionsMap = config.topicPartitionsMap private val topicPartitionsMap = config.topicPartitionsMap
private val logCreationLock = new Object private val logCreationLock = new Object
private val random = new java.util.Random private val random = new java.util.Random
private var zkActor: Actor = null
private val startupLatch: CountDownLatch = new CountDownLatch(1) private val startupLatch: CountDownLatch = new CountDownLatch(1)
private val logFlusherScheduler = new KafkaScheduler(1, "kafka-logflusher-", false) private val logFlusherScheduler = new KafkaScheduler(1, "kafka-logflusher-", false)
private val logFlushIntervalMap = config.flushIntervalMap private val logFlushIntervalMap = config.flushIntervalMap
private val logRetentionMSMap = getLogRetentionMSMap(config.logRetentionHoursMap) private val logRetentionMSMap = getLogRetentionMSMap(config.logRetentionHoursMap)
private val logRetentionSize = config.logRetentionSize 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 */ /* Initialize a log for each subdirectory of the main log directory */
private val logs = new Pool[String, Pool[Int, Log]]() 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] = { private def getLogRetentionMSMap(logRetentionHourMap: Map[String, Int]) : Map[String, Long] = {
var ret = new mutable.HashMap[String, Long] var ret = new mutable.HashMap[String, Long]
for ( (topic, hour) <- logRetentionHourMap ) for ( (topic, hour) <- logRetentionHourMap )
@ -118,22 +87,29 @@ private[kafka] class LogManager(val config: KafkaConfig,
* Register this broker in ZK for the first time. * Register this broker in ZK for the first time.
*/ */
def startup() { def startup() {
kafkaZookeeper.startup
kafkaZookeeper.registerBrokerInZk() kafkaZookeeper.registerBrokerInZk()
for (topic <- getAllTopics)
kafkaZookeeper.registerTopicInZk(topic) /* Schedule the cleanup task to delete old logs */
startupLatch.countDown 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) info("Starting log flusher every " + config.flushSchedulerThreadRate + " ms with the following overrides " + logFlushIntervalMap)
logFlusherScheduler.scheduleWithRate(flushAllLogs, config.flushSchedulerThreadRate, config.flushSchedulerThreadRate) logFlusherScheduler.scheduleWithRate(flushAllLogs, config.flushSchedulerThreadRate, config.flushSchedulerThreadRate)
startupLatch.countDown
} }
private def awaitStartup() { private def awaitStartup() {
startupLatch.await startupLatch.await
} }
private def registerNewTopicInZK(topic: String) {
zkActor ! topic
}
/** /**
* Create a log for the given topic and the given partition * 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 * Create the log if it does not exist, if it exists just return it
*/ */
def getOrCreateLog(topic: String, partition: Int): Log = { 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 hasNewTopic = false
var parts = getLogPool(topic, partition) var parts = getLogPool(topic, partition)
if (parts == null) { if (parts == null) {
@ -196,6 +176,7 @@ private[kafka] class LogManager(val config: KafkaConfig,
} }
var log = parts.get(partition) var log = parts.get(partition)
if(log == null) { if(log == null) {
// check if this broker hosts this partition
log = createLog(topic, partition) log = createLog(topic, partition)
val found = parts.putIfNotExists(partition, log) val found = parts.putIfNotExists(partition, log)
if(found != null) { if(found != null) {
@ -207,8 +188,6 @@ private[kafka] class LogManager(val config: KafkaConfig,
info("Created log for '" + topic + "'-" + partition) info("Created log for '" + topic + "'-" + partition)
} }
if (hasNewTopic)
registerNewTopicInZK(topic)
log log
} }
@ -279,11 +258,11 @@ private[kafka] class LogManager(val config: KafkaConfig,
*/ */
def close() { def close() {
info("Closing log manager") info("Closing log manager")
scheduler.shutdown()
logFlusherScheduler.shutdown() logFlusherScheduler.shutdown()
val iter = getLogIterator val iter = getLogIterator
while(iter.hasNext) while(iter.hasNext)
iter.next.close() iter.next.close()
zkActor ! StopActor
kafkaZookeeper.close kafkaZookeeper.close
} }

View File

@ -16,44 +16,70 @@
*/ */
package kafka.producer package kafka.producer
import collection.Map
import collection.SortedSet
import kafka.cluster.{Broker, Partition} 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). * Return a sequence of (brokerId, numPartitions).
* @param topic the topic for which this information is to be returned * @param topic the topic for which this information is to be returned
* @return a sequence of (brokerId, numPartitions). Returns a zero-length * @return a sequence of (brokerId, numPartitions). Returns a zero-length
* sequence if no brokers are available. * 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 * It updates the cache by issuing a get topic metadata request to a random broker.
* by the given broker id * @param topic the topic for which the metadata is to be fetched
* @param brokerId the broker for which the info is to be returned
* @return host and port of brokerId
*/ */
def getBrokerInfo(brokerId: Int): Option[Broker] def updateInfo(topic: String = null) = {
val producer = producerPool.getAnyProducer
/** if(topic != null) {
* Generate a mapping from broker id to the host and port for all brokers val topicMetadataRequest = new TopicMetadataRequest(List(topic))
* @return mapping from id to host and port of all brokers val topicMetadataList = producer.send(topicMetadataRequest)
*/ val topicMetadata:Option[TopicMetadata] = if(topicMetadataList.size > 0) Some(topicMetadataList.head) else None
def getAllBrokerInfo: Map[Int, Broker] topicMetadata match {
case Some(metadata) =>
/** info("Fetched metadata for topics %s".format(topic))
* This is relevant to the ZKBrokerPartitionInfo. It updates the ZK cache topicPartitionInfo += (topic -> metadata)
* by reading from zookeeper and recreating the data structures. This API case None =>
* is invoked by the producer, when it detects that the ZK cache of }
* ZKBrokerPartitionInfo is stale. }else {
* // refresh cache for all topics
*/ val topics = topicPartitionInfo.keySet.toList
def updateInfo val topicMetadata = producer.send(new TopicMetadataRequest(topics))
info("Fetched metadata for topics %s".format(topicMetadata.mkString(",")))
/** topicMetadata.foreach(metadata => topicPartitionInfo += (metadata.topic -> metadata))
* Cleanup }
*/ }
def close
} }

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 package kafka.producer
import async.MissingConfigException 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.AppenderSkeleton
import org.apache.log4j.helpers.LogLog import org.apache.log4j.helpers.LogLog
import kafka.utils.Logging import kafka.utils.Logging

View File

@ -22,15 +22,14 @@ import kafka.common.InvalidConfigException
import java.util.concurrent.{TimeUnit, LinkedBlockingQueue} import java.util.concurrent.{TimeUnit, LinkedBlockingQueue}
import kafka.serializer.Encoder import kafka.serializer.Encoder
import java.util.concurrent.atomic.{AtomicLong, AtomicBoolean} import java.util.concurrent.atomic.{AtomicLong, AtomicBoolean}
import org.I0Itec.zkclient.ZkClient
class Producer[K,V](config: ProducerConfig, class Producer[K,V](config: ProducerConfig,
private val eventHandler: EventHandler[K,V]) // for testing only private val eventHandler: EventHandler[K,V]) // for testing only
extends Logging { extends Logging {
private val hasShutdown = new AtomicBoolean(false) private val hasShutdown = new AtomicBoolean(false)
if(!Utils.propertyExists(config.zkConnect) && !Utils.propertyExists(config.brokerList)) if(!Utils.propertyExists(config.zkConnect))
throw new InvalidConfigException("At least one of zk.connect or broker.list must be specified") throw new InvalidConfigException("zk.connect property must be specified in the producer")
if (Utils.propertyExists(config.zkConnect) && Utils.propertyExists(config.brokerList))
throw new InvalidConfigException("Only one of zk.connect and broker.list should be provided")
if (config.batchSize > config.queueSize) if (config.batchSize > config.queueSize)
throw new InvalidConfigException("Batch size can't be larger than queue size.") 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 * This constructor can be used when all config parameters will be specified through the
* ProducerConfig object * ProducerConfig object
* @param config Producer Configuration 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, this(config,
new DefaultEventHandler[K,V](config, new DefaultEventHandler[K,V](config,
Utils.getObject[Partitioner[K]](config.partitionerClass), Utils.getObject[Partitioner[K]](config.partitionerClass),
Utils.getObject[Encoder[V]](config.serializerClass), Utils.getObject[Encoder[V]](config.serializerClass),
new ProducerPool(config), new ProducerPool(config, if(zkClient == null)
populateProducerPool= true, new ZkClient(config.zkConnect, config.zkSessionTimeoutMs,
brokerPartitionInfo= null)) config.zkConnectionTimeoutMs, ZKStringSerializer) else zkClient)))
/** /**
* Sends the data, partitioned by key to the topic using either the * 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- * * to pass in static broker and per-broker partition information. Format- *
* brokerid1:host1:port1, brokerid2:host2:port2*/ * brokerid1:host1:port1, brokerid2:host2:port2*/
val brokerList = Utils.getString(props, "broker.list", null) val brokerList = Utils.getString(props, "broker.list", null)
if(brokerList != null && Utils.getString(props, "partitioner.class", null) != null) if(brokerList != null)
throw new InvalidConfigException("partitioner.class cannot be used when broker.list is set") 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 both broker.list and zk.connect options are specified, throw an exception */
if(brokerList != null && zkConnect != null) if(zkConnect == null)
throw new InvalidConfigException("only one of broker.list and zk.connect can be specified") throw new InvalidConfigException("zk.connect property is required")
/** the partitioner class for partitioning events amongst sub-topics */ /** 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 * /** this parameter specifies whether the messages are sent asynchronously *
* or not. Valid values are - async for asynchronous send * * 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. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package kafka.producer package kafka.producer
import java.util.Properties
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.utils.Logging import java.util.Properties
import java.util.concurrent.ConcurrentHashMap 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 { class ProducerPool(val config: ProducerConfig, val zkClient: ZkClient) extends Logging {
private val syncProducers = new ConcurrentHashMap[Int, SyncProducer] 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() val props = new Properties()
props.put("host", broker.host) props.put("host", broker.host)
props.put("port", broker.port.toString) props.put("port", broker.port.toString)
@ -42,17 +46,48 @@ class ProducerPool(private val config: ProducerConfig) extends Logging {
syncProducers.put(brokerId, syncProducer) syncProducers.put(brokerId, syncProducer)
} }
def getProducer(brokerId: Int) : SyncProducer = { def addProducers(config: ProducerConfig) {
syncProducers.get(brokerId) 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 * Closes all the producers in the pool
*/ */
def close() = { def close() = {
info("Closing all sync producers") lock.synchronized {
val iter = syncProducers.values.iterator info("Closing all sync producers")
while(iter.hasNext) val iter = syncProducers.values.iterator
iter.next.close while(iter.hasNext)
iter.next.close
}
} }
} }

View File

@ -26,6 +26,7 @@ import kafka.api._
import scala.math._ import scala.math._
import kafka.common.MessageSizeTooLargeException import kafka.common.MessageSizeTooLargeException
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.utils.Utils._
object SyncProducer { object SyncProducer {
val RequestKey: Short = 0 val RequestKey: Short = 0
@ -124,6 +125,21 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging {
send(new BoundedByteBufferSend(new MultiProducerRequest(produces))) 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() = { def close() = {
lock synchronized { lock synchronized {
disconnect() 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.api.ProducerRequest
import kafka.serializer.Encoder import kafka.serializer.Encoder
import java.util.Properties
import kafka.producer._ import kafka.producer._
import kafka.utils.{ZKConfig, Utils, Logging}
import kafka.cluster.{Partition, Broker} import kafka.cluster.{Partition, Broker}
import collection.mutable.{ListBuffer, HashMap} import collection.mutable.{ListBuffer, HashMap}
import scala.collection.Map import scala.collection.Map
import kafka.common.{FailedToSendMessageException, InvalidPartitionException, NoBrokersForPartitionException} import kafka.common.{FailedToSendMessageException, InvalidPartitionException, NoBrokersForPartitionException}
import kafka.message.{Message, NoCompressionCodec, ByteBufferMessageSet} import kafka.message.{Message, NoCompressionCodec, ByteBufferMessageSet}
import kafka.utils.{Utils, Logging}
class DefaultEventHandler[K,V](config: ProducerConfig, // this api is for testing class DefaultEventHandler[K,V](config: ProducerConfig, // this api is for testing
private val partitioner: Partitioner[K], // use the other constructor private val partitioner: Partitioner[K], // use the other constructor
private val encoder: Encoder[V], private val encoder: Encoder[V],
private val producerPool: ProducerPool, private val producerPool: ProducerPool)
private val populateProducerPool: Boolean,
private var brokerPartitionInfo: BrokerPartitionInfo)
extends EventHandler[K,V] with Logging { 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 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]]) { def handle(events: Seq[ProducerData[K,V]]) {
lock synchronized { lock synchronized {
@ -81,7 +51,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
val partitionedData = partitionAndCollate(messages) val partitionedData = partitionAndCollate(messages)
for ( (brokerid, eventsPerBrokerMap) <- partitionedData) { for ( (brokerid, eventsPerBrokerMap) <- partitionedData) {
if (logger.isTraceEnabled) 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))) .format(partitionAndEvent._1, brokerid, partitionAndEvent._2)))
val messageSetPerBroker = groupMessagesToSet(eventsPerBrokerMap) val messageSetPerBroker = groupMessagesToSet(eventsPerBrokerMap)
@ -98,7 +68,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
numRetries +=1 numRetries +=1
Thread.sleep(config.producerRetryBackoffMs) Thread.sleep(config.producerRetryBackoffMs)
try { try {
brokerPartitionInfo.updateInfo brokerPartitionInfo.updateInfo()
handleSerializedData(eventsPerBroker, 0) handleSerializedData(eventsPerBroker, 0)
return return
} }
@ -125,15 +95,15 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
val brokerPartition = topicPartitionsList(partitionIndex) val brokerPartition = topicPartitionsList(partitionIndex)
var dataPerBroker: HashMap[(String, Int), Seq[ProducerData[K,Message]]] = null var dataPerBroker: HashMap[(String, Int), Seq[ProducerData[K,Message]]] = null
ret.get(brokerPartition.brokerId) match { ret.get(brokerPartition._2.id) match {
case Some(element) => case Some(element) =>
dataPerBroker = element.asInstanceOf[HashMap[(String, Int), Seq[ProducerData[K,Message]]]] dataPerBroker = element.asInstanceOf[HashMap[(String, Int), Seq[ProducerData[K,Message]]]]
case None => case None =>
dataPerBroker = new HashMap[(String, Int), Seq[ProducerData[K,Message]]] 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 var dataPerTopicPartition: ListBuffer[ProducerData[K,Message]] = null
dataPerBroker.get(topicAndPartition) match { dataPerBroker.get(topicAndPartition) match {
case Some(element) => case Some(element) =>
@ -147,9 +117,9 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
ret 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) 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) debug("Broker partitions registered for topic: " + pd.getTopic + " = " + topicPartitionsList)
val totalNumPartitions = topicPartitionsList.length val totalNumPartitions = topicPartitionsList.length
if(totalNumPartitions == 0) throw new NoBrokersForPartitionException("Partition = " + pd.getKey) 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 + throw new InvalidPartitionException("Invalid number of partitions: " + numPartitions +
"\n Valid values are > 0") "\n Valid values are > 0")
val partition = if(key == null) Utils.getNextRandomInt(numPartitions) val partition = if(key == null) Utils.getNextRandomInt(numPartitions)
else partitioner.partition(key , numPartitions) else partitioner.partition(key, numPartitions)
if(partition < 0 || partition >= numPartitions) if(partition < 0 || partition >= numPartitions)
throw new InvalidPartitionException("Invalid partition id : " + partition + throw new InvalidPartitionException("Invalid partition id : " + partition +
"\n Valid values are in the range inclusive [0, " + (numPartitions-1) + "]") "\n Valid values are in the range inclusive [0, " + (numPartitions-1) + "]")
@ -235,7 +205,5 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
def close() { def close() {
if (producerPool != null) if (producerPool != null)
producerPool.close 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)) Some(new TopicMetadataSend(topicsMetadata))
} }
} }

View File

@ -20,7 +20,7 @@ package kafka.server
import java.util.concurrent._ import java.util.concurrent._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import java.io.File 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.network.{SocketServerStats, SocketServer}
import kafka.log.LogManager import kafka.log.LogManager
@ -36,7 +36,6 @@ class KafkaServer(val config: KafkaConfig) extends Logging {
private val statsMBeanName = "kafka:type=kafka.SocketServerStats" private val statsMBeanName = "kafka:type=kafka.SocketServerStats"
var socketServer: SocketServer = null var socketServer: SocketServer = null
var requestHandlerPool: KafkaRequestHandlerPool = null var requestHandlerPool: KafkaRequestHandlerPool = null
val scheduler = new KafkaScheduler(1, "kafka-logcleaner-", false)
private var logManager: LogManager = null private var logManager: LogManager = null
/** /**
@ -52,7 +51,6 @@ class KafkaServer(val config: KafkaConfig) extends Logging {
cleanShutDownFile.delete cleanShutDownFile.delete
} }
logManager = new LogManager(config, logManager = new LogManager(config,
scheduler,
SystemTime, SystemTime,
1000L * 60 * config.logCleanupIntervalMinutes, 1000L * 60 * config.logCleanupIntervalMinutes,
1000L * 60 * 60 * config.logRetentionHours, 1000L * 60 * 60 * config.logRetentionHours,
@ -85,7 +83,6 @@ class KafkaServer(val config: KafkaConfig) extends Logging {
val canShutdown = isShuttingDown.compareAndSet(false, true); val canShutdown = isShuttingDown.compareAndSet(false, true);
if (canShutdown) { if (canShutdown) {
info("Shutting down Kafka server") info("Shutting down Kafka server")
scheduler.shutdown()
if (socketServer != null) if (socketServer != null)
socketServer.shutdown() socketServer.shutdown()
if(requestHandlerPool != null) 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) 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 * 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. * 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() { def handleNewSession() {
info("re-registering broker info in ZK for broker " + config.brokerId) info("re-registering broker info in ZK for broker " + config.brokerId)
registerBrokerInZk() registerBrokerInZk()
lock synchronized {
info("re-registering broker topics in ZK for broker " + config.brokerId)
for (topic <- topics)
registerTopicInZkInternal(topic)
}
info("done re-registering broker") info("done re-registering broker")
} }
} }

View File

@ -19,6 +19,7 @@ package kafka.utils
import java.util.concurrent._ import java.util.concurrent._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import java.lang.IllegalStateException
/** /**
* A scheduler for running jobs in the background * 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 { class KafkaScheduler(val numThreads: Int, val baseThreadName: String, isDaemon: Boolean) extends Logging {
private val threadId = new AtomicLong(0) private val threadId = new AtomicLong(0)
private val executor = new ScheduledThreadPoolExecutor(numThreads, new ThreadFactory() { private var executor:ScheduledThreadPoolExecutor = null
def newThread(runnable: Runnable): Thread = { startUp
val t = new Thread(runnable, baseThreadName + threadId.getAndIncrement)
t.setDaemon(isDaemon)
t
}
})
executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false)
executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false)
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) executor.scheduleAtFixedRate(Utils.loggedRunnable(fun), delayMs, periodMs, TimeUnit.MILLISECONDS)
}
def shutdownNow() { def shutdownNow() {
checkIfExecutorHasStarted
executor.shutdownNow() executor.shutdownNow()
info("force shutdown scheduler " + baseThreadName) info("force shutdown scheduler " + baseThreadName)
} }
def shutdown() { def shutdown() {
checkIfExecutorHasStarted
executor.shutdown() executor.shutdown()
info("shutdown scheduler " + baseThreadName) info("shutdown scheduler " + baseThreadName)
} }

View File

@ -19,12 +19,11 @@ package kafka.utils
import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.ZkClient
import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.consumer.{SimpleConsumer, ConsumerConfig}
import kafka.cluster.Partition
import kafka.api.OffsetRequest import kafka.api.OffsetRequest
import java.lang.IllegalStateException 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 { object UpdateOffsetsInZK {
val Earliest = "earliest" val Earliest = "earliest"
@ -46,7 +45,7 @@ object UpdateOffsetsInZK {
private def getAndSetOffsets(zkClient: ZkClient, offsetOption: Long, config: ConsumerConfig, topic: String): Unit = { private def getAndSetOffsets(zkClient: ZkClient, offsetOption: Long, config: ConsumerConfig, topic: String): Unit = {
val cluster = ZkUtils.getCluster(zkClient) val cluster = ZkUtils.getCluster(zkClient)
val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, List(topic).iterator) val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, List(topic).iterator)
var partitions: List[String] = Nil var partitions: Seq[String] = Nil
partitionsPerTopicMap.get(topic) match { partitionsPerTopicMap.get(topic) match {
case Some(l) => partitions = l.sortWith((s,t) => s < t) case Some(l) => partitions = l.sortWith((s,t) => s < t)
@ -54,22 +53,29 @@ object UpdateOffsetsInZK {
} }
var numParts = 0 var numParts = 0
for (partString <- partitions) { for (partition <- partitions) {
val part = Partition.parse(partString) val brokerHostingPartition = ZkUtils.getLeaderForPartition(zkClient, topic, partition.toInt)
val broker = cluster.getBroker(part.brokerId) match {
val broker = brokerHostingPartition match {
case Some(b) => b 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") "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) val topicDirs = new ZKGroupTopicDirs(config.groupId, topic)
println("updating partition " + part.name + " with new offset: " + offsets(0)) println("updating partition " + partition + " with new offset: " + offsets(0))
ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + part.name, offsets(0).toString) ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + partition, offsets(0).toString)
numParts += 1 numParts += 1
} }
println("updated the offset for " + numParts + " partitions") println("updated the offset for " + numParts + " partitions")
} }
private def usage() = { private def usage() = {

View File

@ -29,6 +29,7 @@ import scala.collection.mutable
import kafka.message.{NoCompressionCodec, CompressionCodec} import kafka.message.{NoCompressionCodec, CompressionCodec}
import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.ZkClient
import java.util.{Random, Properties} import java.util.{Random, Properties}
import kafka.network.{BoundedByteBufferReceive, Receive, BoundedByteBufferSend, Request}
/** /**
* Helper functions! * Helper functions!
@ -669,6 +670,20 @@ object Utils extends Logging {
case _ => // swallow 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) { 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 ={ def getTopicPartitionsPath(topic: String): String ={
getTopicPath(topic) + "/" + "partitions" getTopicPath(topic) + "/partitions"
} }
def getTopicPartitionPath(topic: String, partitionId: String): String ={ def getTopicPartitionPath(topic: String, partitionId: String): String ={
@ -62,6 +62,38 @@ object ZkUtils extends Logging {
ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted 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) { def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, creator: String, port: Int) {
val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id
val broker = new Broker(id, creator, host, port) val broker = new Broker(id, creator, host, port)
@ -77,6 +109,11 @@ object ZkUtils extends Logging {
info("Registering broker " + brokerIdPath + " succeeded with " + broker) 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. * make sure a persistent path exists in ZK. Create the path if not exist.
*/ */
@ -269,30 +306,17 @@ object ZkUtils extends Logging {
cluster cluster
} }
def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, List[String]] = { def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, Seq[String]] = {
val ret = new mutable.HashMap[String, List[String]]() val ret = new mutable.HashMap[String, Seq[String]]()
for (topic <- topics) { topics.foreach { topic =>
var partList: List[String] = Nil // get the partitions that exist for topic
val brokers = getChildrenParentMayNotExist(zkClient, BrokerTopicsPath + "/" + topic) val partitions = getChildrenParentMayNotExist(zkClient, getTopicPartitionsPath(topic))
for (broker <- brokers) { debug("children of /brokers/topics/%s are %s".format(topic, partitions))
val nParts = readData(zkClient, BrokerTopicsPath + "/" + topic + "/" + broker).toInt ret += (topic -> partitions.sortWith((s,t) => s < t))
for (part <- 0 until nParts)
partList ::= broker + "-" + part
}
partList = partList.sortWith((s,t) => s < t)
ret += (topic -> partList)
} }
ret 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) { def deletePartition(zkClient : ZkClient, brokerId: Int, topic: String) {
val brokerIdPath = BrokerIdsPath + "/" + brokerId val brokerIdPath = BrokerIdsPath + "/" + brokerId
zkClient.delete(brokerIdPath) zkClient.delete(brokerIdPath)

View File

@ -169,7 +169,8 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness {
case Some(metadata) => assertEquals(topic, metadata.topic) case Some(metadata) => assertEquals(topic, metadata.topic)
assertNotNull("partition metadata list cannot be null", metadata.partitionsMetadata) assertNotNull("partition metadata list cannot be null", metadata.partitionsMetadata)
assertEquals("partition metadata list length should be 2", 2, metadata.partitionsMetadata.size) 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 actualReplicaAssignment = metadata.partitionsMetadata.map(p => p.replicas)
val actualReplicaList = actualReplicaAssignment.map(r => r.map(b => b.id.toString).toList).toList val actualReplicaList = actualReplicaAssignment.map(r => r.map(b => b.id.toString).toList).toList
assertEquals(expectedReplicaAssignment.toList, actualReplicaList) assertEquals(expectedReplicaAssignment.toList, actualReplicaList)

View File

@ -22,15 +22,19 @@ import junit.framework.Assert._
import kafka.integration.KafkaServerTestHarness import kafka.integration.KafkaServerTestHarness
import kafka.server._ import kafka.server._
import scala.collection._ import scala.collection._
import kafka.utils.{Utils, Logging}
import kafka.utils.{TestZKUtils, TestUtils}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import org.apache.log4j.{Level, Logger} import org.apache.log4j.{Level, Logger}
import kafka.message._ import kafka.message._
import kafka.serializer.StringDecoder 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 { class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging {
var dirs : ZKGroupTopicDirs = null
val zookeeperConnect = TestZKUtils.zookeeperConnect val zookeeperConnect = TestZKUtils.zookeeperConnect
val numNodes = 2 val numNodes = 2
val numParts = 2 val numParts = 2
@ -48,25 +52,28 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val consumer3 = "consumer3" val consumer3 = "consumer3"
val nMessages = 2 val nMessages = 2
override def setUp() {
super.setUp()
dirs = new ZKGroupTopicDirs(group, topic)
}
def testBasic() { def testBasic() {
val requestHandlerLogger = Logger.getLogger(classOf[KafkaApis]) val requestHandlerLogger = Logger.getLogger(classOf[KafkaApis])
requestHandlerLogger.setLevel(Level.FATAL) requestHandlerLogger.setLevel(Level.FATAL)
var actualMessages: List[Message] = Nil
// test consumer timeout logic // test consumer timeout logic
val consumerConfig0 = new ConsumerConfig( val consumerConfig0 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { TestUtils.createConsumerProperties(zkConnect, group, consumer0)) {
override val consumerTimeoutMs = 200 override val consumerTimeoutMs = 200
} }
val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true) 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; // no messages to consume, we should hit timeout;
// also the iterator should support re-entrant, so loop it twice // also the iterator should support re-entrant, so loop it twice
for (i <- 0 until 2) { for (i <- 0 until 2) {
try { try {
getMessagesSortedByChecksum(nMessages*2, topicMessageStreams0) getMessages(nMessages*2, topicMessageStreams0)
fail("should get an exception") fail("should get an exception")
} }
catch { catch {
@ -78,14 +85,26 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
zkConsumerConnector0.shutdown zkConsumerConnector0.shutdown
// send some messages to each broker // 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 // create a consumer
val consumerConfig1 = new ConsumerConfig( val consumerConfig1 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer1)) TestUtils.createConsumerProperties(zkConnect, group, consumer1))
val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true)
val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1))
val receivedMessages1 = getMessagesSortedByChecksum(nMessages*2, topicMessageStreams1) val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1)
assertEquals(sentMessages1.size, receivedMessages1.size)
assertEquals(sentMessages1, receivedMessages1) 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 // commit consumed offsets
zkConsumerConnector1.commitOffsets zkConsumerConnector1.commitOffsets
@ -93,15 +112,25 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val consumerConfig2 = new ConsumerConfig( val consumerConfig2 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer2)) TestUtils.createConsumerProperties(zkConnect, group, consumer2))
val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) 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 // 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) 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) val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum)
assertEquals(sentMessages2, receivedMessages2) 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 // create a consumer with empty map
val consumerConfig3 = new ConsumerConfig( val consumerConfig3 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer3)) TestUtils.createConsumerProperties(zkConnect, group, consumer3))
@ -109,13 +138,20 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]())
// send some messages to each broker // send some messages to each broker
Thread.sleep(200) 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) Thread.sleep(200)
val receivedMessages3_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1) val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1)
val receivedMessages3_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2) val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2)
val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum) val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum)
assertEquals(sentMessages3.size, receivedMessages3.size)
assertEquals(sentMessages3, receivedMessages3) assertEquals(sentMessages3, receivedMessages3)
// also check partition ownership
val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir)
assertEquals(expected_2, actual_3)
zkConsumerConnector1.shutdown zkConsumerConnector1.shutdown
zkConsumerConnector2.shutdown zkConsumerConnector2.shutdown
zkConsumerConnector3.shutdown zkConsumerConnector3.shutdown
@ -127,48 +163,73 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler])
requestHandlerLogger.setLevel(Level.FATAL) requestHandlerLogger.setLevel(Level.FATAL)
println("Sending messages for 1st consumer")
// send some messages to each broker // 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 // create a consumer
val consumerConfig1 = new ConsumerConfig( val consumerConfig1 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer1)) TestUtils.createConsumerProperties(zkConnect, group, consumer1))
val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true)
val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1))
val receivedMessages1 = getMessagesSortedByChecksum(nMessages*2, topicMessageStreams1) val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1)
assertEquals(sentMessages1.size, receivedMessages1.size)
assertEquals(sentMessages1, receivedMessages1) 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 // commit consumed offsets
zkConsumerConnector1.commitOffsets zkConsumerConnector1.commitOffsets
println("Sending more messages for 2nd consumer")
// create a consumer // create a consumer
val consumerConfig2 = new ConsumerConfig( val consumerConfig2 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer2)) TestUtils.createConsumerProperties(zkConnect, group, consumer2))
val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) 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 // 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) 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) val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum)
assertEquals(sentMessages2, receivedMessages2) 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 // create a consumer with empty map
println("Sending more messages for 3rd consumer")
val consumerConfig3 = new ConsumerConfig( val consumerConfig3 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer3)) TestUtils.createConsumerProperties(zkConnect, group, consumer3))
val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true)
val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]())
// send some messages to each broker // send some messages to each broker
Thread.sleep(200) 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) Thread.sleep(200)
val receivedMessages3_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1) val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1)
val receivedMessages3_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2) val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2)
val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum) val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum)
assertEquals(sentMessages3.size, receivedMessages3.size)
assertEquals(sentMessages3, receivedMessages3) assertEquals(sentMessages3, receivedMessages3)
// also check partition ownership
val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir)
assertEquals(expected_2, actual_3)
zkConsumerConnector1.shutdown zkConsumerConnector1.shutdown
zkConsumerConnector2.shutdown zkConsumerConnector2.shutdown
zkConsumerConnector3.shutdown zkConsumerConnector3.shutdown
@ -187,7 +248,10 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
Thread.sleep(500) Thread.sleep(500)
// send some messages to each broker // 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 // test consumer timeout logic
val consumerConfig0 = new ConsumerConfig( val consumerConfig0 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { TestUtils.createConsumerProperties(zkConnect, group, consumer0)) {
@ -195,16 +259,30 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
} }
val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true) val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true)
val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> 1)) 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 zkConsumerConnector0.shutdown
// at this point, only some part of the message set was consumed. So consumed offset should still be 0 // 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 // also fetched offset should be 0
val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig0, true) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig0, true)
val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) 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 sortedReceivedMessages = receivedMessages.sortWith((s,t) => s.checksum < t.checksum)
val sortedSentMessages = sentMessages.sortWith((s,t) => s.checksum < t.checksum) val sortedSentMessages = sentMessages.sortWith((s,t) => s.checksum < t.checksum)
assertEquals(sortedSentMessages, sortedReceivedMessages) 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 zkConsumerConnector1.shutdown
requestHandlerLogger.setLevel(Level.ERROR) requestHandlerLogger.setLevel(Level.ERROR)
@ -214,17 +292,18 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler])
requestHandlerLogger.setLevel(Level.FATAL) requestHandlerLogger.setLevel(Level.FATAL)
val sentMessages = sendMessages(nMessages, "batch1", NoCompressionCodec). // send some messages to each broker
map(m => Utils.toString(m.payload, "UTF-8")). 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) 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 = val zkConsumerConnector =
new ZookeeperConsumerConnector(consumerConfig, true) new ZookeeperConsumerConnector(consumerConfig, true)
val topicMessageStreams = val topicMessageStreams =
zkConsumerConnector.createMessageStreams( zkConsumerConnector.createMessageStreams(Predef.Map(topic -> 1), new StringDecoder)
Predef.Map(topic -> numNodes*numParts/2), new StringDecoder)
var receivedMessages: List[String] = Nil var receivedMessages: List[String] = Nil
for ((topic, messageStreams) <- topicMessageStreams) { for ((topic, messageStreams) <- topicMessageStreams) {
@ -245,31 +324,106 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
requestHandlerLogger.setLevel(Level.ERROR) 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 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) { for (partition <- 0 until numParts) {
val ms = 0.until(messagesPerNode).map(x => val ms = 0.until(messagesPerNode).map(x =>
new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray
val mSet = new ByteBufferMessageSet(compressionCodec = compression, messages = ms: _*)
for (message <- ms) for (message <- ms)
messages ::= message 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() producer.close()
messages messages.reverse
} }
def sendMessages(messagesPerNode: Int, header: String, compression: CompressionCodec = NoCompressionCodec): List[Message]= { def sendMessages(messagesPerNode: Int, header: String, compression: CompressionCodec = NoCompressionCodec): List[Message]= {
var messages: List[Message] = Nil var messages: List[Message] = Nil
for(conf <- configs) { for(conf <- configs) {
messages ++= sendMessages(conf, messagesPerNode, header, compression) messages ++= sendMessages(conf, messagesPerNode, header, compression, numParts)
} }
messages.sortWith((s,t) => s.checksum < t.checksum) messages.sortWith((s,t) => s.checksum < t.checksum)
} }
def getMessagesSortedByChecksum(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[Message]]]): List[Message]= { def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[Message]]]): List[Message]= {
val messages = TestUtils.getConsumedMessages(nMessagesPerThread, topicMessageStreams) 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) 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 package kafka.integration
import junit.framework.Assert._ import junit.framework.Assert._
import kafka.zk.ZooKeeperTestHarness
import java.nio.channels.ClosedByInterruptException import java.nio.channels.ClosedByInterruptException
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import kafka.utils.{ZKGroupTopicDirs, Logging} import kafka.utils.{ZKGroupTopicDirs, Logging}
@ -27,15 +26,16 @@ import kafka.server._
import org.apache.log4j.{Level, Logger} import org.apache.log4j.{Level, Logger}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.utils.TestUtils 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 topic = "test_topic"
val group = "default_group" val group = "default_group"
val testConsumer = "consumer" val testConsumer = "consumer"
val brokerPort = 9892 val brokerPort = 9892
val kafkaConfig = new KafkaConfig(TestUtils.createBrokerConfig(0, brokerPort)) val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0, brokerPort)))
var kafkaServer : KafkaServer = null
val numMessages = 10 val numMessages = 10
val largeOffset = 10000 val largeOffset = 10000
val smallOffset = -1 val smallOffset = -1
@ -44,7 +44,6 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log
override def setUp() { override def setUp() {
super.setUp() super.setUp()
kafkaServer = TestUtils.createServer(kafkaConfig)
// temporarily set request handler logger to a higher level // temporarily set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.FATAL) requestHandlerLogger.setLevel(Level.FATAL)
@ -53,15 +52,14 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log
override def tearDown() { override def tearDown() {
// restore set request handler logger to a higher level // restore set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.ERROR) requestHandlerLogger.setLevel(Level.ERROR)
kafkaServer.shutdown
super.tearDown super.tearDown
} }
def testEarliestOffsetResetForward() = { def testEarliestOffsetResetForward() = {
val producer = TestUtils.createProducer("localhost", brokerPort) val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect)
for(i <- 0 until numMessages) { 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 // 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") consumerProps.put("consumer.timeout.ms", "2000")
val consumerConfig = new ConsumerConfig(consumerProps) 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) info("Updated consumer offset to " + largeOffset)
Thread.sleep(500) Thread.sleep(500)
@ -112,10 +110,10 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log
} }
def testEarliestOffsetResetBackward() = { def testEarliestOffsetResetBackward() = {
val producer = TestUtils.createProducer("localhost", brokerPort) val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect)
for(i <- 0 until numMessages) { 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 // 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") consumerProps.put("consumer.timeout.ms", "2000")
val consumerConfig = new ConsumerConfig(consumerProps) 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) info("Updated consumer offset to " + smallOffset)
@ -145,7 +143,7 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log
} }
} }
catch { catch {
case _: InterruptedException => case _: InterruptedException =>
case _: ClosedByInterruptException => case _: ClosedByInterruptException =>
case e => throw e case e => throw e
} }
@ -159,16 +157,15 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log
threadList(0).join(2000) threadList(0).join(2000)
info("Asserting...")
assertEquals(numMessages, nMessages.get) assertEquals(numMessages, nMessages.get)
consumerConnector.shutdown consumerConnector.shutdown
} }
def testLatestOffsetResetForward() = { def testLatestOffsetResetForward() = {
val producer = TestUtils.createProducer("localhost", brokerPort) val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect)
for(i <- 0 until numMessages) { 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 // 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") consumerProps.put("consumer.timeout.ms", "2000")
val consumerConfig = new ConsumerConfig(consumerProps) 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) info("Updated consumer offset to " + largeOffset)
@ -198,7 +195,7 @@ class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Log
} }
} }
catch { catch {
case _: InterruptedException => case _: InterruptedException =>
case _: ClosedByInterruptException => case _: ClosedByInterruptException =>
case e => throw e case e => throw e
} }

View File

@ -28,6 +28,7 @@ import kafka.server._
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.integration.KafkaServerTestHarness import kafka.integration.KafkaServerTestHarness
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.producer.{ProducerData, Producer}
class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
@ -35,7 +36,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
val configs = val configs =
for(props <- TestUtils.createBrokerConfigs(numNodes)) for(props <- TestUtils.createBrokerConfigs(numNodes))
yield new KafkaConfig(props) yield new KafkaConfig(props)
val messages = new mutable.HashMap[Int, ByteBufferMessageSet] val messages = new mutable.HashMap[Int, Seq[Message]]
val topic = "topic" val topic = "topic"
val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.brokerId.toString, "localhost", c.port))) val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.brokerId.toString, "localhost", c.port)))
val shutdown = ZookeeperConsumerConnector.shutdownCommand val shutdown = ZookeeperConsumerConnector.shutdownCommand
@ -79,11 +80,10 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
def sendMessages(messagesPerNode: Int): Int = { def sendMessages(messagesPerNode: Int): Int = {
var count = 0 var count = 0
for(conf <- configs) { 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 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 -> ms
messages += conf.brokerId -> mSet producer.send(new ProducerData[String, Message](topic, topic, ms))
producer.send(topic, mSet)
producer.close() producer.close()
count += ms.size count += ms.size
} }

View File

@ -17,33 +17,31 @@
package kafka.integration package kafka.integration
import kafka.api.{FetchRequestBuilder, ProducerRequest} import kafka.api.FetchRequestBuilder
import kafka.common.OffsetOutOfRangeException import kafka.common.OffsetOutOfRangeException
import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.message.{Message, ByteBufferMessageSet}
import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig} import kafka.server.{KafkaRequestHandler, KafkaConfig}
import kafka.utils.{TestUtils, Utils} import kafka.utils.TestUtils
import kafka.zk.ZooKeeperTestHarness
import org.apache.log4j.{Level, Logger} import org.apache.log4j.{Level, Logger}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import scala.collection._ import scala.collection._
import kafka.producer.ProducerData
/** /**
* End to end tests of the primitive apis against a local server * 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 port = TestUtils.choosePort
val props = TestUtils.createBrokerConfig(0, port) val props = TestUtils.createBrokerConfig(0, port)
val config = new KafkaConfig(props) val config = new KafkaConfig(props)
val configs = List(config) val configs = List(config)
var servers: List[KafkaServer] = null
val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler])
override def setUp() { override def setUp() {
super.setUp super.setUp
if(configs.size <= 0) if(configs.size <= 0)
throw new IllegalArgumentException("Must suply at least one server config.") throw new IllegalArgumentException("Must suply at least one server config.")
servers = configs.map(TestUtils.createServer(_))
// temporarily set request handler logger to a higher level // temporarily set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.FATAL) requestHandlerLogger.setLevel(Level.FATAL)
@ -54,24 +52,21 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
requestHandlerLogger.setLevel(Level.ERROR) requestHandlerLogger.setLevel(Level.ERROR)
super.tearDown super.tearDown
servers.map(server => server.shutdown())
servers.map(server => Utils.rm(server.config.logDir))
} }
def testProduceAndFetch() { def testProduceAndFetch() {
// send some messages // send some messages
val topic = "test" val topic = "test"
val sent = new ByteBufferMessageSet(NoCompressionCodec, val sentMessages = List(new Message("hello".getBytes()), new Message("there".getBytes()))
new Message("hello".getBytes()), new Message("there".getBytes())) val producerData = new ProducerData[String, Message](topic, topic, sentMessages)
producer.send(topic, sent)
sent.getBuffer.rewind
producer.send(producerData)
var fetchedMessage: ByteBufferMessageSet = null var fetchedMessage: ByteBufferMessageSet = null
while(fetchedMessage == null || fetchedMessage.validBytes == 0) { while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
fetchedMessage = fetched.messageSet(topic, 0) 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 // send an invalid offset
try { try {
@ -87,14 +82,12 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
// send some messages, with non-ordered topics // send some messages, with non-ordered topics
val topicOffsets = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); 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() val builder = new FetchRequestBuilder()
for( (topic, offset) <- topicOffsets) { for( (topic, offset) <- topicOffsets) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val producedData = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) messages += topic -> producedData
producer.send(topic, set) producer.send(new ProducerData[String, Message](topic, topic, producedData))
set.getBuffer.rewind
messages += topic -> set
builder.addFetch(topic, offset, 0, 10000) builder.addFetch(topic, offset, 0, 10000)
} }
@ -104,7 +97,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
val response = consumer.fetch(request) val response = consumer.fetch(request)
for( (topic, offset) <- topicOffsets) { for( (topic, offset) <- topicOffsets) {
val fetched = response.messageSet(topic, offset) 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 responses.messageSet(topic, offset).iterator
fail("Expected an OffsetOutOfRangeException exception to be thrown") fail("Expected an OffsetOutOfRangeException exception to be thrown")
} catch { } catch {
case e: OffsetOutOfRangeException => case e: OffsetOutOfRangeException =>
} }
} }
} }
@ -130,20 +123,16 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
def testMultiProduce() { def testMultiProduce() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); 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() val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerData[String, Message]] = Nil
for(topic <- topics) { for(topic <- topics) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val set = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set messages += topic -> set
produceList ::= new ProducerRequest(topic, 0, set) produceList ::= new ProducerData[String, Message](topic, topic, set)
builder.addFetch(topic, 0, 0, 10000) builder.addFetch(topic, 0, 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 // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
@ -151,39 +140,33 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
val response = consumer.fetch(request) val response = consumer.fetch(request)
for(topic <- topics) { for(topic <- topics) {
val fetched = response.messageSet(topic, 0) 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() { def testMultiProduceResend() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); 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() val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerData[String, Message]] = Nil
for(topic <- topics) { for(topic <- topics) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val set = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set messages += topic -> set
produceList ::= new ProducerRequest(topic, 0, set) produceList ::= new ProducerData[String, Message](topic, topic, set)
builder.addFetch(topic, 0, 0, 10000) builder.addFetch(topic, 0, 0, 10000)
} }
producer.multiSend(produceList.toArray) producer.send(produceList: _*)
// resend the same multisend
producer.multiSend(produceList.toArray)
for (messageSet <- messages.values)
messageSet.getBuffer.rewind
producer.send(produceList: _*)
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(750) Thread.sleep(750)
val request = builder.build() val request = builder.build()
val response = consumer.fetch(request) val response = consumer.fetch(request)
for(topic <- topics) { for(topic <- topics) {
val topicMessages = response.messageSet(topic, 0) val topicMessages = response.messageSet(topic, 0)
TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).map(m => m.message).iterator, TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).iterator,
messages(topic).map(m => m.message).iterator), messages(topic).iterator),
topicMessages.iterator.map(_.message)) topicMessages.iterator.map(_.message))
} }
} }

View File

@ -24,10 +24,11 @@ import kafka.api.FetchRequestBuilder
import kafka.common.InvalidMessageSizeException import kafka.common.InvalidMessageSizeException
import kafka.consumer.{ZookeeperConsumerConnector, ConsumerConfig} import kafka.consumer.{ZookeeperConsumerConnector, ConsumerConfig}
import kafka.integration.{KafkaServerTestHarness, ProducerConsumerTestHarness} import kafka.integration.{KafkaServerTestHarness, ProducerConsumerTestHarness}
import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.message.Message
import kafka.utils.{Utils, TestUtils} import kafka.utils.{Utils, TestUtils}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import org.apache.log4j.{Logger, Level} import org.apache.log4j.{Logger, Level}
import kafka.producer.ProducerData
class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness { class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness {
val port = TestUtils.choosePort val port = TestUtils.choosePort
@ -47,8 +48,9 @@ class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness wit
fetcherLogger.setLevel(Level.FATAL) fetcherLogger.setLevel(Level.FATAL)
// send some messages // send some messages
val sent1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message("hello".getBytes())) val producerData = new ProducerData[String, Message](topic, topic, List(new Message("hello".getBytes())))
producer.send(topic, sent1)
producer.send(producerData)
Thread.sleep(200) Thread.sleep(200)
// corrupt the file on disk // corrupt the file on disk

View File

@ -17,25 +17,25 @@
package kafka.integration package kafka.integration
import java.io.File
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util.Properties
import junit.framework.Assert._ 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.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.server.{KafkaRequestHandler, KafkaConfig}
import kafka.utils.TestUtils
import org.apache.log4j.{Level, Logger} import org.apache.log4j.{Level, Logger}
import org.scalatest.junit.JUnit3Suite 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._ import scala.collection._
/** /**
* End to end tests of the primitive apis against a local server * 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 port = TestUtils.choosePort
val props = TestUtils.createBrokerConfig(0, port) val props = TestUtils.createBrokerConfig(0, port)
@ -45,6 +45,20 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val configs = List(config) val configs = List(config)
val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) 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() { def testFetchRequestCanProperlySerialize() {
val request = new FetchRequestBuilder() val request = new FetchRequestBuilder()
.correlationId(100) .correlationId(100)
@ -83,7 +97,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val topic = "test-topic" val topic = "test-topic"
val props = new Properties() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") 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 config = new ProducerConfig(props)
val stringProducer1 = new Producer[String, String](config) val stringProducer1 = new Producer[String, String](config)
@ -111,7 +125,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val topic = "test-topic" val topic = "test-topic"
val props = new Properties() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") props.put("serializer.class", "kafka.serializer.StringEncoder")
props.put("broker.list", "0:localhost:" + port) props.put("zk.connect", TestZKUtils.zookeeperConnect)
props.put("compression", "true") props.put("compression", "true")
val config = new ProducerConfig(props) val config = new ProducerConfig(props)
@ -133,14 +147,13 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
// send some messages // send some messages
val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); 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() val builder = new FetchRequestBuilder()
for( (topic, partition) <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) val producerData = new ProducerData[String, Message](topic, topic, messageList)
messages += topic -> set messages += topic -> messageList
producer.send(topic, set) producer.send(producerData)
set.getBuffer.rewind
builder.addFetch(topic, partition, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
@ -150,7 +163,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val response = consumer.fetch(request) val response = consumer.fetch(request)
for( (topic, partition) <- topics) { for( (topic, partition) <- topics) {
val fetched = response.messageSet(topic, partition) 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 { } catch {
case e: OffsetOutOfRangeException => "this is good" case e: OffsetOutOfRangeException => "this is good"
} }
} }
{ {
// send some invalid partitions // send some invalid partitions
@ -199,14 +212,13 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
// send some messages // send some messages
val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); 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() val builder = new FetchRequestBuilder()
for( (topic, partition) <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(DefaultCompressionCodec, val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) val producerData = new ProducerData[String, Message](topic, topic, messageList)
messages += topic -> set messages += topic -> messageList
producer.send(topic, set) producer.send(producerData)
set.getBuffer.rewind
builder.addFetch(topic, partition, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
@ -216,7 +228,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val response = consumer.fetch(request) val response = consumer.fetch(request)
for( (topic, partition) <- topics) { for( (topic, partition) <- topics) {
val fetched = response.messageSet(topic, partition) 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() { def testMultiProduce() {
// send some messages // send some messages
val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); 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() val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerData[String, Message]] = Nil
for( (topic, partition) <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) val producerData = new ProducerData[String, Message](topic, topic, messageList)
messages += topic -> set messages += topic -> messageList
produceList ::= new ProducerRequest(topic, 0, set) producer.send(producerData)
builder.addFetch(topic, partition, 0, 10000) 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 // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val request = builder.build() val request = builder.build()
val response = consumer.fetch(request) val response = consumer.fetch(request)
for( (topic, partition) <- topics) { for( (topic, partition) <- topics) {
val fetched = response.messageSet(topic, partition) 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() { def testMultiProduceWithCompression() {
// send some messages // send some messages
val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); 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() val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerData[String, Message]] = Nil
for( (topic, partition) <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(DefaultCompressionCodec, val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) val producerData = new ProducerData[String, Message](topic, topic, messageList)
messages += topic -> set messages += topic -> messageList
produceList ::= new ProducerRequest(topic, 0, set) producer.send(producerData)
builder.addFetch(topic, partition, 0, 10000) 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 // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
@ -313,7 +319,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val response = consumer.fetch(request) val response = consumer.fetch(request)
for( (topic, partition) <- topics) { for( (topic, partition) <- topics) {
val fetched = response.messageSet(topic, 0) 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 kafka.consumer.SimpleConsumer
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import java.util.Properties 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 port: Int
val host = "localhost" val host = "localhost"
var producer: SyncProducer = null var producer: Producer[String, Message] = null
var consumer: SimpleConsumer = null var consumer: SimpleConsumer = null
override def setUp() { override def setUp() {
super.setUp
val props = new Properties() val props = new Properties()
props.put("host", host) props.put("partitioner.class", "kafka.utils.StaticPartitioner")
props.put("port", port.toString) props.put("zk.connect", TestZKUtils.zookeeperConnect)
props.put("buffer.size", "65536") props.put("buffer.size", "65536")
props.put("connect.timeout.ms", "100000") props.put("connect.timeout.ms", "100000")
props.put("reconnect.interval", "10000") props.put("reconnect.interval", "10000")
producer = new SyncProducer(new SyncProducerConfig(props)) producer = new Producer(new ProducerConfig(props))
consumer = new SimpleConsumer(host, consumer = new SimpleConsumer(host,
port, port,
1000000, 1000000,
64*1024) 64*1024)
super.setUp
} }
override def tearDown() { override def tearDown() {
super.tearDown
producer.close() producer.close()
consumer.close() consumer.close()
super.tearDown
} }
} }

View File

@ -24,10 +24,10 @@ import kafka.utils.{Utils, Logging}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import kafka.javaapi.message.ByteBufferMessageSet
import kafka.consumer.{ConsumerConfig, KafkaMessageStream} import kafka.consumer.{ConsumerConfig, KafkaMessageStream}
import org.apache.log4j.{Level, Logger} import org.apache.log4j.{Level, Logger}
import kafka.message._ import kafka.message._
import kafka.javaapi.producer.{ProducerData, Producer}
class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { 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]= { def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compressed: CompressionCodec): List[Message]= {
var messages: List[Message] = Nil 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) { for (partition <- 0 until numParts) {
val ms = 0.until(messagesPerNode).map(x => val ms = 0.until(messagesPerNode).map(x =>
new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray
val mSet = new ByteBufferMessageSet(compressionCodec = compressed, messages = getMessageList(ms: _*))
for (message <- ms) for (message <- ms)
messages ::= message 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 messages
} }
@ -106,12 +107,6 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
messages.sortWith((s,t) => s.checksum < t.checksum) 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] = { private def toJavaMap(scalaMap: Map[String, Int]): java.util.Map[String, java.lang.Integer] = {
val javaMap = new java.util.HashMap[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])) 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.zk.ZooKeeperTestHarness
import kafka.utils.{TestZKUtils, Utils, MockTime, TestUtils} import kafka.utils.{TestZKUtils, Utils, MockTime, TestUtils}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.admin.CreateTopicCommand
class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
@ -34,6 +35,8 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
var logManager: LogManager = null var logManager: LogManager = null
var config:KafkaConfig = null var config:KafkaConfig = null
val zookeeperConnect = TestZKUtils.zookeeperConnect val zookeeperConnect = TestZKUtils.zookeeperConnect
val name = "kafka"
val veryLargeLogFlushInterval = 10000000L
override def setUp() { override def setUp() {
super.setUp() super.setUp()
@ -41,9 +44,13 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
config = new KafkaConfig(props) { config = new KafkaConfig(props) {
override val logFileSize = 1024 override val logFileSize = 1024
} }
logManager = new LogManager(config, null, time, -1, maxLogAge, false) logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false)
logManager.startup logManager.startup
logDir = logManager.logDir 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() { override def tearDown() {
@ -55,7 +62,6 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
@Test @Test
def testCreateLog() { def testCreateLog() {
val name = "kafka"
val log = logManager.getOrCreateLog(name, 0) val log = logManager.getOrCreateLog(name, 0)
val logFile = new File(config.logDir, name + "-0") val logFile = new File(config.logDir, name + "-0")
assertTrue(logFile.exists) assertTrue(logFile.exists)
@ -64,7 +70,6 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
@Test @Test
def testGetLog() { def testGetLog() {
val name = "kafka"
val log = logManager.getLog(name, 0) val log = logManager.getLog(name, 0)
val logFile = new File(config.logDir, name + "-0") val logFile = new File(config.logDir, name + "-0")
assertTrue(!logFile.exists) assertTrue(!logFile.exists)
@ -72,7 +77,7 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
@Test @Test
def testCleanupExpiredSegments() { def testCleanupExpiredSegments() {
val log = logManager.getOrCreateLog("cleanup", 0) val log = logManager.getOrCreateLog(name, 0)
var offset = 0L var offset = 0L
for(i <- 0 until 1000) { for(i <- 0 until 1000) {
var set = TestUtils.singleMessageSet("test".getBytes()) 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 logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Int] // keep exactly 6 segments + 1 roll over
override val logRetentionHours = retentionHours override val logRetentionHours = retentionHours
} }
logManager = new LogManager(config, null, time, -1, retentionMs, false) logManager = new LogManager(config, time, veryLargeLogFlushInterval, retentionMs, false)
logManager.startup logManager.startup
// create a log // create a log
val log = logManager.getOrCreateLog("cleanup", 0) val log = logManager.getOrCreateLog(name, 0)
var offset = 0L var offset = 0L
// add a bunch of messages that should be larger than the retentionSize // add a bunch of messages that should be larger than the retentionSize
@ -151,14 +156,14 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
logManager.close logManager.close
Thread.sleep(100) Thread.sleep(100)
config = new KafkaConfig(props) { config = new KafkaConfig(props) {
override val logFileSize = 1024 *1024 *1024 override val logFileSize = 1024 *1024 *1024
override val flushSchedulerThreadRate = 50 override val flushSchedulerThreadRate = 50
override val flushInterval = Int.MaxValue override val flushInterval = Int.MaxValue
override val flushIntervalMap = Utils.getTopicFlushIntervals("timebasedflush:100") 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 logManager.startup
val log = logManager.getOrCreateLog("timebasedflush", 0) val log = logManager.getOrCreateLog(name, 0)
for(i <- 0 until 200) { for(i <- 0 until 200) {
var set = TestUtils.singleMessageSet("test".getBytes()) var set = TestUtils.singleMessageSet("test".getBytes())
log.append(set) log.append(set)
@ -177,12 +182,12 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
override val logFileSize = 256 override val logFileSize = 256
override val topicPartitionsMap = Utils.getTopicPartitions("testPartition:2") 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 logManager.startup
for(i <- 0 until 2) { for(i <- 0 until 1) {
val log = logManager.getOrCreateLog("testPartition", i) val log = logManager.getOrCreateLog(name, i)
for(i <- 0 until 250) { for(i <- 0 until 250) {
var set = TestUtils.singleMessageSet("test".getBytes()) var set = TestUtils.singleMessageSet("test".getBytes())
log.append(set) log.append(set)
@ -191,7 +196,7 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
try try
{ {
val log = logManager.getOrCreateLog("testPartition", 2) val log = logManager.getOrCreateLog(name, 2)
assertTrue("Should not come here", log != null) assertTrue("Should not come here", log != null)
} catch { } catch {
case _ => case _ =>

View File

@ -29,6 +29,7 @@ import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message}
import org.apache.log4j._ import org.apache.log4j._
import kafka.zk.ZooKeeperTestHarness import kafka.zk.ZooKeeperTestHarness
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.admin.CreateTopicCommand
import kafka.api.{FetchRequestBuilder, OffsetRequest} import kafka.api.{FetchRequestBuilder, OffsetRequest}
object LogOffsetTest { object LogOffsetTest {
@ -51,7 +52,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
val config: Properties = createBrokerConfig(1, brokerPort) val config: Properties = createBrokerConfig(1, brokerPort)
val logDirPath = config.getProperty("log.dir") val logDirPath = config.getProperty("log.dir")
logDir = new File(logDirPath) logDir = new File(logDirPath)
server = TestUtils.createServer(new KafkaConfig(config)) server = TestUtils.createServer(new KafkaConfig(config))
simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024) simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024)
} }
@ -94,10 +95,12 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
@Test @Test
def testGetOffsetsBeforeLatestTime() { def testGetOffsetsBeforeLatestTime() {
val topicPartition = "kafka-" + 0 val topicPartition = "kafka-" + 0
val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition
val topic = topicPartition.split("-").head val topic = topicPartition.split("-").head
val part = Integer.valueOf(topicPartition.split("-").last).intValue 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 logManager = server.getLogManager
val log = logManager.getOrCreateLog(topic, part) val log = logManager.getOrCreateLog(topic, part)
@ -133,6 +136,9 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
val topic = topicPartition.split("-").head val topic = topicPartition.split("-").head
val part = Integer.valueOf(topicPartition.split("-").last).intValue 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 var offsetChanged = false
for(i <- 1 to 14) { for(i <- 1 to 14) {
val consumerOffsets = simpleConsumer.getOffsetsBefore(topic, part, val consumerOffsets = simpleConsumer.getOffsetsBefore(topic, part,
@ -147,11 +153,13 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
@Test @Test
def testGetOffsetsBeforeNow() { def testGetOffsetsBeforeNow() {
val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(10) val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(3)
val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition
val topic = topicPartition.split("-").head val topic = topicPartition.split("-").head
val part = Integer.valueOf(topicPartition.split("-").last).intValue 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 logManager = server.getLogManager
val log = logManager.getOrCreateLog(topic, part) val log = logManager.getOrCreateLog(topic, part)
val message = new Message(Integer.toString(42).getBytes()) val message = new Message(Integer.toString(42).getBytes())
@ -172,11 +180,13 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
@Test @Test
def testGetOffsetsBeforeEarliestTime() { def testGetOffsetsBeforeEarliestTime() {
val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(10) val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(3)
val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition
val topic = topicPartition.split("-").head val topic = topicPartition.split("-").head
val part = Integer.valueOf(topicPartition.split("-").last).intValue 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 logManager = server.getLogManager
val log = logManager.getOrCreateLog(topic, part) val log = logManager.getOrCreateLog(topic, part)
val message = new Message(Integer.toString(42).getBytes()) 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.producer.async.MissingConfigException
import kafka.serializer.Encoder import kafka.serializer.Encoder
import kafka.server.{KafkaConfig, KafkaServer} import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.{TestUtils, TestZKUtils, Utils, Logging}
import kafka.zk.{EmbeddedZookeeper, ZooKeeperTestHarness} import kafka.zk.{EmbeddedZookeeper, ZooKeeperTestHarness}
import org.apache.log4j.spi.LoggingEvent import org.apache.log4j.spi.LoggingEvent
import org.apache.log4j.{PropertyConfigurator, Logger} import org.apache.log4j.{PropertyConfigurator, Logger}
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.utils._
class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
var logDirZk: File = null var logDirZk: File = null
var logDirBl: File = null var logDirBl: File = null
// var topicLogDir: File = null
var serverBl: KafkaServer = null var serverBl: KafkaServer = null
var serverZk: KafkaServer = null var serverZk: KafkaServer = null
@ -63,7 +62,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with
logDirZk = new File(logDirZkPath) logDirZk = new File(logDirZkPath)
serverZk = TestUtils.createServer(new KafkaConfig(propsZk)); 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") val logDirBlPath = propsBl.getProperty("log.dir")
logDirBl = new File(logDirBlPath) logDirBl = new File(logDirBlPath)
serverBl = TestUtils.createServer(new KafkaConfig(propsBl)) serverBl = TestUtils.createServer(new KafkaConfig(propsBl))
@ -85,8 +84,6 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with
Utils.rm(logDirBl) Utils.rm(logDirBl)
Thread.sleep(500) Thread.sleep(500)
// zkServer.shutdown
// Thread.sleep(500)
super.tearDown() 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","org.apache.log4j.PatternLayout")
props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") 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.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") props.put("log4j.logger.kafka.log4j", "INFO, KAFKA")
// topic missing // 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", "kafka.producer.KafkaLog4jAppender")
props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") 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.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.appender.KAFKA.Topic", "test-topic")
props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") 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 @Test
def testZkConnectLog4jAppends() { def testZkConnectLog4jAppends() {
PropertyConfigurator.configure(getLog4jConfigWithZkConnect) PropertyConfigurator.configure(getLog4jConfigWithZkConnect)
@ -208,18 +184,6 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with
assertEquals(5, count) 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 = { private def getLog4jConfigWithZkConnect: Properties = {
var props = new Properties() var props = new Properties()
props.put("log4j.rootLogger", "INFO") props.put("log4j.rootLogger", "INFO")
@ -232,21 +196,6 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with
props 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 = { private def getLogDir(): File = {
val dir = TestUtils.tempDir() val dir = TestUtils.tempDir()
dir dir

View File

@ -18,23 +18,41 @@
package kafka.producer package kafka.producer
import org.easymock.EasyMock import org.easymock.EasyMock
import kafka.api.ProducerRequest
import org.junit.Test import org.junit.Test
import org.scalatest.junit.JUnitSuite
import kafka.producer.async._ import kafka.producer.async._
import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.LinkedBlockingQueue
import junit.framework.Assert._ import junit.framework.Assert._
import collection.SortedSet import kafka.cluster.Broker
import kafka.cluster.{Broker, Partition} import collection.mutable.ListBuffer
import collection.mutable.{HashMap, ListBuffer}
import collection.Map import collection.Map
import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message}
import kafka.serializer.{StringEncoder, StringDecoder, Encoder} import kafka.serializer.{StringEncoder, StringDecoder, Encoder}
import java.util.{LinkedList, Properties} import java.util.{LinkedList, Properties}
import kafka.utils.{TestZKUtils, TestUtils}
import kafka.common.{InvalidConfigException, NoBrokersForPartitionException, InvalidPartitionException} 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 @Test
def testProducerQueueSize() { def testProducerQueueSize() {
@ -50,7 +68,7 @@ class AsyncProducerTest extends JUnitSuite {
val props = new Properties() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") 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("producer.type", "async")
props.put("queue.size", "10") props.put("queue.size", "10")
props.put("batch.size", "1") props.put("batch.size", "1")
@ -72,13 +90,13 @@ class AsyncProducerTest extends JUnitSuite {
def testProduceAfterClosed() { def testProduceAfterClosed() {
val props = new Properties() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") 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("producer.type", "async")
props.put("batch.size", "1") props.put("batch.size", "1")
val config = new ProducerConfig(props) val config = new ProducerConfig(props)
val produceData = getProduceData(10) val produceData = getProduceData(10)
val producer = new Producer[String, String](config) val producer = new Producer[String, String](config, zkClient)
producer.close producer.close
try { try {
@ -157,18 +175,35 @@ class AsyncProducerTest extends JUnitSuite {
producerDataList.append(new ProducerData[Int,Message]("topic2", 4, new Message("msg5".getBytes))) producerDataList.append(new ProducerData[Int,Message]("topic2", 4, new Message("msg5".getBytes)))
val props = new Properties() 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] { val intPartitioner = new Partitioner[Int] {
def partition(key: Int, numPartitions: Int): Int = key % numPartitions def partition(key: Int, numPartitions: Int): Int = key % numPartitions
} }
val config = new ProducerConfig(props) 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, val handler = new DefaultEventHandler[Int,String](config,
partitioner = intPartitioner, partitioner = intPartitioner,
encoder = null.asInstanceOf[Encoder[String]], encoder = null.asInstanceOf[Encoder[String]],
producerPool = null, producerPool)
populateProducerPool = false,
brokerPartitionInfo = null)
val topic1Broker1Data = new ListBuffer[ProducerData[Int,Message]] val topic1Broker1Data = new ListBuffer[ProducerData[Int,Message]]
topic1Broker1Data.appendAll(List(new ProducerData[Int,Message]("topic1", 0, new Message("msg1".getBytes)), 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)))) topic2Broker2Data.appendAll(List(new ProducerData[Int,Message]("topic2", 1, new Message("msg2".getBytes))))
val expectedResult = Map( val expectedResult = Map(
0 -> Map( 0 -> Map(
("topic1", -1) -> topic1Broker1Data, ("topic1", 0) -> topic1Broker1Data,
("topic2", -1) -> topic2Broker1Data), ("topic2", 0) -> topic2Broker1Data),
1 -> Map( 1 -> Map(
("topic1", -1) -> topic1Broker2Data, ("topic1", 1) -> topic1Broker2Data,
("topic2", -1) -> topic2Broker2Data) ("topic2", 1) -> topic2Broker2Data)
) )
val actualResult = handler.partitionAndCollate(producerDataList) val actualResult = handler.partitionAndCollate(producerDataList)
assertEquals(expectedResult, actualResult) assertEquals(expectedResult, actualResult)
EasyMock.verify(syncProducer)
EasyMock.verify(producerPool)
} }
@Test @Test
def testSerializeEvents() { def testSerializeEvents() {
val produceData = TestUtils.getMsgStrings(5).map(m => new ProducerData[String,String]("topic1",m)) val produceData = TestUtils.getMsgStrings(5).map(m => new ProducerData[String,String]("topic1",m))
val props = new Properties() val props = new Properties()
props.put("broker.list", "0:localhost:9092,1:localhost:9092") props.put("zk.connect", zkConnect)
val config = new ProducerConfig(props) 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, val handler = new DefaultEventHandler[String,String](config,
partitioner = null.asInstanceOf[Partitioner[String]], partitioner = null.asInstanceOf[Partitioner[String]],
encoder = new StringEncoder, encoder = new StringEncoder,
producerPool = null, producerPool)
populateProducerPool = false,
brokerPartitionInfo = null)
val serializedData = handler.serialize(produceData) val serializedData = handler.serialize(produceData)
val decoder = new StringDecoder val decoder = new StringDecoder
@ -216,14 +256,20 @@ class AsyncProducerTest extends JUnitSuite {
val producerDataList = new ListBuffer[ProducerData[String,Message]] val producerDataList = new ListBuffer[ProducerData[String,Message]]
producerDataList.append(new ProducerData[String,Message]("topic1", "key1", new Message("msg1".getBytes))) producerDataList.append(new ProducerData[String,Message]("topic1", "key1", new Message("msg1".getBytes)))
val props = new Properties() 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 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, val handler = new DefaultEventHandler[String,String](config,
partitioner = new NegativePartitioner, partitioner = new NegativePartitioner,
encoder = null.asInstanceOf[Encoder[String]], encoder = null.asInstanceOf[Encoder[String]],
producerPool = null, producerPool)
populateProducerPool = false,
brokerPartitionInfo = null)
try { try {
handler.partitionAndCollate(producerDataList) handler.partitionAndCollate(producerDataList)
fail("Should fail with InvalidPartitionException") fail("Should fail with InvalidPartitionException")
@ -231,34 +277,29 @@ class AsyncProducerTest extends JUnitSuite {
catch { catch {
case e: InvalidPartitionException => // expected, do nothing case e: InvalidPartitionException => // expected, do nothing
} }
} EasyMock.verify(syncProducer)
EasyMock.verify(producerPool)
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 = {}
}
} }
@Test @Test
def testNoBroker() { 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]] val producerDataList = new ListBuffer[ProducerData[String,String]]
producerDataList.append(new ProducerData[String,String]("topic1", "msg1")) producerDataList.append(new ProducerData[String,String]("topic1", "msg1"))
val props = new Properties()
val config = new ProducerConfig(props)
val handler = new DefaultEventHandler[String,String](config, val handler = new DefaultEventHandler[String,String](config,
partitioner = null.asInstanceOf[Partitioner[String]], partitioner = null.asInstanceOf[Partitioner[String]],
encoder = new StringEncoder, encoder = new StringEncoder,
producerPool = null, producerPool)
populateProducerPool = false,
brokerPartitionInfo = getMockBrokerPartitionInfo)
try { try {
handler.handle(producerDataList) handler.handle(producerDataList)
fail("Should fail with NoBrokersForPartitionException") fail("Should fail with NoBrokersForPartitionException")
@ -266,12 +307,14 @@ class AsyncProducerTest extends JUnitSuite {
catch { catch {
case e: NoBrokersForPartitionException => // expected, do nothing case e: NoBrokersForPartitionException => // expected, do nothing
} }
EasyMock.verify(syncProducer)
EasyMock.verify(producerPool)
} }
@Test @Test
def testIncompatibleEncoder() { def testIncompatibleEncoder() {
val props = new Properties() 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 config = new ProducerConfig(props)
val producer=new Producer[String, String](config) val producer=new Producer[String, String](config)
@ -286,14 +329,28 @@ class AsyncProducerTest extends JUnitSuite {
@Test @Test
def testRandomPartitioner() { def testRandomPartitioner() {
val props = new Properties() 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 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, val handler = new DefaultEventHandler[String,String](config,
partitioner = null.asInstanceOf[Partitioner[String]], partitioner = null.asInstanceOf[Partitioner[String]],
encoder = null.asInstanceOf[Encoder[String]], encoder = null.asInstanceOf[Encoder[String]],
producerPool = null, producerPool)
populateProducerPool = false,
brokerPartitionInfo = null)
val producerDataList = new ListBuffer[ProducerData[String,Message]] val producerDataList = new ListBuffer[ProducerData[String,Message]]
producerDataList.append(new ProducerData[String,Message]("topic1", new Message("msg1".getBytes))) producerDataList.append(new ProducerData[String,Message]("topic1", new Message("msg1".getBytes)))
producerDataList.append(new ProducerData[String,Message]("topic2", new Message("msg2".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) val partitionedData = handler.partitionAndCollate(producerDataList)
for ((brokerId, dataPerBroker) <- partitionedData) { for ((brokerId, dataPerBroker) <- partitionedData) {
for ( ((topic, partitionId), dataPerTopic) <- dataPerBroker) for ( ((topic, partitionId), dataPerTopic) <- dataPerBroker)
assertTrue(partitionId == ProducerRequest.RandomPartition) assertTrue(partitionId == 0)
} }
EasyMock.verify(producerPool)
} }
@Test @Test
def testBrokerListAndAsync() { 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() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") props.put("serializer.class", "kafka.serializer.StringEncoder")
props.put("producer.type", "async") props.put("producer.type", "async")
props.put("batch.size", "5") props.put("batch.size", "5")
props.put("broker.list", "0:localhost:9092") props.put("zk.connect", TestZKUtils.zookeeperConnect)
val config = new ProducerConfig(props) 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, val handler = new DefaultEventHandler[String,String](config,
partitioner = null.asInstanceOf[Partitioner[String]], partitioner = null.asInstanceOf[Partitioner[String]],
encoder = new StringEncoder, encoder = new StringEncoder,
producerPool = producerPool, producerPool = producerPool)
populateProducerPool = false,
brokerPartitionInfo = null)
val producer = new Producer[String, String](config, handler) val producer = new Producer[String, String](config, handler)
try { try {
@ -349,6 +416,7 @@ class AsyncProducerTest extends JUnitSuite {
} }
EasyMock.verify(mockSyncProducer) EasyMock.verify(mockSyncProducer)
EasyMock.verify(producerPool)
} }
@Test @Test
@ -380,7 +448,7 @@ class AsyncProducerTest extends JUnitSuite {
def testInvalidConfiguration() { def testInvalidConfiguration() {
val props = new Properties() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") 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("zk.connect", TestZKUtils.zookeeperConnect)
props.put("producer.type", "async") props.put("producer.type", "async")
@ -398,6 +466,34 @@ class AsyncProducerTest extends JUnitSuite {
new ByteBufferMessageSet(NoCompressionCodec, messages.map(m => encoder.toMessage(m)): _*) 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) { class MockProducer(override val config: SyncProducerConfig) extends SyncProducer(config) {
override def send(topic: String, messages: ByteBufferMessageSet): Unit = { override def send(topic: String, messages: ByteBufferMessageSet): Unit = {
Thread.sleep(1000) Thread.sleep(1000)

View File

@ -17,20 +17,21 @@
package kafka.producer package kafka.producer
import junit.framework.Assert._ import org.scalatest.junit.JUnit3Suite
import java.util.Properties import kafka.zk.ZooKeeperTestHarness
import kafka.api.FetchRequestBuilder
import kafka.consumer.SimpleConsumer 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.message.Message
import kafka.serializer.Encoder import kafka.admin.CreateTopicCommand
import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig} import kafka.api.FetchRequestBuilder
import kafka.utils.{TestUtils, TestZKUtils, Utils} import org.junit.Assert._
import kafka.zk.EmbeddedZookeeper
import org.apache.log4j.{Logger, Level}
import org.junit.{After, Before, Test}
import org.scalatest.junit.JUnitSuite
class ProducerTest extends JUnitSuite { class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness {
private val topic = "test-topic" private val topic = "test-topic"
private val brokerId1 = 0 private val brokerId1 = 0
private val brokerId2 = 1 private val brokerId2 = 1
@ -40,13 +41,13 @@ class ProducerTest extends JUnitSuite {
private var server2: KafkaServer = null private var server2: KafkaServer = null
private var consumer1: SimpleConsumer = null private var consumer1: SimpleConsumer = null
private var consumer2: SimpleConsumer = null private var consumer2: SimpleConsumer = null
private var zkServer:EmbeddedZookeeper = null
private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler])
private var zkClient: ZkClient = null
@Before override def setUp() {
def setUp() { super.setUp()
// set up 2 brokers with 4 partitions each // set up 2 brokers with 4 partitions each
zkServer = new EmbeddedZookeeper(TestZKUtils.zookeeperConnect) zkClient = zookeeper.client
val props1 = TestUtils.createBrokerConfig(brokerId1, port1) val props1 = TestUtils.createBrokerConfig(brokerId1, port1)
val config1 = new KafkaConfig(props1) { val config1 = new KafkaConfig(props1) {
@ -73,8 +74,7 @@ class ProducerTest extends JUnitSuite {
Thread.sleep(500) Thread.sleep(500)
} }
@After override def tearDown() {
def tearDown() {
// restore set request handler logger to a higher level // restore set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.ERROR) requestHandlerLogger.setLevel(Level.ERROR)
server1.shutdown server1.shutdown
@ -82,38 +82,43 @@ class ProducerTest extends JUnitSuite {
Utils.rm(server1.config.logDir) Utils.rm(server1.config.logDir)
Utils.rm(server2.config.logDir) Utils.rm(server2.config.logDir)
Thread.sleep(500) Thread.sleep(500)
zkServer.shutdown super.tearDown()
Thread.sleep(500)
} }
@Test @Test
def testZKSendToNewTopic() { def testZKSendToNewTopic() {
val props = new Properties() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") 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) props.put("zk.connect", TestZKUtils.zookeeperConnect)
val config = new ProducerConfig(props) val config = new ProducerConfig(props)
val producer = new Producer[String, String](config) val producer = new Producer[String, String](config)
try { try {
// Available broker id, partition id at this stage should be (0,0), (1,0) // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0, but
// this should send the message to broker 0 on partition 0 // 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"))) producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
Thread.sleep(100) 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"))) producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
Thread.sleep(100) Thread.sleep(1000)
// cross check if brokers got the messages // cross check if one of the brokers got the messages
val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val messageSet1 = response1.messageSet("new-topic", 0) val messageSet1 = response1.messageSet("new-topic", 0).iterator
assertTrue("Message set should have 1 message", messageSet1.iterator.hasNext)
assertEquals(new Message("test1".getBytes), messageSet1.head.message)
val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val messageSet2 = response2.messageSet("new-topic", 0) val messageSet2 = response2.messageSet("new-topic", 0).iterator
assertTrue("Message set should have 1 message", messageSet2.iterator.hasNext) assertTrue("Message set should have 1 message", (messageSet1.hasNext || messageSet2.hasNext))
assertEquals(new Message("test1".getBytes), messageSet2.head.message)
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 { } catch {
case e: Exception => fail("Not expected", e) case e: Exception => fail("Not expected", e)
} }
@ -124,34 +129,40 @@ class ProducerTest extends JUnitSuite {
def testZKSendWithDeadBroker() { def testZKSendWithDeadBroker() {
val props = new Properties() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") 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) 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 config = new ProducerConfig(props)
val producer = new Producer[String, String](config) val producer = new Producer[String, String](config)
try { try {
// Available broker id, partition id at this stage should be (0,0), (1,0) // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0 and
// Hence, this should send the message to broker 0 on partition 0 // all partitions have broker 0 as the leader.
producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1"))) producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
Thread.sleep(100) Thread.sleep(100)
// kill 2nd broker // kill 2nd broker
server2.shutdown server1.shutdown
Thread.sleep(100) 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, // Since all partitions are unavailable, this request will be dropped
// 4 % 4 = 0, So it should send the message to broker 0 on partition 0
producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1"))) producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
Thread.sleep(100) Thread.sleep(100)
// restart server 1
server1.startup()
Thread.sleep(100)
// cross check if brokers got the messages // cross check if brokers got the messages
val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val messageSet1Iter = response1.messageSet("new-topic", 0).iterator val messageSet1 = response1.messageSet("new-topic", 0).iterator
assertTrue("Message set should have 1 message", messageSet1Iter.hasNext) assertTrue("Message set should have 1 message", messageSet1.hasNext)
assertEquals(new Message("test1".getBytes), messageSet1Iter.next.message) assertEquals(new Message("test1".getBytes), messageSet1.next.message)
assertTrue("Message set should have another message", messageSet1Iter.hasNext) assertFalse("Message set should have another message", messageSet1.hasNext)
assertEquals(new Message("test1".getBytes), messageSet1Iter.next.message)
} catch { } catch {
case e: Exception => fail("Not expected") case e: Exception => fail("Not expected", e)
} }
producer.close producer.close
} }
@ -160,7 +171,7 @@ class ProducerTest extends JUnitSuite {
def testZKSendToExistingTopicWithNoBrokers() { def testZKSendToExistingTopicWithNoBrokers() {
val props = new Properties() val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder") 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) props.put("zk.connect", TestZKUtils.zookeeperConnect)
val config = new ProducerConfig(props) val config = new ProducerConfig(props)
@ -168,19 +179,19 @@ class ProducerTest extends JUnitSuite {
val producer = new Producer[String, String](config) val producer = new Producer[String, String](config)
var server: KafkaServer = null var server: KafkaServer = null
// create topic
CreateTopicCommand.createTopic(zkClient, "new-topic", 4, 2, "0:1,0:1,0:1,0:1")
try { try {
// shutdown server1 // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0 and
server1.shutdown // all partitions have broker 0 as the leader.
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
producer.send(new ProducerData[String, String]("new-topic", "test", Array("test"))) producer.send(new ProducerData[String, String]("new-topic", "test", Array("test")))
Thread.sleep(100) Thread.sleep(100)
// cross check if brokers got the messages // cross check if brokers got the messages
val response1 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val messageSet1 = response1.messageSet("new-topic", 0) val messageSet1 = response1.messageSet("new-topic", 0).iterator
assertTrue("Message set should have 1 message", messageSet1.iterator.hasNext) assertTrue("Message set should have 1 message", messageSet1.hasNext)
assertEquals(new Message("test".getBytes), messageSet1.head.message) assertEquals(new Message("test".getBytes), messageSet1.next.message)
// shutdown server2 // shutdown server2
server2.shutdown server2.shutdown
@ -189,7 +200,7 @@ class ProducerTest extends JUnitSuite {
Utils.rm(server2.config.logDir) Utils.rm(server2.config.logDir)
Thread.sleep(100) Thread.sleep(100)
// start it up again. So broker 2 exists under /broker/ids, but nothing exists under /broker/topics/new-topic // 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) { val config2 = new KafkaConfig(props2) {
override val numPartitions = 4 override val numPartitions = 4
} }
@ -202,9 +213,9 @@ class ProducerTest extends JUnitSuite {
// cross check if brokers got the messages // cross check if brokers got the messages
val response2 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) val response2 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val messageSet2 = response2.messageSet("new-topic", 0) val messageSet2 = response1.messageSet("new-topic", 0).iterator
assertTrue("Message set should have 1 message", messageSet2.iterator.hasNext) assertTrue("Message set should have 1 message", messageSet2.hasNext)
assertEquals(new Message("test".getBytes), messageSet2.head.message) assertEquals(new Message("test".getBytes), messageSet2.next.message)
} catch { } catch {
case e: Exception => fail("Not expected", e) case e: Exception => fail("Not expected", e)
@ -213,29 +224,5 @@ class ProducerTest extends JUnitSuite {
producer.close 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.server.KafkaConfig
import kafka.common.MessageSizeTooLargeException import kafka.common.MessageSizeTooLargeException
import java.util.Properties import java.util.Properties
import kafka.api.ProducerRequest
import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
import kafka.integration.KafkaServerTestHarness import kafka.integration.KafkaServerTestHarness
import kafka.utils.{TestZKUtils, SystemTime, TestUtils} import kafka.utils.{TestZKUtils, SystemTime, TestUtils}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.api.ProducerRequest
class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
private var messageBytes = new Array[Byte](2); private var messageBytes = new Array[Byte](2);

View File

@ -17,16 +17,17 @@
package kafka.server package kafka.server
import java.io.File import java.io.File
import kafka.producer.{SyncProducer, SyncProducerConfig}
import kafka.consumer.SimpleConsumer import kafka.consumer.SimpleConsumer
import java.util.Properties import java.util.Properties
import org.junit.Test import org.junit.Test
import junit.framework.Assert._ import junit.framework.Assert._
import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.message.{Message, ByteBufferMessageSet}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.zk.ZooKeeperTestHarness import kafka.zk.ZooKeeperTestHarness
import kafka.utils.{TestUtils, Utils} 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 { class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
val port = TestUtils.choosePort val port = TestUtils.choosePort
@ -38,26 +39,20 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
val host = "localhost" val host = "localhost"
val topic = "test" val topic = "test"
val sent1 = new ByteBufferMessageSet(NoCompressionCodec, new Message("hello".getBytes()), new Message("there".getBytes())) val sent1 = List(new Message("hello".getBytes()), new Message("there".getBytes()))
val sent2 = new ByteBufferMessageSet(NoCompressionCodec, new Message("more".getBytes()), new Message("messages".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) val server = new KafkaServer(config)
server.startup() 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 // send some messages
producer.send(topic, sent1) producer.send(new ProducerData[Int, Message](topic, 0, sent1))
sent1.getBuffer.rewind
Thread.sleep(200) Thread.sleep(200)
// do a clean shutdown // do a clean shutdown
@ -68,11 +63,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
{ {
val producer = new SyncProducer(getProducerConfig(host, val producer = new Producer[Int, Message](getProducerConfig(64*1024, 100000, 10000))
port,
64*1024,
100000,
10000))
val consumer = new SimpleConsumer(host, val consumer = new SimpleConsumer(host,
port, port,
1000000, 1000000,
@ -81,18 +72,19 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
val server = new KafkaServer(config) val server = new KafkaServer(config)
server.startup() server.startup()
// bring the server back again and read the messages Thread.sleep(100)
var fetchedMessage: ByteBufferMessageSet = null var fetchedMessage: ByteBufferMessageSet = null
while(fetchedMessage == null || fetchedMessage.validBytes == 0) { while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
fetchedMessage = fetched.messageSet(topic, 0) 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 val newOffset = fetchedMessage.validBytes
// send some more messages // send some more messages
producer.send(topic, sent2) println("Sending messages to topic " + topic)
sent2.getBuffer.rewind producer.send(new ProducerData[Int, Message](topic, 0, sent2))
Thread.sleep(200) 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()) val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, newOffset, 10000).build())
fetchedMessage = fetched.messageSet(topic, 0) 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() server.shutdown()
Utils.rm(server.config.logDir) 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, private def getProducerConfig(bufferSize: Int, connectTimeout: Int,
reconnectInterval: Int): SyncProducerConfig = { reconnectInterval: Int): ProducerConfig = {
val props = new Properties() val props = new Properties()
props.put("host", host) props.put("zk.connect", zkConnect)
props.put("port", port.toString) props.put("partitioner.class", "kafka.utils.FixedValuePartitioner")
props.put("buffer.size", bufferSize.toString) props.put("buffer.size", bufferSize.toString)
props.put("connect.timeout.ms", connectTimeout.toString) props.put("connect.timeout.ms", connectTimeout.toString)
props.put("reconnect.interval", reconnectInterval.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 collection.mutable.ListBuffer
import kafka.consumer.{KafkaMessageStream, ConsumerConfig} import kafka.consumer.{KafkaMessageStream, ConsumerConfig}
import scala.collection.Map import scala.collection.Map
import kafka.serializer.Encoder
/** /**
* Utility functions to help with testing * Utility functions to help with testing
@ -134,6 +135,7 @@ object TestUtils {
props.put("zk.sessiontimeout.ms", "400") props.put("zk.sessiontimeout.ms", "400")
props.put("zk.synctime.ms", "200") props.put("zk.synctime.ms", "200")
props.put("autocommit.interval.ms", "1000") props.put("autocommit.interval.ms", "1000")
props.put("rebalance.retries.max", "4")
props props
} }
@ -275,14 +277,13 @@ object TestUtils {
/** /**
* Create a producer for the given host and port * 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() val props = new Properties()
props.put("host", host) props.put("zk.connect", zkConnect)
props.put("port", port.toString)
props.put("buffer.size", "65536") props.put("buffer.size", "65536")
props.put("connect.timeout.ms", "100000") props.put("connect.timeout.ms", "100000")
props.put("reconnect.interval", "10000") 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) = { def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = {
@ -308,6 +309,12 @@ object TestUtils {
brokers 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]= { def getConsumedMessages[T](nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[T]]]): List[T]= {
var messages: List[T] = Nil var messages: List[T] = Nil
for ((topic, messageStreams) <- topicMessageStreams) { for ((topic, messageStreams) <- topicMessageStreams) {
@ -335,3 +342,31 @@ object TestUtils {
object TestZKUtils { object TestZKUtils {
val zookeeperConnect = "127.0.0.1:2182" 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 num_msg_per_batch=500
readonly batches_per_iteration=5 readonly batches_per_iteration=5
readonly num_iterations=12 readonly num_iterations=5
readonly zk_source_port=2181 readonly zk_source_port=2181
readonly zk_mirror_port=2182 readonly zk_mirror_port=2182