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