kafka-965; merge c39d37e9dd from 0.8 to trunk; patched by Jun Rao; reviewed by Jay Kreps

This commit is contained in:
Jun Rao 2013-07-10 23:36:51 -07:00
commit db37ed0054
87 changed files with 1574 additions and 560 deletions

View File

@ -65,3 +65,19 @@ else
fi
$JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH "$@"
exitval=$?
if [ $exitval -eq "1" ] ; then
$JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH "$@" >& exception.txt
exception=`cat exception.txt`
noBuildMessage='Please build the project using sbt. Documentation is available at http://kafka.apache.org/'
pattern="(Could not find or load main class)|(java\.lang\.NoClassDefFoundError)"
match=`echo $exception | grep -E "$pattern"`
if [[ -n "$match" ]]; then
echo $noBuildMessage
fi
rm exception.txt
fi

View File

@ -52,7 +52,7 @@ log.dir=/tmp/kafka-logs
# The number of logical partitions per topic per server. More partitions allow greater parallelism
# for consumption, but also mean more files.
num.partitions=1
num.partitions=2
############################# Log Flush Policy #############################

View File

@ -49,9 +49,9 @@ public class KafkaOutputFormat<K, V> extends OutputFormat<K, V>
Map<String, String> cMap = new HashMap<String, String>();
// default Hadoop producer configs
cMap.put("producer.type", "sync");
cMap.put("send.buffer.bytes", Integer.toString(64*1024));
cMap.put("compression.codec", Integer.toString(1));
cMap.put("producer.type", "sync");
cMap.put("compression.codec", Integer.toString(1));
cMap.put("request.required.acks", Integer.toString(1));
kafkaConfigMap = Collections.unmodifiableMap(cMap);
}

View File

@ -12,7 +12,7 @@ libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _ )
libraryDependencies ++= Seq(
"org.apache.zookeeper" % "zookeeper" % "3.3.4",
"com.101tec" % "zkclient" % "0.2",
"com.101tec" % "zkclient" % "0.3",
"org.xerial.snappy" % "snappy-java" % "1.0.4.1",
"com.yammer.metrics" % "metrics-core" % "2.2.0",
"com.yammer.metrics" % "metrics-annotation" % "2.2.0",

View File

@ -21,17 +21,14 @@ import java.util.Random
import java.util.Properties
import kafka.api.{TopicMetadata, PartitionMetadata}
import kafka.cluster.Broker
import kafka.utils.{Logging, ZkUtils}
import kafka.log.LogConfig
import kafka.server.TopicConfigManager
import kafka.utils.{Logging, Utils, ZkUtils, Json}
import kafka.utils.{Logging, ZkUtils, Json}
import org.I0Itec.zkclient.ZkClient
import org.I0Itec.zkclient.exception.ZkNodeExistsException
import scala.collection._
import mutable.ListBuffer
import scala.collection.mutable
import kafka.common._
import scala.Some
object AdminUtils extends Logging {
val rand = new Random

View File

@ -1,4 +1,19 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.admin
import joptsimple.OptionParser

View File

@ -25,7 +25,7 @@ import javax.management.remote.{JMXServiceURL, JMXConnectorFactory}
import javax.management.ObjectName
import kafka.controller.KafkaController
import scala.Some
import kafka.common.BrokerNotAvailableException
import kafka.common.{TopicAndPartition, BrokerNotAvailableException}
object ShutdownBroker extends Logging {
@ -58,8 +58,8 @@ object ShutdownBroker extends Logging {
val leaderPartitionsRemaining = mbsc.invoke(new ObjectName(KafkaController.MBeanName),
"shutdownBroker",
Array(params.brokerId),
Array(classOf[Int].getName)).asInstanceOf[Int]
val shutdownComplete = (leaderPartitionsRemaining == 0)
Array(classOf[Int].getName)).asInstanceOf[Set[TopicAndPartition]]
val shutdownComplete = (leaderPartitionsRemaining.size == 0)
info("Shutdown status: " +
(if (shutdownComplete) "complete" else "incomplete (broker still leads %d partitions)".format(leaderPartitionsRemaining)))
shutdownComplete

View File

@ -1,4 +1,20 @@
package kafka.api
/**
* 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.api
import java.nio._
import kafka.common._

View File

@ -0,0 +1,73 @@
/**
* 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.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import collection.mutable.ListBuffer
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common.{TopicAndPartition, ErrorMapping}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
object ControlledShutdownRequest extends Logging {
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
def readFrom(buffer: ByteBuffer): ControlledShutdownRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val brokerId = buffer.getInt
new ControlledShutdownRequest(versionId, correlationId, brokerId)
}
}
case class ControlledShutdownRequest(val versionId: Short,
override val correlationId: Int,
val brokerId: Int)
extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey), correlationId){
def this(correlationId: Int, brokerId: Int) =
this(ControlledShutdownRequest.CurrentVersion, correlationId, brokerId)
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
buffer.putInt(brokerId)
}
def sizeInBytes(): Int = {
2 + /* version id */
4 + /* correlation id */
4 /* broker id */
}
override def toString(): String = {
val controlledShutdownRequest = new StringBuilder
controlledShutdownRequest.append("Name: " + this.getClass.getSimpleName)
controlledShutdownRequest.append("; Version: " + versionId)
controlledShutdownRequest.append("; CorrelationId: " + correlationId)
controlledShutdownRequest.append("; BrokerId: " + brokerId)
controlledShutdownRequest.toString()
}
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
val errorResponse = ControlledShutdownResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]), Set.empty[TopicAndPartition])
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
}
}

View File

@ -0,0 +1,70 @@
/**
* 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.api
import java.nio.ByteBuffer
import collection.mutable.HashMap
import collection.immutable.Map
import kafka.common.{TopicAndPartition, ErrorMapping}
import kafka.api.ApiUtils._
object ControlledShutdownResponse {
def readFrom(buffer: ByteBuffer): ControlledShutdownResponse = {
val correlationId = buffer.getInt
val errorCode = buffer.getShort
val numEntries = buffer.getInt
var partitionsRemaining = Set[TopicAndPartition]()
for (i<- 0 until numEntries){
val topic = readShortString(buffer)
val partition = buffer.getInt
partitionsRemaining += new TopicAndPartition(topic, partition)
}
new ControlledShutdownResponse(correlationId, errorCode, partitionsRemaining)
}
}
case class ControlledShutdownResponse(override val correlationId: Int,
val errorCode: Short = ErrorMapping.NoError,
val partitionsRemaining: Set[TopicAndPartition])
extends RequestOrResponse(correlationId = correlationId) {
def sizeInBytes(): Int ={
var size =
4 /* correlation id */ +
2 /* error code */ +
4 /* number of responses */
for (topicAndPartition <- partitionsRemaining) {
size +=
2 + topicAndPartition.topic.length /* topic */ +
4 /* partition */
}
size
}
def writeTo(buffer: ByteBuffer) {
buffer.putInt(correlationId)
buffer.putShort(errorCode)
buffer.putInt(partitionsRemaining.size)
for (topicAndPartition:TopicAndPartition <- partitionsRemaining){
writeShortString(buffer, topicAndPartition.topic)
buffer.putInt(topicAndPartition.partition)
}
}
}

View File

@ -50,23 +50,29 @@ object PartitionStateInfo {
val controllerEpoch = buffer.getInt
val leader = buffer.getInt
val leaderEpoch = buffer.getInt
val isrString = readShortString(buffer)
val isr = Utils.parseCsvList(isrString).map(_.toInt).toList
val isrSize = buffer.getInt
val isr = for(i <- 0 until isrSize) yield buffer.getInt
val zkVersion = buffer.getInt
val replicationFactor = buffer.getInt
PartitionStateInfo(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, leaderEpoch, isr, zkVersion), controllerEpoch),
replicationFactor)
val replicas = for(i <- 0 until replicationFactor) yield buffer.getInt
PartitionStateInfo(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, leaderEpoch, isr.toList, zkVersion), controllerEpoch),
replicas.toSet)
}
}
case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, val replicationFactor: Int) {
case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch,
val allReplicas: Set[Int]) {
def replicationFactor = allReplicas.size
def writeTo(buffer: ByteBuffer) {
buffer.putInt(leaderIsrAndControllerEpoch.controllerEpoch)
buffer.putInt(leaderIsrAndControllerEpoch.leaderAndIsr.leader)
buffer.putInt(leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch)
writeShortString(buffer, leaderIsrAndControllerEpoch.leaderAndIsr.isr.mkString(","))
buffer.putInt(leaderIsrAndControllerEpoch.leaderAndIsr.isr.size)
leaderIsrAndControllerEpoch.leaderAndIsr.isr.foreach(buffer.putInt(_))
buffer.putInt(leaderIsrAndControllerEpoch.leaderAndIsr.zkVersion)
buffer.putInt(replicationFactor)
allReplicas.foreach(buffer.putInt(_))
}
def sizeInBytes(): Int = {
@ -74,9 +80,11 @@ case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndContr
4 /* epoch of the controller that elected the leader */ +
4 /* leader broker id */ +
4 /* leader epoch */ +
(2 + leaderIsrAndControllerEpoch.leaderAndIsr.isr.mkString(",").length) +
4 /* number of replicas in isr */ +
4 * leaderIsrAndControllerEpoch.leaderAndIsr.isr.size /* replicas in isr */ +
4 /* zk version */ +
4 /* replication factor */
4 /* replication factor */ +
allReplicas.size * 4
size
}
@ -84,6 +92,7 @@ case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndContr
val partitionStateInfo = new StringBuilder
partitionStateInfo.append("(LeaderAndIsrInfo:" + leaderIsrAndControllerEpoch.toString)
partitionStateInfo.append(",ReplicationFactor:" + replicationFactor + ")")
partitionStateInfo.append(",AllReplicas:" + allReplicas.mkString(",") + ")")
partitionStateInfo.toString()
}
}
@ -98,7 +107,6 @@ object LeaderAndIsrRequest {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val ackTimeoutMs = buffer.getInt
val controllerId = buffer.getInt
val controllerEpoch = buffer.getInt
val partitionStateInfosCount = buffer.getInt
@ -117,14 +125,13 @@ object LeaderAndIsrRequest {
for (i <- 0 until leadersCount)
leaders += Broker.readFrom(buffer)
new LeaderAndIsrRequest(versionId, correlationId, clientId, ackTimeoutMs, controllerId, controllerEpoch, partitionStateInfos.toMap, leaders)
new LeaderAndIsrRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos.toMap, leaders)
}
}
case class LeaderAndIsrRequest (versionId: Short,
override val correlationId: Int,
clientId: String,
ackTimeoutMs: Int,
controllerId: Int,
controllerEpoch: Int,
partitionStateInfos: Map[(String, Int), PartitionStateInfo],
@ -133,7 +140,7 @@ case class LeaderAndIsrRequest (versionId: Short,
def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], aliveLeaders: Set[Broker], controllerId: Int,
controllerEpoch: Int, correlationId: Int, clientId: String) = {
this(LeaderAndIsrRequest.CurrentVersion, correlationId, clientId, LeaderAndIsrRequest.DefaultAckTimeout,
this(LeaderAndIsrRequest.CurrentVersion, correlationId, clientId,
controllerId, controllerEpoch, partitionStateInfos, aliveLeaders)
}
@ -141,7 +148,6 @@ case class LeaderAndIsrRequest (versionId: Short,
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(ackTimeoutMs)
buffer.putInt(controllerId)
buffer.putInt(controllerEpoch)
buffer.putInt(partitionStateInfos.size)
@ -159,7 +165,6 @@ case class LeaderAndIsrRequest (versionId: Short,
2 /* version id */ +
4 /* correlation id */ +
(2 + clientId.length) /* client id */ +
4 /* ack timeout */ +
4 /* controller id */ +
4 /* controller epoch */ +
4 /* number of partitions */
@ -179,7 +184,6 @@ case class LeaderAndIsrRequest (versionId: Short,
leaderAndIsrRequest.append(";ControllerEpoch:" + controllerEpoch)
leaderAndIsrRequest.append(";CorrelationId:" + correlationId)
leaderAndIsrRequest.append(";ClientId:" + clientId)
leaderAndIsrRequest.append(";AckTimeoutMs:" + ackTimeoutMs + " ms")
leaderAndIsrRequest.append(";PartitionState:" + partitionStateInfos.mkString(","))
leaderAndIsrRequest.append(";Leaders:" + aliveLeaders.mkString(","))
leaderAndIsrRequest.toString()

View File

@ -63,7 +63,7 @@ case class LeaderAndIsrResponse(override val correlationId: Int,
buffer.putInt(correlationId)
buffer.putShort(errorCode)
buffer.putInt(responseMap.size)
for ((key:(String, Int), value) <- responseMap){
for ((key:(String, Int), value) <- responseMap) {
writeShortString(buffer, key._1)
buffer.putInt(key._2)
buffer.putShort(value)

View File

@ -49,7 +49,7 @@ object OffsetRequest {
(TopicAndPartition(topic, partitionId), PartitionOffsetRequestInfo(time, maxNumOffsets))
})
})
OffsetRequest(Map(pairs:_*), versionId = versionId, clientId = clientId, replicaId = replicaId)
OffsetRequest(Map(pairs:_*), versionId= versionId, clientId = clientId, correlationId = correlationId, replicaId = replicaId)
}
}

View File

@ -27,8 +27,10 @@ object RequestKeys {
val MetadataKey: Short = 3
val LeaderAndIsrKey: Short = 4
val StopReplicaKey: Short = 5
val OffsetCommitKey: Short = 6
val OffsetFetchKey: Short = 7
val UpdateMetadataKey: Short = 6
val ControlledShutdownKey: Short = 7
val OffsetCommitKey: Short = 8
val OffsetFetchKey: Short = 9
val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]=
Map(ProduceKey -> ("Produce", ProducerRequest.readFrom),
@ -37,6 +39,8 @@ object RequestKeys {
MetadataKey -> ("Metadata", TopicMetadataRequest.readFrom),
LeaderAndIsrKey -> ("LeaderAndIsr", LeaderAndIsrRequest.readFrom),
StopReplicaKey -> ("StopReplica", StopReplicaRequest.readFrom),
UpdateMetadataKey -> ("UpdateMetadata", UpdateMetadataRequest.readFrom),
ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom),
OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom),
OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom))

View File

@ -35,7 +35,6 @@ object StopReplicaRequest extends Logging {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val ackTimeoutMs = buffer.getInt
val controllerId = buffer.getInt
val controllerEpoch = buffer.getInt
val deletePartitions = buffer.get match {
@ -49,7 +48,7 @@ object StopReplicaRequest extends Logging {
(1 to topicPartitionPairCount) foreach { _ =>
topicPartitionPairSet.add(readShortString(buffer), buffer.getInt)
}
StopReplicaRequest(versionId, correlationId, clientId, ackTimeoutMs, controllerId, controllerEpoch,
StopReplicaRequest(versionId, correlationId, clientId, controllerId, controllerEpoch,
deletePartitions, topicPartitionPairSet.toSet)
}
}
@ -57,7 +56,6 @@ object StopReplicaRequest extends Logging {
case class StopReplicaRequest(versionId: Short,
override val correlationId: Int,
clientId: String,
ackTimeoutMs: Int,
controllerId: Int,
controllerEpoch: Int,
deletePartitions: Boolean,
@ -65,7 +63,7 @@ case class StopReplicaRequest(versionId: Short,
extends RequestOrResponse(Some(RequestKeys.StopReplicaKey), correlationId) {
def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerId: Int, controllerEpoch: Int, correlationId: Int) = {
this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId, StopReplicaRequest.DefaultAckTimeout,
this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId,
controllerId, controllerEpoch, deletePartitions, partitions)
}
@ -73,7 +71,6 @@ case class StopReplicaRequest(versionId: Short,
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(ackTimeoutMs)
buffer.putInt(controllerId)
buffer.putInt(controllerEpoch)
buffer.put(if (deletePartitions) 1.toByte else 0.toByte)
@ -89,7 +86,6 @@ case class StopReplicaRequest(versionId: Short,
2 + /* versionId */
4 + /* correlation id */
ApiUtils.shortStringLength(clientId) +
4 + /* ackTimeoutMs */
4 + /* controller id*/
4 + /* controller epoch */
1 + /* deletePartitions */
@ -107,7 +103,6 @@ case class StopReplicaRequest(versionId: Short,
stopReplicaRequest.append("; Version: " + versionId)
stopReplicaRequest.append("; CorrelationId: " + correlationId)
stopReplicaRequest.append("; ClientId: " + clientId)
stopReplicaRequest.append("; AckTimeoutMs: " + ackTimeoutMs + " ms")
stopReplicaRequest.append("; DeletePartitions: " + deletePartitions)
stopReplicaRequest.append("; ControllerId: " + controllerId)
stopReplicaRequest.append("; ControllerEpoch: " + controllerEpoch)

View File

@ -55,6 +55,34 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat
buffer.putInt(partitionsMetadata.size)
partitionsMetadata.foreach(m => m.writeTo(buffer))
}
override def toString(): String = {
val topicMetadataInfo = new StringBuilder
topicMetadataInfo.append("{TopicMetadata for topic %s -> ".format(topic))
errorCode match {
case ErrorMapping.NoError =>
partitionsMetadata.foreach { partitionMetadata =>
partitionMetadata.errorCode match {
case ErrorMapping.NoError =>
topicMetadataInfo.append("\nMetadata for partition [%s,%d] is %s".format(topic,
partitionMetadata.partitionId, partitionMetadata.toString()))
case ErrorMapping.ReplicaNotAvailableCode =>
// this error message means some replica other than the leader is not available. The consumer
// doesn't care about non leader replicas, so ignore this
topicMetadataInfo.append("\nMetadata for partition [%s,%d] is %s".format(topic,
partitionMetadata.partitionId, partitionMetadata.toString()))
case _ =>
topicMetadataInfo.append("\nMetadata for partition [%s,%d] is not available due to %s".format(topic,
partitionMetadata.partitionId, ErrorMapping.exceptionFor(partitionMetadata.errorCode).getClass.getName))
}
}
case _ =>
topicMetadataInfo.append("\nNo partition metadata for topic %s due to %s".format(topic,
ErrorMapping.exceptionFor(errorCode).getClass.getName))
}
topicMetadataInfo.append("}")
topicMetadataInfo.toString()
}
}
object PartitionMetadata {

View File

@ -81,7 +81,7 @@ case class TopicMetadataRequest(val versionId: Short,
topicMetadataRequest.toString()
}
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
val topicMetadata = topics.map {
topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
}

View File

@ -0,0 +1,120 @@
/**
* 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.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.cluster.Broker
import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.network.RequestChannel.Response
object UpdateMetadataRequest {
val CurrentVersion = 0.shortValue
val IsInit: Boolean = true
val NotInit: Boolean = false
val DefaultAckTimeout: Int = 1000
def readFrom(buffer: ByteBuffer): UpdateMetadataRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val controllerId = buffer.getInt
val controllerEpoch = buffer.getInt
val partitionStateInfosCount = buffer.getInt
val partitionStateInfos = new collection.mutable.HashMap[TopicAndPartition, PartitionStateInfo]
for(i <- 0 until partitionStateInfosCount){
val topic = readShortString(buffer)
val partition = buffer.getInt
val partitionStateInfo = PartitionStateInfo.readFrom(buffer)
partitionStateInfos.put(TopicAndPartition(topic, partition), partitionStateInfo)
}
val numAliveBrokers = buffer.getInt
val aliveBrokers = for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer)
new UpdateMetadataRequest(versionId, correlationId, clientId, controllerId, controllerEpoch,
partitionStateInfos.toMap, aliveBrokers.toSet)
}
}
case class UpdateMetadataRequest (versionId: Short,
override val correlationId: Int,
clientId: String,
controllerId: Int,
controllerEpoch: Int,
partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo],
aliveBrokers: Set[Broker])
extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey), correlationId) {
def this(controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String,
partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) = {
this(UpdateMetadataRequest.CurrentVersion, correlationId, clientId,
controllerId, controllerEpoch, partitionStateInfos, aliveBrokers)
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(controllerId)
buffer.putInt(controllerEpoch)
buffer.putInt(partitionStateInfos.size)
for((key, value) <- partitionStateInfos){
writeShortString(buffer, key.topic)
buffer.putInt(key.partition)
value.writeTo(buffer)
}
buffer.putInt(aliveBrokers.size)
aliveBrokers.foreach(_.writeTo(buffer))
}
def sizeInBytes(): Int = {
var size =
2 /* version id */ +
4 /* correlation id */ +
(2 + clientId.length) /* client id */ +
4 /* controller id */ +
4 /* controller epoch */ +
4 /* number of partitions */
for((key, value) <- partitionStateInfos)
size += (2 + key.topic.length) /* topic */ + 4 /* partition */ + value.sizeInBytes /* partition state info */
size += 4 /* number of alive brokers in the cluster */
for(broker <- aliveBrokers)
size += broker.sizeInBytes /* broker info */
size
}
override def toString(): String = {
val updateMetadataRequest = new StringBuilder
updateMetadataRequest.append("Name:" + this.getClass.getSimpleName)
updateMetadataRequest.append(";Version:" + versionId)
updateMetadataRequest.append(";Controller:" + controllerId)
updateMetadataRequest.append(";ControllerEpoch:" + controllerEpoch)
updateMetadataRequest.append(";CorrelationId:" + correlationId)
updateMetadataRequest.append(";ClientId:" + clientId)
updateMetadataRequest.append(";PartitionState:" + partitionStateInfos.mkString(","))
updateMetadataRequest.append(";AliveBrokers:" + aliveBrokers.mkString(","))
updateMetadataRequest.toString()
}
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
val errorResponse = new UpdateMetadataResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]))
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
}
}

View File

@ -0,0 +1,44 @@
/**
* 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.api
import kafka.common.{TopicAndPartition, ErrorMapping}
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import collection.mutable.HashMap
import collection.Map
object UpdateMetadataResponse {
def readFrom(buffer: ByteBuffer): UpdateMetadataResponse = {
val correlationId = buffer.getInt
val errorCode = buffer.getShort
new UpdateMetadataResponse(correlationId, errorCode)
}
}
case class UpdateMetadataResponse(override val correlationId: Int,
errorCode: Short = ErrorMapping.NoError)
extends RequestOrResponse(correlationId = correlationId) {
def sizeInBytes(): Int = 4 /* correlation id */ + 2 /* error code */
def writeTo(buffer: ByteBuffer) {
buffer.putInt(correlationId)
buffer.putShort(errorCode)
}
}

View File

@ -1,4 +1,20 @@
package kafka.client
/**
* 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.client
import scala.collection._
import kafka.cluster._
@ -47,7 +63,7 @@ object ClientUtils extends Logging{
producer.close()
}
}
if(!fetchMetaDataSucceeded){
if(!fetchMetaDataSucceeded) {
throw new KafkaException("fetching topic metadata for topics [%s] from broker [%s] failed".format(topics, shuffledBrokers), t)
} else {
debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics))
@ -62,13 +78,14 @@ object ClientUtils extends Logging{
* @param clientId The client's identifier
* @return topic metadata response
*/
def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int): TopicMetadataResponse = {
def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int,
correlationId: Int = 0): TopicMetadataResponse = {
val props = new Properties()
props.put("metadata.broker.list", brokers.map(_.getConnectionString()).mkString(","))
props.put("client.id", clientId)
props.put("request.timeout.ms", timeoutMs.toString)
val producerConfig = new ProducerConfig(props)
fetchTopicMetadata(topics, brokers, producerConfig, 0)
fetchTopicMetadata(topics, brokers, producerConfig, correlationId)
}
/**

View File

@ -201,8 +201,15 @@ class Partition(val topic: String,
leaderEpoch = leaderAndIsr.leaderEpoch
zkVersion = leaderAndIsr.zkVersion
leaderReplicaIdOpt = Some(newLeaderBrokerId)
// start fetcher thread to current leader
replicaFetcherManager.addFetcher(topic, partitionId, localReplica.logEndOffset, leaderBroker)
if (!replicaManager.isShuttingDown.get()) {
// start fetcher thread to current leader if we are not shutting down
replicaFetcherManager.addFetcher(topic, partitionId, localReplica.logEndOffset, leaderBroker)
}
else {
stateChangeLogger.trace("Broker %d ignored the become-follower state change with correlation id %d from " +
" controller %d epoch %d since it is shutting down"
.format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch))
}
case None => // leader went down
stateChangeLogger.trace("Broker %d aborted the become-follower state change with correlation id %d from " +
" controller %d epoch %d since leader %d for partition [%s,%d] is unavailable during the state change operation"

View File

@ -17,9 +17,6 @@
package kafka.common
/**
* Thrown when a request is made for a topic, that hasn't been created in a Kafka cluster
*/
class UnknownTopicException(message: String) extends RuntimeException(message) {
class InvalidOffsetException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -84,6 +84,11 @@ object ConsoleConsumer extends Logging {
.describedAs("ms")
.ofType(classOf[java.lang.Integer])
.defaultsTo(ConsumerConfig.SocketTimeout)
val refreshMetadataBackoffMsOpt = parser.accepts("refresh-leader-backoff-ms", "Backoff time before refreshing metadata")
.withRequiredArg
.describedAs("ms")
.ofType(classOf[java.lang.Integer])
.defaultsTo(ConsumerConfig.RefreshMetadataBackoffMs)
val consumerTimeoutMsOpt = parser.accepts("consumer-timeout-ms", "consumer throws timeout exception after waiting this much " +
"of time without incoming messages")
.withRequiredArg
@ -105,7 +110,7 @@ object ConsoleConsumer extends Logging {
.withRequiredArg
.describedAs("ms")
.ofType(classOf[java.lang.Integer])
.defaultsTo(10*1000)
.defaultsTo(ConsumerConfig.AutoCommitInterval)
val maxMessagesOpt = parser.accepts("max-messages", "The maximum number of messages to consume before exiting. If not set, consumption is continual.")
.withRequiredArg
.describedAs("num_messages")
@ -160,6 +165,8 @@ object ConsoleConsumer extends Logging {
props.put("auto.offset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest")
props.put("zookeeper.connect", options.valueOf(zkConnectOpt))
props.put("consumer.timeout.ms", options.valueOf(consumerTimeoutMsOpt).toString)
props.put("refresh.leader.backoff.ms", options.valueOf(refreshMetadataBackoffMsOpt).toString)
val config = new ConsumerConfig(props)
val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false

View File

@ -23,6 +23,7 @@ import kafka.utils._
import kafka.common.{InvalidConfigException, Config}
object ConsumerConfig extends Config {
val RefreshMetadataBackoffMs = 200
val SocketTimeout = 30 * 1000
val SocketBufferSize = 64*1024
val FetchSize = 1024 * 1024
@ -30,9 +31,9 @@ object ConsumerConfig extends Config {
val DefaultFetcherBackoffMs = 1000
val AutoCommit = true
val AutoCommitInterval = 60 * 1000
val MaxQueuedChunks = 10
val MaxQueuedChunks = 2
val MaxRebalanceRetries = 4
val AutoOffsetReset = OffsetRequest.SmallestTimeString
val AutoOffsetReset = OffsetRequest.LargestTimeString
val ConsumerTimeoutMs = -1
val MinFetchBytes = 1
val MaxFetchWaitMs = 100
@ -99,8 +100,8 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
/** the frequency in ms that the consumer offsets are committed to zookeeper */
val autoCommitIntervalMs = props.getInt("auto.commit.interval.ms", AutoCommitInterval)
/** max number of messages buffered for consumption */
val queuedMaxMessages = props.getInt("queued.max.messages", MaxQueuedChunks)
/** max number of message chunks buffered for consumption, each chunk can be up to fetch.message.max.bytes*/
val queuedMaxMessages = props.getInt("queued.max.message.chunks", MaxQueuedChunks)
/** max number of retries during rebalance */
val rebalanceMaxRetries = props.getInt("rebalance.max.retries", MaxRebalanceRetries)
@ -115,7 +116,7 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
val rebalanceBackoffMs = props.getInt("rebalance.backoff.ms", zkSyncTimeMs)
/** backoff time to refresh the leader of a partition after it loses the current leader */
val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", 200)
val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", RefreshMetadataBackoffMs)
/* what to do if an offset is out of range.
smallest : automatically reset the offset to the smallest offset

View File

@ -28,6 +28,7 @@ import kafka.utils.ZkUtils._
import kafka.utils.{ShutdownableThread, SystemTime}
import kafka.common.TopicAndPartition
import kafka.client.ClientUtils
import java.util.concurrent.atomic.AtomicInteger
/**
* Usage:
@ -37,17 +38,20 @@ import kafka.client.ClientUtils
class ConsumerFetcherManager(private val consumerIdString: String,
private val config: ConsumerConfig,
private val zkClient : ZkClient)
extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(SystemTime.milliseconds), 1) {
extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(SystemTime.milliseconds),
config.groupId, 1) {
private var partitionMap: immutable.Map[TopicAndPartition, PartitionTopicInfo] = null
private var cluster: Cluster = null
private val noLeaderPartitionSet = new mutable.HashSet[TopicAndPartition]
private val lock = new ReentrantLock
private val cond = lock.newCondition()
private var leaderFinderThread: ShutdownableThread = null
private val correlationId = new AtomicInteger(0)
private class LeaderFinderThread(name: String) extends ShutdownableThread(name) {
// thread responsible for adding the fetcher to the right broker when leader is available
override def doWork() {
val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker]
lock.lock()
try {
if (noLeaderPartitionSet.isEmpty) {
@ -55,45 +59,43 @@ class ConsumerFetcherManager(private val consumerIdString: String,
cond.await()
}
try {
trace("Partitions without leader %s".format(noLeaderPartitionSet))
val brokers = getAllBrokersInCluster(zkClient)
val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet,
brokers,
config.clientId,
config.socketTimeoutMs).topicsMetadata
val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker]
topicsMetadata.foreach(
tmd => {
val topic = tmd.topic
tmd.partitionsMetadata.foreach(
pmd => {
val topicAndPartition = TopicAndPartition(topic, pmd.partitionId)
if(pmd.leader.isDefined && noLeaderPartitionSet.contains(topicAndPartition)) {
val leaderBroker = pmd.leader.get
leaderForPartitionsMap.put(topicAndPartition, leaderBroker)
}
})
})
leaderForPartitionsMap.foreach{
case(topicAndPartition, leaderBroker) =>
val pti = partitionMap(topicAndPartition)
try {
addFetcher(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset(), leaderBroker)
noLeaderPartitionSet -= topicAndPartition
} catch {
case t => warn("Failed to add fetcher for %s to broker %s".format(topicAndPartition, leaderBroker), t)
}
trace("Partitions without leader %s".format(noLeaderPartitionSet))
val brokers = getAllBrokersInCluster(zkClient)
val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet,
brokers,
config.clientId,
config.socketTimeoutMs,
correlationId.getAndIncrement).topicsMetadata
if(logger.isDebugEnabled) topicsMetadata.foreach(topicMetadata => debug(topicMetadata.toString()))
topicsMetadata.foreach { tmd =>
val topic = tmd.topic
tmd.partitionsMetadata.foreach { pmd =>
val topicAndPartition = TopicAndPartition(topic, pmd.partitionId)
if(pmd.leader.isDefined && noLeaderPartitionSet.contains(topicAndPartition)) {
val leaderBroker = pmd.leader.get
leaderForPartitionsMap.put(topicAndPartition, leaderBroker)
noLeaderPartitionSet -= topicAndPartition
}
}
shutdownIdleFetcherThreads()
} catch {
case t => warn("Failed to find leader for %s".format(noLeaderPartitionSet), t)
}
} catch {
case t => {
if (!isRunning.get())
throw t /* If this thread is stopped, propagate this exception to kill the thread. */
else
warn("Failed to find leader for %s".format(noLeaderPartitionSet), t)
}
} finally {
lock.unlock()
}
leaderForPartitionsMap.foreach {
case(topicAndPartition, leaderBroker) =>
val pti = partitionMap(topicAndPartition)
addFetcher(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset(), leaderBroker)
}
shutdownIdleFetcherThreads()
Thread.sleep(config.refreshLeaderBackoffMs)
}
}
@ -120,6 +122,11 @@ class ConsumerFetcherManager(private val consumerIdString: String,
}
def stopConnections() {
/*
* Stop the leader finder thread first before stopping fetchers. Otherwise, if there are more partitions without
* leader, then the leader finder thread will process these partitions (before shutting down) and add fetchers for
* these partitions.
*/
info("Stopping leader finder thread")
if (leaderFinderThread != null) {
leaderFinderThread.shutdown()
@ -141,7 +148,6 @@ class ConsumerFetcherManager(private val consumerIdString: String,
lock.lock()
try {
if (partitionMap != null) {
partitionList.foreach(tp => removeFetcher(tp.topic, tp.partition))
noLeaderPartitionSet ++= partitionList
cond.signalAll()
}

View File

@ -20,9 +20,8 @@ package kafka.consumer
import kafka.cluster.Broker
import kafka.server.AbstractFetcherThread
import kafka.message.ByteBufferMessageSet
import kafka.api.{PartitionOffsetRequestInfo, Request, OffsetRequest, FetchResponsePartitionData}
import kafka.api.{Request, OffsetRequest, FetchResponsePartitionData}
import kafka.common.TopicAndPartition
import kafka.common.ErrorMapping
class ConsumerFetcherThread(name: String,
@ -67,6 +66,7 @@ class ConsumerFetcherThread(name: String,
// any logic for partitions whose leader has changed
def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) {
partitions.foreach(tap => removePartition(tap.topic, tap.partition))
consumerFetcherManager.addPartitionsWithError(partitions)
}
}

View File

@ -20,11 +20,7 @@ package kafka.consumer
import kafka.api._
import kafka.network._
import kafka.utils._
import kafka.utils.ZkUtils._
import collection.immutable
import kafka.common.{ErrorMapping, TopicAndPartition, KafkaException}
import org.I0Itec.zkclient.ZkClient
import kafka.cluster.Broker
import kafka.common.{ErrorMapping, TopicAndPartition}
/**
* A consumer of kafka messages
@ -41,6 +37,7 @@ class SimpleConsumer(val host: String,
private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout)
val brokerInfo = "host_%s-port_%s".format(host, port)
private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId)
private var isClosed = false
private def connect(): BlockingChannel = {
close
@ -62,7 +59,8 @@ class SimpleConsumer(val host: String,
def close() {
lock synchronized {
disconnect()
disconnect()
isClosed = true
}
}
@ -141,7 +139,7 @@ class SimpleConsumer(val host: String,
def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).buffer)
private def getOrMakeConnection() {
if(!blockingChannel.isConnected) {
if(!isClosed && !blockingChannel.isConnected) {
connect()
}
}

View File

@ -306,6 +306,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
class ZKRebalancerListener(val group: String, val consumerIdString: String,
val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_,_]]])
extends IZkChildListener {
private val correlationId = new AtomicInteger(0)
private var isWatcherTriggered = false
private val lock = new ReentrantLock
private val cond = lock.newCondition()
@ -406,9 +407,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val topicsMetadata = ClientUtils.fetchTopicMetadata(myTopicThreadIdsMap.keySet,
brokers,
config.clientId,
config.socketTimeoutMs).topicsMetadata
config.socketTimeoutMs,
correlationId.getAndIncrement).topicsMetadata
val partitionsPerTopicMap = new mutable.HashMap[String, Seq[Int]]
topicsMetadata.foreach(m =>{
topicsMetadata.foreach(m => {
val topic = m.topic
val partitions = m.partitionsMetadata.map(m1 => m1.partitionId)
partitionsPerTopicMap.put(topic, partitions)
@ -497,6 +499,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
* by the consumer, there will be no more messages returned by this iterator until the rebalancing finishes
* successfully and the fetchers restart to fetch more data chunks
**/
if (config.autoCommitEnable)
commitOffsets
case None =>
}

View File

@ -25,6 +25,7 @@ import kafka.server.KafkaConfig
import collection.mutable
import kafka.api._
import org.apache.log4j.Logger
import kafka.common.TopicAndPartition
class ControllerChannelManager (private val controllerContext: ControllerContext, config: KafkaConfig) extends Logging {
private val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo]
@ -75,6 +76,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext
private def addNewBroker(broker: Broker) {
val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize)
debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id))
val channel = new BlockingChannel(broker.host, broker.port,
BlockingChannel.UseDefaultBufferSize,
BlockingChannel.UseDefaultBufferSize,
@ -120,6 +122,7 @@ class RequestSendThread(val controllerId: Int,
try{
lock synchronized {
channel.connect() // establish a socket connection if needed
channel.send(request)
receive = channel.receive()
var response: RequestOrResponse = null
@ -128,6 +131,8 @@ class RequestSendThread(val controllerId: Int,
response = LeaderAndIsrResponse.readFrom(receive.buffer)
case RequestKeys.StopReplicaKey =>
response = StopReplicaResponse.readFrom(receive.buffer)
case RequestKeys.UpdateMetadataKey =>
response = UpdateMetadataResponse.readFrom(receive.buffer)
}
stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %d"
.format(controllerId, controllerContext.epoch, response.correlationId, toBrokerId))
@ -138,36 +143,51 @@ class RequestSendThread(val controllerId: Int,
}
} catch {
case e =>
// log it and let it go. Let controller shut it down.
debug("Exception occurs", e)
warn("Controller %d fails to send a request to broker %d".format(controllerId, toBrokerId), e)
// If there is any socket error (eg, socket timeout), the channel is no longer usable and needs to be recreated.
channel.disconnect()
}
}
}
class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit,
class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit,
controllerId: Int, clientId: String)
extends Logging {
val leaderAndIsrRequestMap = new mutable.HashMap[Int, mutable.HashMap[(String, Int), PartitionStateInfo]]
val stopReplicaRequestMap = new mutable.HashMap[Int, Seq[(String, Int)]]
val stopAndDeleteReplicaRequestMap = new mutable.HashMap[Int, Seq[(String, Int)]]
val updateMetadataRequestMap = new mutable.HashMap[Int, mutable.HashMap[TopicAndPartition, PartitionStateInfo]]
private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger)
def newBatch() {
// raise error if the previous batch is not empty
if(leaderAndIsrRequestMap.size > 0 || stopReplicaRequestMap.size > 0)
if(leaderAndIsrRequestMap.size > 0)
throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating " +
"a new one. Some state changes %s might be lost ".format(leaderAndIsrRequestMap.toString()))
"a new one. Some LeaderAndIsr state changes %s might be lost ".format(leaderAndIsrRequestMap.toString()))
if(stopReplicaRequestMap.size > 0)
throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " +
"new one. Some StopReplica state changes %s might be lost ".format(stopReplicaRequestMap.toString()))
if(updateMetadataRequestMap.size > 0)
throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " +
"new one. Some UpdateMetadata state changes %s might be lost ".format(updateMetadataRequestMap.toString()))
if(stopAndDeleteReplicaRequestMap.size > 0)
throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " +
"new one. Some StopReplica with delete state changes %s might be lost ".format(stopAndDeleteReplicaRequestMap.toString()))
leaderAndIsrRequestMap.clear()
stopReplicaRequestMap.clear()
updateMetadataRequestMap.clear()
stopAndDeleteReplicaRequestMap.clear()
}
def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int,
leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, replicationFactor: Int) {
leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch,
replicas: Seq[Int]) {
brokerIds.foreach { brokerId =>
leaderAndIsrRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[(String, Int), PartitionStateInfo])
leaderAndIsrRequestMap(brokerId).put((topic, partition), PartitionStateInfo(leaderIsrAndControllerEpoch, replicationFactor))
leaderAndIsrRequestMap(brokerId).put((topic, partition),
PartitionStateInfo(leaderIsrAndControllerEpoch, replicas.toSet))
}
addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(TopicAndPartition(topic, partition)))
}
def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, deletePartition: Boolean) {
@ -185,6 +205,30 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques
}
}
def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int],
partitions:scala.collection.Set[TopicAndPartition] = Set.empty[TopicAndPartition]) {
val partitionList =
if(partitions.isEmpty) {
controllerContext.partitionLeadershipInfo.keySet
} else {
partitions
}
partitionList.foreach { partition =>
val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition)
leaderIsrAndControllerEpochOpt match {
case Some(leaderIsrAndControllerEpoch) =>
val replicas = controllerContext.partitionReplicaAssignment(partition).toSet
val partitionStateInfo = PartitionStateInfo(leaderIsrAndControllerEpoch, replicas)
brokerIds.foreach { brokerId =>
updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo])
updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo)
}
case None =>
info("Leader not assigned yet for partition %s. Skip sending udpate metadata request".format(partition))
}
}
}
def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int, liveBrokers: Set[Broker]) {
leaderAndIsrRequestMap.foreach { m =>
val broker = m._1
@ -202,6 +246,16 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques
sendRequest(broker, leaderAndIsrRequest, null)
}
leaderAndIsrRequestMap.clear()
updateMetadataRequestMap.foreach { m =>
val broker = m._1
val partitionStateInfos = m._2.toMap
val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId,
partitionStateInfos, controllerContext.liveOrShuttingDownBrokers)
partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request with " +
"correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, correlationId, broker, p._1)))
sendRequest(broker, updateMetadataRequest, null)
}
updateMetadataRequestMap.clear()
Seq((stopReplicaRequestMap, false), (stopAndDeleteReplicaRequestMap, true)) foreach {
case(m, deletePartitions) => {
m foreach {

View File

@ -32,9 +32,9 @@ import kafka.utils.{Utils, ZkUtils, Logging}
import org.apache.zookeeper.Watcher.Event.KeeperState
import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient}
import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException}
import java.util.concurrent.atomic.AtomicInteger
import scala.Some
import kafka.common.TopicAndPartition
import java.util.concurrent.atomic.AtomicInteger
class ControllerContext(val zkClient: ZkClient,
var controllerChannelManager: ControllerChannelManager = null,
@ -65,11 +65,12 @@ class ControllerContext(val zkClient: ZkClient,
def liveBrokers = liveBrokersUnderlying.filter(broker => !shuttingDownBrokerIds.contains(broker.id))
def liveBrokerIds = liveBrokerIdsUnderlying.filter(brokerId => !shuttingDownBrokerIds.contains(brokerId))
def liveOrShuttingDownBrokerIds = liveBrokerIdsUnderlying ++ shuttingDownBrokerIds
def liveOrShuttingDownBrokerIds = liveBrokerIdsUnderlying
def liveOrShuttingDownBrokers = liveBrokersUnderlying
}
trait KafkaControllerMBean {
def shutdownBroker(id: Int): Int
def shutdownBroker(id: Int): Set[TopicAndPartition]
}
object KafkaController {
@ -91,7 +92,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext)
private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext)
private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext)
private val brokerRequestBatch = new ControllerBrokerRequestBatch(sendRequest, this.config.brokerId, this.clientId)
private val brokerRequestBatch = new ControllerBrokerRequestBatch(controllerContext, sendRequest, this.config.brokerId, this.clientId)
registerControllerChangedListener()
newGauge(
@ -117,17 +118,18 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
def clientId = "id_%d-host_%s-port_%d".format(config.brokerId, config.hostName, config.port)
/**
* JMX operation to initiate clean shutdown of a broker. On clean shutdown,
* the controller first determines the partitions that the shutting down
* broker leads, and moves leadership of those partitions to another broker
* that is in that partition's ISR. When all partitions have been moved, the
* broker process can be stopped normally (i.e., by sending it a SIGTERM or
* SIGINT) and no data loss should be observed.
* On clean shutdown, the controller first determines the partitions that the
* shutting down broker leads, and moves leadership of those partitions to another broker
* that is in that partition's ISR.
*
* @param id Id of the broker to shutdown.
* @return The number of partitions that the broker still leads.
*/
def shutdownBroker(id: Int) = {
def shutdownBroker(id: Int) : Set[TopicAndPartition] = {
if (!isActive()) {
throw new ControllerMovedException("Controller moved to another broker. Aborting controlled shutdown")
}
controllerContext.brokerShutdownLock synchronized {
info("Shutting down broker " + id)
@ -150,6 +152,32 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
}
}
allPartitionsAndReplicationFactorOnBroker.foreach {
case(topicAndPartition, replicationFactor) =>
// Move leadership serially to relinquish lock.
controllerContext.controllerLock synchronized {
controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch =>
if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) {
// If the broker leads the topic partition, transition the leader and update isr. Updates zk and
// notifies all affected brokers
partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition,
controlledShutdownPartitionLeaderSelector)
}
else {
// Stop the replica first. The state change below initiates ZK changes which should take some time
// before which the stop replica request should be completed (in most cases)
brokerRequestBatch.newBatch()
brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, topicAndPartition.partition, deletePartition = false)
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers)
// If the broker is a follower, updates the isr in ZK and notifies the current leader
replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic,
topicAndPartition.partition, id)), OfflineReplica)
}
}
}
}
def replicatedPartitionsBrokerLeads() = controllerContext.controllerLock.synchronized {
trace("All leaders = " + controllerContext.partitionLeadershipInfo.mkString(","))
controllerContext.partitionLeadershipInfo.filter {
@ -157,60 +185,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
leaderIsrAndControllerEpoch.leaderAndIsr.leader == id && controllerContext.partitionReplicaAssignment(topicAndPartition).size > 1
}.map(_._1)
}
val partitionsToMove = replicatedPartitionsBrokerLeads().toSet
debug("Partitions to move leadership from broker %d: %s".format(id, partitionsToMove.mkString(",")))
partitionsToMove.foreach{ topicAndPartition =>
val (topic, partition) = topicAndPartition.asTuple
// move leadership serially to relinquish lock.
controllerContext.controllerLock synchronized {
controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch =>
if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) {
partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition,
controlledShutdownPartitionLeaderSelector)
val newLeaderIsrAndControllerEpoch = controllerContext.partitionLeadershipInfo(topicAndPartition)
// mark replica offline only if leadership was moved successfully
if (newLeaderIsrAndControllerEpoch.leaderAndIsr.leader != currLeaderIsrAndControllerEpoch.leaderAndIsr.leader)
replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topic, partition, id)), OfflineReplica)
} else
debug("Partition %s moved from leader %d to new leader %d during shutdown."
.format(topicAndPartition, id, currLeaderIsrAndControllerEpoch.leaderAndIsr.leader))
}
}
}
val partitionsRemaining = replicatedPartitionsBrokerLeads().toSet
/*
* Force the shutting down broker out of the ISR of partitions that it
* follows, and shutdown the corresponding replica fetcher threads.
* This is really an optimization, so no need to register any callback
* to wait until completion.
*/
if (partitionsRemaining.size == 0) {
brokerRequestBatch.newBatch()
allPartitionsAndReplicationFactorOnBroker foreach {
case(topicAndPartition, replicationFactor) =>
val (topic, partition) = topicAndPartition.asTuple
if (controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader != id) {
brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topic, partition, deletePartition = false)
removeReplicaFromIsr(topic, partition, id) match {
case Some(updatedLeaderIsrAndControllerEpoch) =>
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(
Seq(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader), topic, partition,
updatedLeaderIsrAndControllerEpoch, replicationFactor)
case None =>
// ignore
}
}
}
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers)
}
debug("Remaining partitions to move from broker %d: %s".format(id, partitionsRemaining.mkString(",")))
partitionsRemaining.size
replicatedPartitionsBrokerLeads().toSet
}
}
@ -244,6 +219,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
info("Broker %d is ready to serve as the new controller with epoch %d".format(config.brokerId, epoch))
initializeAndMaybeTriggerPartitionReassignment()
initializeAndMaybeTriggerPreferredReplicaElection()
/* send partition leadership info to all live brokers */
sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq)
}
else
info("Controller has been shut down, aborting startup/failover")
@ -275,13 +252,16 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
info("New broker startup callback for %s".format(newBrokers.mkString(",")))
val newBrokersSet = newBrokers.toSet
// send update metadata request for all partitions to the newly restarted brokers. In cases of controlled shutdown
// leaders will not be elected when a new broker comes up. So at least in the common controlled shutdown case, the
// metadata will reach the new brokers faster
sendUpdateMetadataRequest(newBrokers)
// the very first thing to do when a new broker comes up is send it the entire list of partitions that it is
// supposed to host. Based on that the broker starts the high watermark threads for the input list of partitions
replicaStateMachine.handleStateChanges(getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq, newBrokers), OnlineReplica)
// when a new broker comes up, the controller needs to trigger leader election for all new and offline partitions
// to see if these brokers can become leaders for some/all of those
partitionStateMachine.triggerOnlinePartitionStateChange()
// check if reassignment of some partitions need to be restarted
val partitionsWithReplicasOnNewBrokers = controllerContext.partitionsBeingReassigned.filter{
case (topicAndPartition, reassignmentContext) =>
@ -379,6 +359,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
removePartitionFromReassignedPartitions(topicAndPartition)
info("Removed partition %s from the list of reassigned partitions in zookeeper".format(topicAndPartition))
controllerContext.partitionsBeingReassigned.remove(topicAndPartition)
// after electing leader, the replicas and isr information changes, so resend the update metadata request
sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicAndPartition))
case false =>
info("New replicas %s for partition %s being ".format(reassignedReplicas.mkString(","), topicAndPartition) +
"reassigned not yet caught up with the leader")
@ -478,6 +460,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
controllerContext.allTopics = ZkUtils.getAllTopics(zkClient).toSet
controllerContext.partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, controllerContext.allTopics.toSeq)
controllerContext.partitionLeadershipInfo = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch]
controllerContext.shuttingDownBrokerIds = mutable.Set.empty[Int]
// update the leader and isr cache for all existing partitions from Zookeeper
updateLeaderAndIsrCache()
// start the channel manager
@ -522,7 +505,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
}
private def updateLeaderAndIsrCache() {
val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, controllerContext.allTopics.toSeq)
val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, controllerContext.partitionReplicaAssignment.keySet)
for((topicPartition, leaderIsrAndControllerEpoch) <- leaderAndIsrInfo)
controllerContext.partitionLeadershipInfo.put(topicPartition, leaderIsrAndControllerEpoch)
}
@ -662,6 +645,18 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
}.flatten
}
/**
* Send the leader information for selected partitions to selected brokers so that they can correctly respond to
* metadata requests
* @param brokers The brokers that the update metadata request should be sent to
* @param partitions The partitions for which the metadata is to be sent
*/
private def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) {
brokerRequestBatch.newBatch()
brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions)
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers)
}
/**
* Removes a given partition replica from the ISR; if it is not the current
* leader and there are sufficient remaining replicas in ISR.
@ -673,7 +668,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
*/
def removeReplicaFromIsr(topic: String, partition: Int, replicaId: Int): Option[LeaderIsrAndControllerEpoch] = {
val topicAndPartition = TopicAndPartition(topic, partition)
debug("Removing replica %d from ISR of %s.".format(replicaId, topicAndPartition))
debug("Removing replica %d from ISR %s for partition %s.".format(replicaId,
controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.isr.mkString(","), topicAndPartition))
var finalLeaderIsrAndControllerEpoch: Option[LeaderIsrAndControllerEpoch] = None
var zkWriteCompleteOrUnnecessary = false
while (!zkWriteCompleteOrUnnecessary) {
@ -701,6 +697,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
newLeaderAndIsr.zkVersion = newVersion
finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch))
controllerContext.partitionLeadershipInfo.put(topicAndPartition, finalLeaderIsrAndControllerEpoch.get)
if (updateSucceeded)
info("New leader and ISR for partition %s is %s".format(topicAndPartition, newLeaderAndIsr.toString()))
updateSucceeded
@ -708,6 +705,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
warn("Cannot remove replica %d from ISR of %s. Leader = %d ; ISR = %s"
.format(replicaId, topicAndPartition, leaderAndIsr.leader, leaderAndIsr.isr))
finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(leaderAndIsr, epoch))
controllerContext.partitionLeadershipInfo.put(topicAndPartition, finalLeaderIsrAndControllerEpoch.get)
true
}
case None =>

View File

@ -63,13 +63,14 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten
case false =>
ControllerStats.uncleanLeaderElectionRate.mark()
val newLeader = liveAssignedReplicasToThisPartition.head
warn("No broker in ISR is alive for %s. Elect leader from broker %s. There's potential data loss."
.format(topicAndPartition, newLeader))
warn("No broker in ISR is alive for %s. Elect leader %d from live brokers %s. There's potential data loss."
.format(topicAndPartition, newLeader, liveAssignedReplicasToThisPartition.mkString(",")))
new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1)
}
case false =>
val newLeader = liveBrokersInIsr.head
debug("Some broker in ISR is alive for %s. Select %d from ISR to be the leader.".format(topicAndPartition, newLeader))
debug("Some broker in ISR is alive for %s. Select %d from ISR %s to be the leader."
.format(topicAndPartition, newLeader, liveBrokersInIsr.mkString(",")))
new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1)
}
info("Selected new leader and ISR %s for offline partition %s".format(newLeaderAndIsr.toString(), topicAndPartition))

View File

@ -43,9 +43,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
private val controllerId = controller.config.brokerId
private val zkClient = controllerContext.zkClient
var partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty
val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.sendRequest, controllerId, controller.clientId)
val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.controllerContext, controller.sendRequest,
controllerId, controller.clientId)
private val hasStarted = new AtomicBoolean(false)
private val hasShutdown = new AtomicBoolean(false)
private val noOpPartitionLeaderSelector = new NoOpLeaderSelector(controllerContext)
this.logIdent = "[Partition state machine on Controller " + controllerId + "]: "
private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger)
@ -73,7 +73,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
* Invoked on controller shutdown.
*/
def shutdown() {
hasShutdown.compareAndSet(false, true)
hasStarted.set(false)
partitionState.clear()
}
@ -188,13 +188,11 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
*/
private def initializePartitionState() {
for((topicPartition, replicaAssignment) <- controllerContext.partitionReplicaAssignment) {
val topic = topicPartition.topic
val partition = topicPartition.partition
// check if leader and isr path exists for partition. If not, then it is in NEW state
ZkUtils.getLeaderAndIsrForPartition(zkClient, topic, partition) match {
case Some(currentLeaderAndIsr) =>
controllerContext.partitionLeadershipInfo.get(topicPartition) match {
case Some(currentLeaderIsrAndEpoch) =>
// else, check if the leader for partition is alive. If yes, it is in Online state, else it is in Offline state
controllerContext.liveBrokerIds.contains(currentLeaderAndIsr.leader) match {
controllerContext.liveBrokerIds.contains(currentLeaderIsrAndEpoch.leaderAndIsr.leader) match {
case true => // leader is alive
partitionState.put(topicPartition, OnlinePartition)
case false =>
@ -233,7 +231,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
* @param topicAndPartition The topic/partition whose leader and isr path is to be initialized
*/
private def initializeLeaderAndIsrForPartition(topicAndPartition: TopicAndPartition) {
debug("Initializing leader and isr for partition %s".format(topicAndPartition))
val replicaAssignment = controllerContext.partitionReplicaAssignment(topicAndPartition)
val liveAssignedReplicas = replicaAssignment.filter(r => controllerContext.liveBrokerIds.contains(r))
liveAssignedReplicas.size match {
@ -249,6 +246,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
val leader = liveAssignedReplicas.head
val leaderIsrAndControllerEpoch = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader, liveAssignedReplicas.toList),
controller.epoch)
debug("Initializing leader and isr for partition %s to %s".format(topicAndPartition, leaderIsrAndControllerEpoch))
try {
ZkUtils.createPersistentPath(controllerContext.zkClient,
ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition),
@ -256,9 +254,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
// NOTE: the above write can fail only if the current controller lost its zk session and the new controller
// took over and initialized this partition. This can happen if the current controller went into a long
// GC pause
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(liveAssignedReplicas, topicAndPartition.topic,
topicAndPartition.partition, leaderIsrAndControllerEpoch, replicaAssignment.size)
controllerContext.partitionLeadershipInfo.put(topicAndPartition, leaderIsrAndControllerEpoch)
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(liveAssignedReplicas, topicAndPartition.topic,
topicAndPartition.partition, leaderIsrAndControllerEpoch, replicaAssignment)
} catch {
case e: ZkNodeExistsException =>
// read the controller epoch
@ -316,9 +314,10 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
controllerContext.partitionLeadershipInfo.put(TopicAndPartition(topic, partition), newLeaderIsrAndControllerEpoch)
stateChangeLogger.trace("Controller %d epoch %d elected leader %d for Offline partition %s"
.format(controllerId, controller.epoch, newLeaderAndIsr.leader, topicAndPartition))
val replicas = controllerContext.partitionReplicaAssignment(TopicAndPartition(topic, partition))
// store new leader and isr info in cache
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasForThisPartition, topic, partition,
newLeaderIsrAndControllerEpoch, controllerContext.partitionReplicaAssignment(TopicAndPartition(topic, partition)).size)
newLeaderIsrAndControllerEpoch, replicas)
} catch {
case lenne: LeaderElectionNotNeededException => // swallow
case nroe: NoReplicaOnlineException => throw nroe
@ -358,7 +357,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
@throws(classOf[Exception])
def handleChildChange(parentPath : String, children : java.util.List[String]) {
controllerContext.controllerLock synchronized {
if (!hasShutdown.get) {
if (hasStarted.get) {
try {
debug("Topic change listener fired for path %s with children %s".format(parentPath, children.mkString(",")))
val currentChildren = JavaConversions.asBuffer(children).toSet

View File

@ -42,9 +42,9 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
private val controllerId = controller.config.brokerId
private val zkClient = controllerContext.zkClient
var replicaState: mutable.Map[(String, Int, Int), ReplicaState] = mutable.Map.empty
val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.sendRequest, controllerId, controller.clientId)
val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.controllerContext, controller.sendRequest,
controllerId, controller.clientId)
private val hasStarted = new AtomicBoolean(false)
private val hasShutdown = new AtomicBoolean(false)
this.logIdent = "[Replica state machine on controller " + controller.config.brokerId + "]: "
private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger)
@ -58,7 +58,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
initializeReplicaState()
hasStarted.set(true)
// move all Online replicas to Online
handleStateChanges(ZkUtils.getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq,
handleStateChanges(getAllReplicasOnBroker(controllerContext.allTopics.toSeq,
controllerContext.liveBrokerIds.toSeq), OnlineReplica)
info("Started replica state machine with initial state -> " + replicaState.toString())
}
@ -72,7 +72,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
* Invoked on controller shutdown.
*/
def shutdown() {
hasShutdown.compareAndSet(false, true)
hasStarted.set(false)
replicaState.clear()
}
@ -121,7 +121,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
throw new StateChangeFailedException("Replica %d for partition %s cannot be moved to NewReplica"
.format(replicaId, topicAndPartition) + "state as it is being requested to become leader")
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId),
topic, partition, leaderIsrAndControllerEpoch, replicaAssignment.size)
topic, partition, leaderIsrAndControllerEpoch,
replicaAssignment)
case None => // new leader request will be sent to this replica when one gets elected
}
replicaState.put((topic, partition, replicaId), NewReplica)
@ -152,7 +153,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
controllerContext.partitionLeadershipInfo.get(topicAndPartition) match {
case Some(leaderIsrAndControllerEpoch) =>
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), topic, partition, leaderIsrAndControllerEpoch,
replicaAssignment.size)
replicaAssignment)
replicaState.put((topic, partition, replicaId), OnlineReplica)
stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica"
.format(controllerId, controller.epoch, replicaId, topicAndPartition))
@ -173,8 +174,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
case Some(updatedLeaderIsrAndControllerEpoch) =>
// send the shrunk ISR state change request only to the leader
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader),
topic, partition, updatedLeaderIsrAndControllerEpoch,
replicaAssignment.size)
topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment)
replicaState.put((topic, partition, replicaId), OfflineReplica)
stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica"
.format(controllerId, controller.epoch, replicaId, topicAndPartition))
@ -228,6 +228,16 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
}
}
private def getAllReplicasOnBroker(topics: Seq[String], brokerIds: Seq[Int]): Set[PartitionAndReplica] = {
brokerIds.map { brokerId =>
val partitionsAssignedToThisBroker =
controllerContext.partitionReplicaAssignment.filter(p => topics.contains(p._1.topic) && p._2.contains(brokerId))
if(partitionsAssignedToThisBroker.size == 0)
info("No state transitions triggered since no partitions are assigned to brokers %s".format(brokerIds.mkString(",")))
partitionsAssignedToThisBroker.map(p => new PartitionAndReplica(p._1.topic, p._1.partition, brokerId))
}.flatten.toSet
}
def getPartitionsAssignedToBroker(topics: Seq[String], brokerId: Int):Seq[TopicAndPartition] = {
controllerContext.partitionReplicaAssignment.filter(_._2.contains(brokerId)).keySet.toSeq
}
@ -238,14 +248,15 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
class BrokerChangeListener() extends IZkChildListener with Logging {
this.logIdent = "[BrokerChangeListener on Controller " + controller.config.brokerId + "]: "
def handleChildChange(parentPath : String, currentBrokerList : java.util.List[String]) {
ControllerStats.leaderElectionTimer.time {
info("Broker change listener fired for path %s with children %s".format(parentPath, currentBrokerList.mkString(",")))
controllerContext.controllerLock synchronized {
if (!hasShutdown.get) {
info("Broker change listener fired for path %s with children %s".format(parentPath, currentBrokerList.mkString(",")))
controllerContext.controllerLock synchronized {
if (hasStarted.get) {
ControllerStats.leaderElectionTimer.time {
try {
val curBrokerIds = currentBrokerList.map(_.toInt).toSet
val newBrokerIds = curBrokerIds -- controllerContext.liveOrShuttingDownBrokerIds
val newBrokers = newBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get)
val newBrokerInfo = newBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _))
val newBrokers = newBrokerInfo.filter(_.isDefined).map(_.get)
val deadBrokerIds = controllerContext.liveOrShuttingDownBrokerIds -- curBrokerIds
controllerContext.liveBrokers = curBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get)
info("Newly added brokers: %s, deleted brokers: %s, all live brokers: %s"

View File

@ -43,7 +43,7 @@ private[javaapi] object Implicits extends Logging {
implicit def optionToJavaRef[T](opt: Option[T]): T = {
opt match {
case Some(obj) => obj
case None => null
case None => null.asInstanceOf[T]
}
}
}

View File

@ -163,8 +163,17 @@ class Log(val dir: File,
// run recovery on the active segment if necessary
if(needsRecovery) {
info("Recovering active segment of %s.".format(name))
active.recover(config.maxMessageSize)
try {
info("Recovering active segment of %s.".format(name))
active.recover(config.maxMessageSize)
} catch {
case e: InvalidOffsetException =>
val startOffset = active.baseOffset
warn("Found invalid offset during recovery of the active segment for topic partition " + dir.getName +". Deleting the segment and " +
"creating an empty one with starting offset " + startOffset)
// truncate the active segment to its starting offset
active.truncateTo(startOffset)
}
}
}

View File

@ -1,4 +1,20 @@
package kafka.log
/**
* 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.log
import scala.math._
import java.io.File
@ -189,6 +205,7 @@ class LogSegment(val log: FileMessageSet,
val bytesTruncated = log.truncateTo(mapping.position)
if(log.sizeInBytes == 0)
created = time.milliseconds
bytesSinceLastIndexEntry = 0
bytesTruncated
}

View File

@ -23,6 +23,7 @@ import java.nio._
import java.nio.channels._
import java.util.concurrent.atomic._
import kafka.utils._
import kafka.common.InvalidOffsetException
/**
* An index that maps offsets to physical file locations for a particular log segment. This index may be sparse:
@ -190,13 +191,18 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi
def append(offset: Long, position: Int) {
this synchronized {
require(!isFull, "Attempt to append to a full index (size = " + size + ").")
require(size.get == 0 || offset > lastOffset, "Attempt to append an offset (%d) to position %d no larger than the last offset appended (%d).".format(offset, entries, lastOffset))
debug("Adding index entry %d => %d to %s.".format(offset, position, file.getName))
this.mmap.putInt((offset - baseOffset).toInt)
this.mmap.putInt(position)
this.size.incrementAndGet()
this.lastOffset = offset
require(entries * 8 == mmap.position, entries + " entries but file position in index is " + mmap.position + ".")
if (size.get == 0 || offset > lastOffset) {
debug("Adding index entry %d => %d to %s.".format(offset, position, file.getName))
this.mmap.putInt((offset - baseOffset).toInt)
this.mmap.putInt(position)
this.size.incrementAndGet()
this.lastOffset = offset
require(entries * 8 == mmap.position, entries + " entries but file position in index is " + mmap.position + ".")
}
else {
throw new InvalidOffsetException("Attempt to append an offset (%d) to position %d no larger than the last offset appended (%d) to %s."
.format(offset, entries, lastOffset, file.getName))
}
}
}

View File

@ -74,8 +74,9 @@ object Message {
* 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used)
* 4. 4 byte key length, containing length K
* 5. K byte key
* 6. (N - K - 10) byte payload
*
* 6. 4 byte payload length, containing length V
* 7. V byte payload
*
* Default constructor wraps an existing ByteBuffer with the Message object with no change to the contents.
*/
class Message(val buffer: ByteBuffer) {

View File

@ -49,7 +49,7 @@ object RequestChannel extends Logging {
val requestObj: RequestOrResponse = RequestKeys.deserializerForKey(requestId)(buffer)
buffer = null
private val requestLogger = Logger.getLogger("kafka.request.logger")
trace("Received request : %s".format(requestObj))
trace("Processor %d received request : %s".format(processor, requestObj))
def updateRequestMetrics() {
val endTimeMs = SystemTime.milliseconds

View File

@ -61,20 +61,19 @@ class SocketServer(val brokerId: Int,
this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize)
Utils.newThread("kafka-acceptor", acceptor, false).start()
acceptor.awaitStartup
info("started")
info("Started")
}
/**
* Shutdown the socket server
*/
def shutdown() = {
info("shutting down")
info("Shutting down")
if(acceptor != null)
acceptor.shutdown()
for(processor <- processors)
processor.shutdown()
requestChannel.shutdown
info("shut down completely")
info("Shutdown completed")
}
}

View File

@ -54,6 +54,13 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig,
}
}
val partitionMetadata = metadata.partitionsMetadata
if(partitionMetadata.size == 0) {
if(metadata.errorCode != ErrorMapping.NoError) {
throw new KafkaException(ErrorMapping.exceptionFor(metadata.errorCode))
} else {
throw new KafkaException("Topic metadata %s has empty partition metadata and no error code".format(metadata))
}
}
partitionMetadata.map { m =>
m.leader match {
case Some(leader) =>
@ -77,7 +84,7 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig,
// throw partition specific exception
topicsMetadata.foreach(tmd =>{
trace("Metadata for topic %s is %s".format(tmd.topic, tmd))
if(tmd.errorCode == ErrorMapping.NoError){
if(tmd.errorCode == ErrorMapping.NoError) {
topicPartitionInfo.put(tmd.topic, tmd)
} else
warn("Error while fetching metadata [%s] for topic [%s]: %s ".format(tmd, tmd.topic, ErrorMapping.exceptionFor(tmd.errorCode).getClass))

View File

@ -106,7 +106,6 @@ class Producer[K,V](val config: ProducerConfig,
if(!added) {
producerTopicStats.getProducerTopicStats(message.topic).droppedMessageRate.mark()
producerTopicStats.getProducerAllTopicsStats.droppedMessageRate.mark()
error("Event queue is full of unsent messages, could not send event: " + message.toString)
throw new QueueFullException("Event queue is full of unsent messages, could not send event: " + message.toString)
}else {
trace("Added to send queue an event: " + message.toString)

View File

@ -58,5 +58,5 @@ trait SyncProducerConfigShared {
object SyncProducerConfig {
val DefaultClientId = ""
val DefaultRequiredAcks : Short = 0
val DefaultAckTimeoutMs = 1500
val DefaultAckTimeoutMs = 10000
}

View File

@ -52,7 +52,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
def handle(events: Seq[KeyedMessage[K,V]]) {
lock synchronized {
val serializedData = serialize(events)
serializedData.foreach{
serializedData.foreach {
keyed =>
val dataSize = keyed.message.payloadSize
producerTopicStats.getProducerTopicStats(keyed.topic).byteRate.mark(dataSize)
@ -61,6 +61,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
var outstandingProduceRequests = serializedData
var remainingRetries = config.messageSendMaxRetries + 1
val correlationIdStart = correlationId.get()
debug("Handling %d events".format(events.size))
while (remainingRetries > 0 && outstandingProduceRequests.size > 0) {
topicMetadataToRefresh ++= outstandingProduceRequests.map(_.topic)
if (topicMetadataRefreshInterval >= 0 &&
@ -70,7 +71,8 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
lastTopicMetadataRefreshTime = SystemTime.milliseconds
}
outstandingProduceRequests = dispatchSerializedData(outstandingProduceRequests)
if (outstandingProduceRequests.size > 0) {
if (outstandingProduceRequests.size > 0) {
info("Back off for %d ms before retrying send. Remaining retries = %d".format(config.retryBackoffMs, remainingRetries-1))
// back off and update the topic metadata cache before attempting another send operation
Thread.sleep(config.retryBackoffMs)
// get topics of the outstanding produce requests and refresh metadata for those
@ -147,7 +149,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
try {
for (message <- messages) {
val topicPartitionsList = getPartitionListForTopic(message)
val partitionIndex = getPartition(message.key, topicPartitionsList)
val partitionIndex = getPartition(message.topic, message.key, topicPartitionsList)
val brokerPartition = topicPartitionsList(partitionIndex)
// postpone the failure until the send operation, so that requests for other brokers are handled correctly
@ -175,9 +177,9 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
}
Some(ret)
}catch { // Swallow recoverable exceptions and return None so that they can be retried.
case ute: UnknownTopicException => warn("Failed to collate messages by topic,partition due to", ute); None
case lnae: LeaderNotAvailableException => warn("Failed to collate messages by topic,partition due to", lnae); None
case oe => error("Failed to collate messages by topic, partition due to", oe); throw oe
case ute: UnknownTopicOrPartitionException => warn("Failed to collate messages by topic,partition due to: " + ute.getMessage); None
case lnae: LeaderNotAvailableException => warn("Failed to collate messages by topic,partition due to: " + lnae.getMessage); None
case oe => error("Failed to collate messages by topic, partition due to: " + oe.getMessage); None
}
}
@ -198,25 +200,24 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
* @param topicPartitionList the list of available partitions
* @return the partition id
*/
private def getPartition(key: K, topicPartitionList: Seq[PartitionAndLeader]): Int = {
private def getPartition(topic: String, key: K, topicPartitionList: Seq[PartitionAndLeader]): Int = {
val numPartitions = topicPartitionList.size
if(numPartitions <= 0)
throw new UnknownTopicOrPartitionException("Invalid number of partitions: " + numPartitions +
"\n Valid values are > 0")
throw new UnknownTopicOrPartitionException("Topic " + topic + " doesn't exist")
val partition =
if(key == null) {
// If the key is null, we don't really need a partitioner so we just send to the next
// available partition
val availablePartitions = topicPartitionList.filter(_.leaderBrokerIdOpt.isDefined)
if (availablePartitions.isEmpty)
throw new LeaderNotAvailableException("No leader for any partition")
throw new LeaderNotAvailableException("No leader for any partition in topic " + topic)
val index = Utils.abs(partitionCounter.getAndIncrement()) % availablePartitions.size
availablePartitions(index).partitionId
} else
partitioner.partition(key, numPartitions)
if(partition < 0 || partition >= numPartitions)
throw new UnknownTopicOrPartitionException("Invalid partition id : " + partition +
"\n Valid values are in the range inclusive [0, " + (numPartitions-1) + "]")
throw new UnknownTopicOrPartitionException("Invalid partition id: " + partition + " for topic " + topic +
"; Valid values are in the inclusive range of [0, " + (numPartitions-1) + "]")
partition
}
@ -251,11 +252,18 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
successfullySentData.foreach(m => messagesPerTopic(m._1).foreach(message =>
trace("Successfully sent message: %s".format(if(message.message.isNull) null else Utils.readString(message.message.payload)))))
}
failedTopicPartitions = response.status.filter(_._2.error != ErrorMapping.NoError).toSeq
.map(partitionStatus => partitionStatus._1)
if(failedTopicPartitions.size > 0)
error("Produce request with correlation id %d failed due to response %s. List of failed topic partitions is %s"
.format(currentCorrelationId, response.toString, failedTopicPartitions.mkString(",")))
val failedPartitionsAndStatus = response.status.filter(_._2.error != ErrorMapping.NoError).toSeq
failedTopicPartitions = failedPartitionsAndStatus.map(partitionStatus => partitionStatus._1)
if(failedTopicPartitions.size > 0) {
val errorString = failedPartitionsAndStatus
.sortWith((p1, p2) => p1._1.topic.compareTo(p2._1.topic) < 0 ||
(p1._1.topic.compareTo(p2._1.topic) == 0 && p1._1.partition < p2._1.partition))
.map{
case(topicAndPartition, status) =>
topicAndPartition.toString + ": " + ErrorMapping.exceptionFor(status.error).getClass.getName
}.mkString(",")
warn("Produce request with correlation id %d failed due to %s".format(currentCorrelationId, errorString))
}
failedTopicPartitions
} else {
Seq.empty[TopicAndPartition]

View File

@ -50,7 +50,7 @@ class ProducerSendThread[K,V](val threadName: String,
}
def shutdown = {
info("Beging shutting down ProducerSendThread")
info("Begin shutting down ProducerSendThread")
queue.put(shutdownCommand)
shutdownLatch.await
info("Shutdown ProducerSendThread complete")

View File

@ -20,13 +20,45 @@ package kafka.server
import scala.collection.mutable
import kafka.utils.Logging
import kafka.cluster.Broker
import kafka.metrics.KafkaMetricsGroup
import com.yammer.metrics.core.Gauge
abstract class AbstractFetcherManager(protected val name: String, numFetchers: Int = 1) extends Logging {
abstract class AbstractFetcherManager(protected val name: String, metricPrefix: String, numFetchers: Int = 1)
extends Logging with KafkaMetricsGroup {
// map of (source brokerid, fetcher Id per source broker) => fetcher
private val fetcherThreadMap = new mutable.HashMap[BrokerAndFetcherId, AbstractFetcherThread]
private val mapLock = new Object
this.logIdent = "[" + name + "] "
newGauge(
metricPrefix + "-MaxLag",
new Gauge[Long] {
// current max lag across all fetchers/topics/partitions
def value = fetcherThreadMap.foldLeft(0L)((curMaxAll, fetcherThreadMapEntry) => {
fetcherThreadMapEntry._2.fetcherLagStats.stats.foldLeft(0L)((curMaxThread, fetcherLagStatsEntry) => {
curMaxThread.max(fetcherLagStatsEntry._2.lag)
}).max(curMaxAll)
})
}
)
newGauge(
metricPrefix + "-MinFetchRate",
{
new Gauge[Double] {
// current min fetch rate across all fetchers/topics/partitions
def value = {
val headRate: Double =
fetcherThreadMap.headOption.map(_._2.fetcherStats.requestRate.oneMinuteRate).getOrElse(0)
fetcherThreadMap.foldLeft(headRate)((curMinAll, fetcherThreadMapEntry) => {
fetcherThreadMapEntry._2.fetcherStats.requestRate.oneMinuteRate.min(curMinAll)
})
}
}
}
)
private def getFetcherId(topic: String, partitionId: Int) : Int = {
(topic.hashCode() + 31 * partitionId) % numFetchers
}

View File

@ -96,8 +96,8 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke
response = simpleConsumer.fetch(fetchRequest)
} catch {
case t =>
warn("Error in fetch %s".format(fetchRequest), t)
if (isRunning.get) {
warn("Error in fetch %s".format(fetchRequest), t)
partitionMapLock synchronized {
partitionsWithError ++= partitionMap.keys
}
@ -152,9 +152,11 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke
partitionsWithError += topicAndPartition
}
case _ =>
warn("error for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id),
ErrorMapping.exceptionFor(partitionData.error))
partitionsWithError += topicAndPartition
if (isRunning.get) {
warn("error for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id),
ErrorMapping.exceptionFor(partitionData.error))
partitionsWithError += topicAndPartition
}
}
}
}
@ -219,7 +221,7 @@ class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMet
class FetcherLagStats(metricId: ClientIdAndBroker) {
private val valueFactory = (k: ClientIdBrokerTopicPartition) => new FetcherLagMetrics(k)
private val stats = new Pool[ClientIdBrokerTopicPartition, FetcherLagMetrics](Some(valueFactory))
val stats = new Pool[ClientIdBrokerTopicPartition, FetcherLagMetrics](Some(valueFactory))
def getFetcherLagStats(topic: String, partitionId: Int): FetcherLagMetrics = {
stats.getAndMaybePut(new ClientIdBrokerTopicPartition(metricId.clientId, metricId.brokerInfo, topic, partitionId))

View File

@ -33,6 +33,8 @@ import org.I0Itec.zkclient.ZkClient
import kafka.common._
import kafka.utils.{ZkUtils, Pool, SystemTime, Logging}
import kafka.network.RequestChannel.Response
import kafka.cluster.Broker
import kafka.controller.KafkaController
/**
@ -42,14 +44,21 @@ class KafkaApis(val requestChannel: RequestChannel,
val replicaManager: ReplicaManager,
val zkClient: ZkClient,
val brokerId: Int,
val config: KafkaConfig) extends Logging {
val config: KafkaConfig,
val controller: KafkaController) extends Logging {
private val producerRequestPurgatory =
new ProducerRequestPurgatory(replicaManager.config.producerPurgatoryPurgeIntervalRequests)
private val fetchRequestPurgatory =
new FetchRequestPurgatory(requestChannel, replicaManager.config.fetchPurgatoryPurgeIntervalRequests)
private val delayedRequestMetrics = new DelayedRequestMetrics
/* following 3 data structures are updated by the update metadata request
* and is queried by the topic metadata request. */
var leaderCache: mutable.Map[TopicAndPartition, PartitionStateInfo] =
new mutable.HashMap[TopicAndPartition, PartitionStateInfo]()
// private var allBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]()
private var aliveBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]()
private val partitionMetadataLock = new Object
this.logIdent = "[KafkaApi-%d] ".format(brokerId)
/**
@ -65,6 +74,8 @@ class KafkaApis(val requestChannel: RequestChannel,
case RequestKeys.MetadataKey => handleTopicMetadataRequest(request)
case RequestKeys.LeaderAndIsrKey => handleLeaderAndIsrRequest(request)
case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request)
case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request)
case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request)
case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request)
case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request)
case requestId => throw new KafkaException("Unknown api code " + requestId)
@ -90,7 +101,6 @@ class KafkaApis(val requestChannel: RequestChannel,
}
}
def handleStopReplicaRequest(request: RequestChannel.Request) {
val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest]
val (response, error) = replicaManager.stopReplicas(stopReplicaRequest)
@ -99,6 +109,41 @@ class KafkaApis(val requestChannel: RequestChannel,
replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads()
}
def handleUpdateMetadataRequest(request: RequestChannel.Request) {
val updateMetadataRequest = request.requestObj.asInstanceOf[UpdateMetadataRequest]
val stateChangeLogger = replicaManager.stateChangeLogger
if(updateMetadataRequest.controllerEpoch < replicaManager.controllerEpoch) {
val stateControllerEpochErrorMessage = ("Broker %d received update metadata request with correlation id %d from an " +
"old controller %d with epoch %d. Latest known controller epoch is %d").format(brokerId,
updateMetadataRequest.correlationId, updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch,
replicaManager.controllerEpoch)
stateChangeLogger.warn(stateControllerEpochErrorMessage)
throw new ControllerMovedException(stateControllerEpochErrorMessage)
}
partitionMetadataLock synchronized {
replicaManager.controllerEpoch = updateMetadataRequest.controllerEpoch
// cache the list of alive brokers in the cluster
updateMetadataRequest.aliveBrokers.foreach(b => aliveBrokers.put(b.id, b))
updateMetadataRequest.partitionStateInfos.foreach { partitionState =>
leaderCache.put(partitionState._1, partitionState._2)
if(stateChangeLogger.isTraceEnabled)
stateChangeLogger.trace(("Broker %d cached leader info %s for partition %s in response to UpdateMetadata request " +
"sent by controller %d epoch %d with correlation id %d").format(brokerId, partitionState._2, partitionState._1,
updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId))
}
}
val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(updateMetadataResponse)))
}
def handleControlledShutdownRequest(request: RequestChannel.Request) {
val controlledShutdownRequest = request.requestObj.asInstanceOf[ControlledShutdownRequest]
val partitionsRemaining = controller.shutdownBroker(controlledShutdownRequest.brokerId)
val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId,
ErrorMapping.NoError, partitionsRemaining)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse)))
}
/**
* Check if a partitionData from a produce request can unblock any
* DelayedFetch requests.
@ -218,16 +263,18 @@ class KafkaApis(val requestChannel: RequestChannel,
Runtime.getRuntime.halt(1)
null
case utpe: UnknownTopicOrPartitionException =>
warn("Produce request: " + utpe.getMessage)
warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format(
producerRequest.correlationId, producerRequest.clientId, topicAndPartition, utpe.getMessage))
new ProduceResult(topicAndPartition, utpe)
case nle: NotLeaderForPartitionException =>
warn("Produce request: " + nle.getMessage)
warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format(
producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nle.getMessage))
new ProduceResult(topicAndPartition, nle)
case e =>
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark()
error("Error processing ProducerRequest with correlation id %d from client %s on %s:%d"
.format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition.topic, topicAndPartition.partition), e)
error("Error processing ProducerRequest with correlation id %d from client %s on partition %s"
.format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition), e)
new ProduceResult(topicAndPartition, e)
}
}
@ -305,10 +352,12 @@ class KafkaApis(val requestChannel: RequestChannel,
// since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request
// for a partition it is the leader for
case utpe: UnknownTopicOrPartitionException =>
warn("Fetch request: " + utpe.getMessage)
warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format(
fetchRequest.correlationId, fetchRequest.clientId, topic, partition, utpe.getMessage))
new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty)
case nle: NotLeaderForPartitionException =>
warn("Fetch request: " + nle.getMessage)
warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format(
fetchRequest.correlationId, fetchRequest.clientId, topic, partition, nle.getMessage))
new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty)
case t =>
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
@ -384,10 +433,12 @@ class KafkaApis(val requestChannel: RequestChannel,
// NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages
// are typically transient and there is no value in logging the entire stack trace for the same
case utpe: UnknownTopicOrPartitionException =>
warn(utpe.getMessage)
warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format(
offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition, utpe.getMessage))
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), Nil) )
case nle: NotLeaderForPartitionException =>
warn(nle.getMessage)
warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format(
offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition,nle.getMessage))
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), Nil) )
case e =>
warn("Error while responding to offset request", e)
@ -458,46 +509,79 @@ class KafkaApis(val requestChannel: RequestChannel,
val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest]
val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]()
val config = replicaManager.config
val uniqueTopics = {
var uniqueTopics = Set.empty[String]
uniqueTopics = {
if(metadataRequest.topics.size > 0)
metadataRequest.topics.toSet
else
ZkUtils.getAllTopics(zkClient).toSet
}
val topicMetadataList = AdminUtils.fetchTopicMetadataFromZk(uniqueTopics, zkClient)
topicMetadataList.foreach(
topicAndMetadata => {
topicAndMetadata.errorCode match {
case ErrorMapping.NoError => topicsMetadata += topicAndMetadata
case ErrorMapping.UnknownTopicOrPartitionCode =>
try {
/* check if auto creation of topics is turned on */
if (config.autoCreateTopicsEnable) {
try {
AdminUtils.createTopic(zkClient, topicAndMetadata.topic, config.numPartitions, config.defaultReplicationFactor)
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
.format(topicAndMetadata.topic, config.numPartitions, config.defaultReplicationFactor))
} catch {
case e: TopicExistsException => // let it go, possibly another broker created this topic
}
val newTopicMetadata = AdminUtils.fetchTopicMetadataFromZk(topicAndMetadata.topic, zkClient)
topicsMetadata += newTopicMetadata
newTopicMetadata.errorCode match {
case ErrorMapping.NoError =>
case _ => throw new KafkaException("Topic metadata for automatically created topic %s does not exist".format(topicAndMetadata.topic))
}
val topicMetadataList =
partitionMetadataLock synchronized {
uniqueTopics.map { topic =>
if(leaderCache.keySet.map(_.topic).contains(topic)) {
val partitionStateInfo = leaderCache.filter(p => p._1.topic.equals(topic))
val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition)
val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) =>
val replicas = leaderCache(topicAndPartition).allReplicas
var replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq
var leaderInfo: Option[Broker] = None
var isrInfo: Seq[Broker] = Nil
val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch
val leader = leaderIsrAndEpoch.leaderAndIsr.leader
val isr = leaderIsrAndEpoch.leaderAndIsr.isr
debug("%s".format(topicAndPartition) + ";replicas = " + replicas + ", in sync replicas = " + isr + ", leader = " + leader)
try {
if(aliveBrokers.keySet.contains(leader))
leaderInfo = Some(aliveBrokers(leader))
else throw new LeaderNotAvailableException("Leader not available for partition %s".format(topicAndPartition))
isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null)
if(replicaInfo.size < replicas.size)
throw new ReplicaNotAvailableException("Replica information not available for following brokers: " +
replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(","))
if(isrInfo.size < isr.size)
throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " +
isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(","))
new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError)
} catch {
case e =>
error("Error while fetching metadata for partition %s".format(topicAndPartition), e)
new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo,
ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
}
} catch {
case e => error("Error while retrieving topic metadata", e)
}
case _ =>
error("Error while fetching topic metadata for topic " + topicAndMetadata.topic,
ErrorMapping.exceptionFor(topicAndMetadata.errorCode).getCause)
topicsMetadata += topicAndMetadata
new TopicMetadata(topic, partitionMetadata)
} else {
// topic doesn't exist, send appropriate error code
new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode)
}
}
})
trace("Sending topic metadata for correlation id %d to client %s".format(metadataRequest.correlationId, metadataRequest.clientId))
topicsMetadata.foreach(metadata => trace("Sending topic metadata " + metadata.toString))
}
// handle auto create topics
topicMetadataList.foreach { topicMetadata =>
topicMetadata.errorCode match {
case ErrorMapping.NoError => topicsMetadata += topicMetadata
case ErrorMapping.UnknownTopicOrPartitionCode =>
if (config.autoCreateTopicsEnable) {
try {
AdminUtils.createTopic(zkClient, topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor)
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
.format(topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor))
} catch {
case e: TopicExistsException => // let it go, possibly another broker created this topic
}
topicsMetadata += new TopicMetadata(topicMetadata.topic, topicMetadata.partitionsMetadata, ErrorMapping.LeaderNotAvailableCode)
} else {
topicsMetadata += topicMetadata
}
case _ =>
debug("Error while fetching topic metadata for topic %s due to %s ".format(topicMetadata.topic,
ErrorMapping.exceptionFor(topicMetadata.errorCode).getClass.getName))
topicsMetadata += topicMetadata
}
}
trace("Sending topic metadata %s for correlation id %d to client %s".format(topicsMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId))
val response = new TopicMetadataResponse(topicsMetadata.toSeq, metadataRequest.correlationId)
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
}

View File

@ -207,9 +207,20 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
/* the purge interval (in number of requests) of the producer request purgatory */
val producerPurgatoryPurgeIntervalRequests = props.getInt("producer.purgatory.purge.interval.requests", 10000)
/*********** Controlled shutdown configuration ***********/
/** Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens */
val controlledShutdownMaxRetries = props.getInt("controlled.shutdown.max.retries", 3)
/** Before each retry, the system needs time to recover from the state that caused the previous failure (Controller
* fail over, replica lag etc). This config determines the amount of time to wait before retrying. */
val controlledShutdownRetryBackoffMs = props.getInt("controlled.shutdown.retry.backoff.ms", 5000)
/* enable controlled shutdown of the server */
val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", false)
/*********** Misc configuration ***********/
/* the maximum size for a metadata entry associated with an offset commit */
val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", 1024)
}
}

View File

@ -33,14 +33,15 @@ class KafkaRequestHandler(id: Int, brokerId: Int, val requestChannel: RequestCha
try {
val req = requestChannel.receiveRequest()
if(req eq RequestChannel.AllDone) {
trace("receives shut down command, shut down".format(brokerId, id))
debug("Kafka request handler %d on broker %d received shut down command".format(
id, brokerId))
return
}
req.dequeueTimeMs = SystemTime.milliseconds
debug("handles request " + req)
trace("Kafka request handler %d on broker %d handling request %s".format(id, brokerId, req))
apis.handle(req)
} catch {
case e: Throwable => error("exception when handling request", e)
case e: Throwable => error("Exception when handling request")
}
}
}
@ -55,12 +56,12 @@ class KafkaRequestHandlerPool(val brokerId: Int,
this.logIdent = "[Kafka Request Handler on Broker " + brokerId + "], "
val threads = new Array[Thread](numThreads)
val runnables = new Array[KafkaRequestHandler](numThreads)
for(i <- 0 until numThreads) {
for(i <- 0 until numThreads) {
runnables(i) = new KafkaRequestHandler(i, brokerId, requestChannel, apis)
threads(i) = Utils.daemonThread("kafka-request-handler-" + i, runnables(i))
threads(i).start()
}
def shutdown() {
info("shutting down")
for(handler <- runnables)

View File

@ -17,17 +17,20 @@
package kafka.server
import kafka.network.SocketServer
import kafka.admin._
import kafka.log.LogConfig
import kafka.log.CleanerConfig
import kafka.log.LogManager
import kafka.utils._
import java.util.concurrent._
import atomic.{AtomicInteger, AtomicBoolean}
import java.io.File
import atomic.AtomicBoolean
import org.I0Itec.zkclient.ZkClient
import kafka.controller.{ControllerStats, KafkaController}
import kafka.cluster.Broker
import kafka.api.{ControlledShutdownResponse, ControlledShutdownRequest}
import kafka.common.ErrorMapping
import kafka.network.{Receive, BlockingChannel, SocketServer}
/**
* Represents the lifecycle of a single Kafka broker. Handles all functionality required
@ -37,6 +40,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
this.logIdent = "[Kafka Server " + config.brokerId + "], "
private var isShuttingDown = new AtomicBoolean(false)
private var shutdownLatch = new CountDownLatch(1)
private var startupComplete = new AtomicBoolean(false);
val correlationId: AtomicInteger = new AtomicInteger(0)
var socketServer: SocketServer = null
var requestHandlerPool: KafkaRequestHandlerPool = null
var logManager: LogManager = null
@ -77,11 +82,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
config.socketRequestMaxBytes)
socketServer.startup()
replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager)
replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown)
kafkaController = new KafkaController(config, zkClient)
/* start processing requests */
apis = new KafkaApis(socketServer.requestChannel, replicaManager, zkClient, config.brokerId, config)
apis = new KafkaApis(socketServer.requestChannel, replicaManager, zkClient, config.brokerId, config, kafkaController)
requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads)
Mx4jLoader.maybeLoad()
@ -99,7 +104,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
registerStats()
startupComplete.set(true);
info("started")
}
@ -119,6 +124,91 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
ControllerStats.leaderElectionTimer
}
/**
* Performs controlled shutdown
*/
private def controlledShutdown() {
if (startupComplete.get() && config.controlledShutdownEnable) {
// We request the controller to do a controlled shutdown. On failure, we backoff for a configured period
// of time and try again for a configured number of retries. If all the attempt fails, we simply force
// the shutdown.
var remainingRetries = config.controlledShutdownMaxRetries
info("Starting controlled shutdown")
var channel : BlockingChannel = null;
var prevController : Broker = null
var shutdownSuceeded : Boolean =false
try {
while (!shutdownSuceeded && remainingRetries > 0) {
remainingRetries = remainingRetries - 1
// 1. Find the controller and establish a connection to it.
// Get the current controller info. This is to ensure we use the most recent info to issue the
// controlled shutdown request
val controllerId = ZkUtils.getController(zkClient)
ZkUtils.getBrokerInfo(zkClient, controllerId) match {
case Some(broker) =>
if (channel == null || prevController == null || !prevController.equals(broker)) {
// if this is the first attempt or if the controller has changed, create a channel to the most recent
// controller
if (channel != null) {
channel.disconnect()
}
channel = new BlockingChannel(broker.host, broker.port,
BlockingChannel.UseDefaultBufferSize,
BlockingChannel.UseDefaultBufferSize,
config.controllerSocketTimeoutMs)
channel.connect()
prevController = broker
}
case None=>
//ignore and try again
}
// 2. issue a controlled shutdown to the controller
if (channel != null) {
var response: Receive = null
try {
// send the controlled shutdown request
val request = new ControlledShutdownRequest(correlationId.getAndIncrement, config.brokerId)
channel.send(request)
response = channel.receive()
val shutdownResponse = ControlledShutdownResponse.readFrom(response.buffer)
if (shutdownResponse.errorCode == ErrorMapping.NoError && shutdownResponse.partitionsRemaining != null &&
shutdownResponse.partitionsRemaining.size == 0) {
shutdownSuceeded = true
info ("Controlled shutdown succeeded")
}
else {
info("Remaining partitions to move: %s".format(shutdownResponse.partitionsRemaining.mkString(",")))
info("Error code from controller: %d".format(shutdownResponse.errorCode))
}
}
catch {
case ioe: java.io.IOException =>
channel.disconnect()
channel = null
// ignore and try again
}
}
if (!shutdownSuceeded) {
Thread.sleep(config.controlledShutdownRetryBackoffMs)
warn("Retrying controlled shutdown after the previous attempt failed...")
}
}
}
finally {
if (channel != null) {
channel.disconnect()
channel = null
}
}
if (!shutdownSuceeded) {
warn("Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed")
}
}
}
/**
* Shutdown API for shutting down a single instance of the Kafka server.
* Shuts down the LogManager, the SocketServer and the log cleaner scheduler thread
@ -127,6 +217,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
info("shutting down")
val canShutdown = isShuttingDown.compareAndSet(false, true);
if (canShutdown) {
Utils.swallow(controlledShutdown())
if(socketServer != null)
Utils.swallow(socketServer.shutdown())
if(requestHandlerPool != null)
@ -144,6 +235,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
Utils.swallow(zkClient.close())
shutdownLatch.countDown()
startupComplete.set(false);
info("shut down completed")
}
}

View File

@ -53,8 +53,14 @@ class OffsetCheckpoint(val file: File) extends Logging {
// flush and overwrite old file
writer.flush()
if(!temp.renameTo(file))
throw new IOException("File rename from %s to %s failed.".format(temp.getAbsolutePath, file.getAbsolutePath))
// swap new offset checkpoint file with previous one
if(!temp.renameTo(file)) {
// renameTo() fails on Windows if the destination file exists.
file.delete()
if(!temp.renameTo(file)) {
throw new IOException("File rename from %s to %s failed.".format(temp.getAbsolutePath, file.getAbsolutePath))
}
}
} finally {
writer.close()
}

View File

@ -20,7 +20,8 @@ package kafka.server
import kafka.cluster.Broker
class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager)
extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId, brokerConfig.numReplicaFetchers) {
extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId,
"Replica", brokerConfig.numReplicaFetchers) {
override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = {
new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d".format(fetcherId, sourceBroker.id), sourceBroker, brokerConfig, replicaMgr)

View File

@ -19,8 +19,8 @@ package kafka.server
import kafka.cluster.Broker
import kafka.message.ByteBufferMessageSet
import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest, FetchResponsePartitionData}
import kafka.common.{KafkaStorageException, TopicAndPartition, ErrorMapping}
import kafka.api.{OffsetRequest, FetchResponsePartitionData}
import kafka.common.{KafkaStorageException, TopicAndPartition}
class ReplicaFetcherThread(name:String,
sourceBroker: Broker,

View File

@ -42,7 +42,8 @@ class ReplicaManager(val config: KafkaConfig,
time: Time,
val zkClient: ZkClient,
scheduler: Scheduler,
val logManager: LogManager) extends Logging with KafkaMetricsGroup {
val logManager: LogManager,
val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup {
/* epoch of the controller that last changed the leader */
@volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1
private val localBrokerId = config.brokerId
@ -54,7 +55,7 @@ class ReplicaManager(val config: KafkaConfig,
val highWatermarkCheckpoints = config.logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap
private var hwThreadInitialized = false
this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: "
private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger)
val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger)
newGauge(
"LeaderCount",
@ -283,9 +284,11 @@ class ReplicaManager(val config: KafkaConfig,
private def maybeShrinkIsr(): Unit = {
trace("Evaluating ISR list of partitions to see which replicas can be removed from the ISR")
var curLeaderPartitions: List[Partition] = null
leaderPartitionsLock synchronized {
leaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages))
curLeaderPartitions = leaderPartitions.toList
}
curLeaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages))
}
def recordFollowerPosition(topic: String, partitionId: Int, replicaId: Int, offset: Long) = {

View File

@ -74,7 +74,6 @@ object ConsumerOffsetChecker extends Logging {
val lag = logSize - offset
println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offset, logSize, lag,
owner match {case Some(ownerStr) => ownerStr case None => "none"}))
consumer.close()
case None => // ignore
}
case None =>
@ -157,6 +156,11 @@ object ConsumerOffsetChecker extends Logging {
if (options.has("broker-info"))
printBrokerInfo();
for ((_, consumerOpt) <- consumerMap)
consumerOpt match {
case Some(consumer) => consumer.close()
case None => // ignore
}
}
finally {
for (consumerOpt <- consumerMap.values) {

View File

@ -28,7 +28,6 @@ import collection.mutable.ListBuffer
import kafka.tools.KafkaMigrationTool.{ProducerThread, ProducerDataChannel}
import kafka.javaapi
object MirrorMaker extends Logging {
def main(args: Array[String]) {
@ -114,23 +113,33 @@ object MirrorMaker extends Logging {
else
new Blacklist(options.valueOf(blacklistOpt))
val streams =
connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue(), new DefaultDecoder(), new DefaultDecoder()))
var streams: Seq[KafkaStream[Array[Byte], Array[Byte]]] = Nil
try {
streams = connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue(), new DefaultDecoder(), new DefaultDecoder())).flatten
} catch {
case t =>
fatal("Unable to create stream - shutting down mirror maker.")
connectors.foreach(_.shutdown)
}
val producerDataChannel = new ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]](bufferSize);
val consumerThreads =
streams.flatten.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, streamAndIndex._2))
streams.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, streamAndIndex._2))
val producerThreads = new ListBuffer[ProducerThread]()
def cleanShutdown() {
connectors.foreach(_.shutdown)
consumerThreads.foreach(_.awaitShutdown)
producerThreads.foreach(_.shutdown)
producerThreads.foreach(_.awaitShutdown)
info("Kafka mirror maker shutdown successfully")
}
Runtime.getRuntime.addShutdownHook(new Thread() {
override def run() {
connectors.foreach(_.shutdown)
consumerThreads.foreach(_.awaitShutdown)
producerThreads.foreach(_.shutdown)
producerThreads.foreach(_.awaitShutdown)
logger.info("Kafka migration tool shutdown successfully");
cleanShutdown()
}
})
@ -145,6 +154,10 @@ object MirrorMaker extends Logging {
consumerThreads.foreach(_.start)
producerThreads.foreach(_.start)
// in case the consumer threads hit a timeout/other exception
consumerThreads.foreach(_.awaitShutdown)
cleanShutdown()
}
class MirrorMakerThread(stream: KafkaStream[Array[Byte], Array[Byte]],
@ -158,6 +171,7 @@ object MirrorMaker extends Logging {
this.setName(threadName)
override def run() {
info("Starting mirror maker thread " + threadName)
try {
for (msgAndMetadata <- stream) {
val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.message)

View File

@ -1,4 +1,20 @@
package kafka.utils
/**
* 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.utils
import joptsimple.{OptionSpec, OptionSet, OptionParser}

View File

@ -37,7 +37,7 @@ class DelayedItem[T](val item: T, delay: Long, unit: TimeUnit) extends Delayed w
*/
def getDelay(unit: TimeUnit): Long = {
val elapsedMs = (SystemTime.milliseconds - createdMs)
unit.convert(max(delayMs - elapsedMs, 0), unit)
unit.convert(max(delayMs - elapsedMs, 0), TimeUnit.MILLISECONDS)
}
def compareTo(d: Delayed): Int = {

View File

@ -1,4 +1,20 @@
package kafka.utils
/**
* 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.utils
import java.io._
import java.nio.channels._

View File

@ -1,4 +1,20 @@
package kafka.utils
/**
* 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.utils
import kafka.common._
import scala.collection._

View File

@ -305,24 +305,21 @@ object ZkUtils extends Logging {
* create parrent directory if necessary. Never throw NodeExistException.
* Return the updated path zkVersion
*/
def updatePersistentPath(client: ZkClient, path: String, data: String): Int = {
var stat: Stat = null
def updatePersistentPath(client: ZkClient, path: String, data: String) = {
try {
stat = client.writeData(path, data)
return stat.getVersion
client.writeData(path, data)
} catch {
case e: ZkNoNodeException => {
createParentPath(client, path)
try {
client.createPersistent(path, data)
// When the new path is created, its zkVersion always starts from 0
return 0
} catch {
case e: ZkNodeExistsException =>
stat = client.writeData(path, data)
return stat.getVersion
client.writeData(path, data)
case e2 => throw e2
}
}
case e2 => throw e2
}
}
@ -332,14 +329,14 @@ object ZkUtils extends Logging {
*/
def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = {
try {
val stat = client.writeData(path, data, expectVersion)
val stat = client.writeDataReturnStat(path, data, expectVersion)
debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d"
.format(path, data, expectVersion, stat.getVersion))
(true, stat.getVersion)
} catch {
case e: Exception =>
error("Conditional update of path %s with data %s and expected version %d failed".format(path, data,
expectVersion), e)
error("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data,
expectVersion, e.getMessage))
(false, -1)
}
}
@ -350,15 +347,15 @@ object ZkUtils extends Logging {
*/
def conditionalUpdatePersistentPathIfExists(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = {
try {
val stat = client.writeData(path, data, expectVersion)
val stat = client.writeDataReturnStat(path, data, expectVersion)
debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d"
.format(path, data, expectVersion, stat.getVersion))
(true, stat.getVersion)
} catch {
case nne: ZkNoNodeException => throw nne
case e: Exception =>
error("Conditional update of path %s with data %s and expected version %d failed".format(path, data,
expectVersion), e)
error("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data,
expectVersion, e.getMessage))
(false, -1)
}
}
@ -466,15 +463,13 @@ object ZkUtils extends Logging {
cluster
}
def getPartitionLeaderAndIsrForTopics(zkClient: ZkClient, topics: Seq[String]): mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = {
def getPartitionLeaderAndIsrForTopics(zkClient: ZkClient, topicAndPartitions: Set[TopicAndPartition])
: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = {
val ret = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch]
val partitionsForTopics = getPartitionsForTopics(zkClient, topics)
for((topic, partitions) <- partitionsForTopics) {
for(partition <- partitions) {
ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition.toInt) match {
case Some(leaderIsrAndControllerEpoch) => ret.put(TopicAndPartition(topic, partition.toInt), leaderIsrAndControllerEpoch)
case None =>
}
for(topicAndPartition <- topicAndPartitions) {
ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition) match {
case Some(leaderIsrAndControllerEpoch) => ret.put(topicAndPartition, leaderIsrAndControllerEpoch)
case None =>
}
}
ret
@ -777,7 +772,7 @@ class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group)
class ZKConfig(props: VerifiableProperties) {
/** ZK host string */
val zkConnect = props.getString("zookeeper.connect", null)
val zkConnect = props.getString("zookeeper.connect")
/** zookeeper session timeout */
val zkSessionTimeoutMs = props.getInt("zookeeper.session.timeout.ms", 6000)

View File

@ -121,8 +121,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment)
// create leaders for all partitions
TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1)
val actualReplicaAssignment = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata.map(p => p.replicas)
val actualReplicaList = actualReplicaAssignment.map(r => r.map(b => b.id).toList).toList
val actualReplicaList = leaderForPartitionMap.keys.toArray.map(p => (p -> ZkUtils.getReplicasForPartition(zkClient, topic, p))).toMap
assertEquals(expectedReplicaAssignment.size, actualReplicaList.size)
for(i <- 0 until actualReplicaList.size)
assertEquals(expectedReplicaAssignment.get(i).get, actualReplicaList(i))
@ -133,37 +132,6 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
}
}
@Test
def testGetTopicMetadata() {
val expectedReplicaAssignment = Map(
0 -> List(0, 1, 2),
1 -> List(1, 2, 3))
val leaderForPartitionMap = Map(
0 -> 0,
1 -> 1)
val topic = "auto-topic"
TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3))
AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment)
// create leaders for all partitions
TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1)
val newTopicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
newTopicMetadata.errorCode match {
case ErrorMapping.UnknownTopicOrPartitionCode =>
fail("Topic " + topic + " should've been automatically created")
case _ =>
assertEquals(topic, newTopicMetadata.topic)
assertNotNull("partition metadata list cannot be null", newTopicMetadata.partitionsMetadata)
assertEquals("partition metadata list length should be 2", 2, newTopicMetadata.partitionsMetadata.size)
val actualReplicaAssignment = newTopicMetadata.partitionsMetadata.map(p => p.replicas)
val actualReplicaList = actualReplicaAssignment.map(r => r.map(b => b.id).toList).toList
assertEquals(expectedReplicaAssignment.size, actualReplicaList.size)
for(i <- 0 until actualReplicaList.size) {
assertEquals(expectedReplicaAssignment(i), actualReplicaList(i))
}
}
}
@Test
def testPartitionReassignmentWithLeaderInNewReplicas() {
val expectedReplicaAssignment = Map(0 -> List(0, 1, 2))
@ -234,7 +202,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas);
CheckReassignmentStatus.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas,
Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted;
}, 1000)
}, 2000)
val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned)
assertEquals("Partition should have been reassigned to 2, 3", newReplicas, assignedReplicas)
// leader should be 2
@ -316,48 +284,45 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
@Test
def testShutdownBroker() {
info("inside testShutdownBroker")
val expectedReplicaAssignment = Map(1 -> List(0, 1, 2))
val topic = "test"
val partition = 1
// create brokers
val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_))
val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s))
// create the topic
AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment)
val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s))
TestUtils.waitUntilMetadataIsPropagated(servers, topic, partition, 1000)
// broker 2 should be the leader since it was started first
var leaderBeforeShutdown = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, 1000, None).get
var controllerId = ZkUtils.getController(zkClient)
var controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
val controllerId = ZkUtils.getController(zkClient)
val controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
var partitionsRemaining = controller.shutdownBroker(2)
var activeServers = servers.filter(s => s.config.brokerId != 2)
try {
assertEquals(0, partitionsRemaining)
var topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
var leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
assertTrue(leaderAfterShutdown != leaderBeforeShutdown)
assertEquals(2, controller.controllerContext.partitionLeadershipInfo(TopicAndPartition("test", 1)).leaderAndIsr.isr.size)
// wait for the update metadata request to trickle to the brokers
assertTrue("Topic test not created after timeout", TestUtils.waitUntilTrue(() =>
activeServers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.isr.size != 3), 1000))
assertEquals(0, partitionsRemaining.size)
var partitionStateInfo = activeServers.head.apis.leaderCache(TopicAndPartition(topic, partition))
var leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader
assertEquals(0, leaderAfterShutdown)
assertEquals(2, partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.isr.size)
assertEquals(List(0,1), partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.isr)
leaderBeforeShutdown = leaderAfterShutdown
controllerId = ZkUtils.getController(zkClient)
controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
partitionsRemaining = controller.shutdownBroker(1)
assertEquals(0, partitionsRemaining)
topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
assertTrue(leaderAfterShutdown != leaderBeforeShutdown)
assertEquals(1, controller.controllerContext.partitionLeadershipInfo(TopicAndPartition("test", 1)).leaderAndIsr.isr.size)
assertEquals(0, partitionsRemaining.size)
activeServers = servers.filter(s => s.config.brokerId == 0)
partitionStateInfo = activeServers.head.apis.leaderCache(TopicAndPartition(topic, partition))
leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader
assertEquals(0, leaderAfterShutdown)
leaderBeforeShutdown = leaderAfterShutdown
controllerId = ZkUtils.getController(zkClient)
controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
assertTrue(servers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0))
partitionsRemaining = controller.shutdownBroker(0)
assertEquals(1, partitionsRemaining)
topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
assertTrue(leaderAfterShutdown == leaderBeforeShutdown)
assertEquals(1, controller.controllerContext.partitionLeadershipInfo(TopicAndPartition("test", 1)).leaderAndIsr.isr.size)
} finally {
assertEquals(1, partitionsRemaining.size)
// leader doesn't change since all the replicas are shut down
assertTrue(servers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0))
}
finally {
servers.foreach(_.shutdown())
}
}

View File

@ -87,8 +87,8 @@ object SerializationTestUtils{
def createTestLeaderAndIsrRequest() : LeaderAndIsrRequest = {
val leaderAndIsr1 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader1, 1, isr1, 1), 1)
val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1)
val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, 3)),
((topic2, 0), PartitionStateInfo(leaderAndIsr2, 3)))
val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, isr1.toSet)),
((topic2, 0), PartitionStateInfo(leaderAndIsr2, isr2.toSet)))
new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 0, 1, 0, "")
}

View File

@ -23,7 +23,6 @@ import kafka.integration.KafkaServerTestHarness
import kafka.server._
import scala.collection._
import org.scalatest.junit.JUnit3Suite
import org.apache.log4j.{Level, Logger}
import kafka.message._
import kafka.serializer._
import kafka.admin.AdminUtils
@ -31,6 +30,7 @@ import org.I0Itec.zkclient.ZkClient
import kafka.utils._
import kafka.producer.{ProducerConfig, KeyedMessage, Producer}
import java.util.{Collections, Properties}
import org.apache.log4j.{Logger, Level}
import kafka.utils.TestUtils._
class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging {
@ -97,6 +97,9 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000)
// create a consumer
val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1))
val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true)
@ -142,7 +145,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true)
val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]())
// send some messages to each broker
val sentMessages3 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++
val sentMessages3 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++
sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages)
waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
@ -167,12 +170,15 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
requestHandlerLogger.setLevel(Level.FATAL)
// send some messages to each broker
val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++
val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++
sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec)
waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000)
// create a consumer
val consumerConfig1 = new ConsumerConfig(
TestUtils.createConsumerProperties(zkConnect, group, consumer1))
@ -240,9 +246,12 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
def testCompressionSetConsumption() {
// send some messages to each broker
val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, 200, DefaultCompressionCodec) ++
val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, 200, DefaultCompressionCodec) ++
sendMessagesToBrokerPartition(configs.last, topic, 1, 200, DefaultCompressionCodec)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000)
val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0))
val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true)
val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder())
@ -263,9 +272,12 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
requestHandlerLogger.setLevel(Level.FATAL)
// send some messages to each broker
val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, NoCompressionCodec) ++
val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, NoCompressionCodec) ++
sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, NoCompressionCodec)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000)
val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1))
waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
@ -303,6 +315,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
// send some messages to each broker
val sentMessages1 = sendMessages(configs.head, nMessages, "batch1", NoCompressionCodec, 1)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
// create a consumer
val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1))
val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true)
@ -321,12 +335,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val receivedMessages1 = getMessages(nMessages, topicMessageStreams1)
assertEquals(sentMessages1, receivedMessages1)
zkConsumerConnector1.shutdown()
zkClient.close()
}
def sendMessagesToBrokerPartition(config: KafkaConfig,
topic: String,
partition: Int,
numMessages: Int,
def sendMessagesToBrokerPartition(config: KafkaConfig,
topic: String,
partition: Int,
numMessages: Int,
compression: CompressionCodec = NoCompressionCodec): List[String] = {
val header = "test-%d-%d".format(config.brokerId, partition)
val props = new Properties()

View File

@ -56,7 +56,7 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L
def testResetToEarliestWhenOffsetTooLow() =
assertEquals(NumMessages, resetAndConsume(NumMessages, "smallest", SmallOffset))
def testResetToLatestWhenOffsetTooHigh() =
assertEquals(0, resetAndConsume(NumMessages, "largest", LargeOffset))
@ -68,12 +68,16 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L
* Returns the count of messages received.
*/
def resetAndConsume(numMessages: Int, resetTo: String, offset: Long): Int = {
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
val producer: Producer[String, Array[Byte]] = TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs),
new DefaultEncoder(), new StringEncoder())
for(i <- 0 until numMessages)
producer.send(new KeyedMessage[String, Array[Byte]](topic, topic, "test".getBytes))
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
// update offset in zookeeper for consumer to jump "forward" in time
val dirs = new ZKGroupTopicDirs(group, topic)
var consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer)
@ -98,8 +102,10 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L
} catch {
case e: ConsumerTimeoutException =>
info("consumer timed out after receiving " + received + " messages.")
} finally {
producer.close()
consumerConnector.shutdown
}
consumerConnector.shutdown
received
}

View File

@ -21,12 +21,12 @@ import kafka.api.FetchRequestBuilder
import kafka.message.ByteBufferMessageSet
import kafka.server.{KafkaRequestHandler, KafkaConfig}
import org.apache.log4j.{Level, Logger}
import org.junit.Assert._
import org.scalatest.junit.JUnit3Suite
import scala.collection._
import kafka.producer.KeyedMessage
import kafka.utils._
import kafka.common.{ErrorMapping, KafkaException, OffsetOutOfRangeException}
import kafka.producer.KeyedMessage
import org.junit.Assert.assertEquals
/**
* End to end tests of the primitive apis against a local server
@ -63,6 +63,8 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
producer.send(producerData:_*)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
var fetchedMessage: ByteBufferMessageSet = null
while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
@ -90,6 +92,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
val producedData = List("a_" + topic, "b_" + topic)
messages += topic -> producedData
producer.send(producedData.map(m => new KeyedMessage[String, String](topic, topic, m)):_*)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
builder.addFetch(topic, offset, 0, 10000)
}
@ -132,6 +135,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
builder.addFetch(topic, 0, 0, 10000)
}
producer.send(produceList: _*)
topics.foreach(topic => TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000))
// wait a bit for produced message to be available
val request = builder.build()
@ -155,6 +159,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
builder.addFetch(topic, 0, 0, 10000)
}
producer.send(produceList: _*)
topics.foreach(topic => TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000))
producer.send(produceList: _*)
// wait a bit for produced message to be available

View File

@ -35,7 +35,7 @@ import kafka.utils.{TestUtils, Utils}
* End to end tests of the primitive apis against a local server
*/
class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness {
val port = TestUtils.choosePort
val props = TestUtils.createBrokerConfig(0, port)
val config = new KafkaConfig(props)
@ -287,8 +287,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
def testConsumerEmptyTopic() {
val newTopic = "new-topic"
AdminUtils.createTopic(zkClient, newTopic, 1, 1)
assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() =>
AdminUtils.fetchTopicMetadataFromZk(newTopic, zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime))
TestUtils.waitUntilMetadataIsPropagated(servers, newTopic, 0, 1000)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, newTopic, 0, 500)
val fetchResponse = consumer.fetch(new FetchRequestBuilder().addFetch(newTopic, 0, 0, 10000).build())
assertFalse(fetchResponse.messageSet(newTopic, 0).iterator.hasNext)

View File

@ -0,0 +1,139 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
import org.scalatest.junit.JUnit3Suite
import kafka.zk.ZooKeeperTestHarness
import kafka.utils.TestUtils._
import junit.framework.Assert._
import kafka.utils.{ZkUtils, Utils, TestUtils}
import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager}
import kafka.cluster.Broker
import kafka.common.ErrorMapping
import kafka.api._
import kafka.admin.AdminUtils
class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness {
val brokerId1 = 0
val brokerId2 = 1
val brokerId3 = 2
val brokerId4 = 3
val port1 = TestUtils.choosePort()
val port2 = TestUtils.choosePort()
val port3 = TestUtils.choosePort()
val port4 = TestUtils.choosePort()
val enableShutdown = true
val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1)
configProps1.put("controlled.shutdown.enable", "true")
val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2)
configProps2.put("controlled.shutdown.enable", "true")
val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3)
configProps3.put("controlled.shutdown.enable", "true")
val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4)
configProps4.put("controlled.shutdown.enable", "true")
configProps4.put("controlled.shutdown.retry.backoff.ms", "100")
var servers: Seq[KafkaServer] = Seq.empty[KafkaServer]
val partitionId = 0
override def setUp() {
super.setUp()
// start all the servers
val server1 = TestUtils.createServer(new KafkaConfig(configProps1))
val server2 = TestUtils.createServer(new KafkaConfig(configProps2))
val server3 = TestUtils.createServer(new KafkaConfig(configProps3))
val server4 = TestUtils.createServer(new KafkaConfig(configProps4))
servers ++= List(server1, server2, server3, server4)
}
override def tearDown() {
servers.map(server => server.shutdown())
servers.map(server => Utils.rm(server.config.logDirs))
super.tearDown()
}
def testRollingBounce {
// start all the brokers
val topic1 = "new-topic1"
val topic2 = "new-topic2"
val topic3 = "new-topic3"
val topic4 = "new-topic4"
// create topics with 1 partition, 2 replicas, one on each broker
AdminUtils.createTopicWithAssignment(zkClient, topic1, Map(0->Seq(0,1)))
AdminUtils.createTopicWithAssignment(zkClient, topic2, Map(0->Seq(1,2)))
AdminUtils.createTopicWithAssignment(zkClient, topic3, Map(0->Seq(2,3)))
AdminUtils.createTopicWithAssignment(zkClient, topic4, Map(0->Seq(0,3)))
// wait until leader is elected
var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partitionId, 500)
var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, partitionId, 500)
var leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, partitionId, 500)
var leader4 = waitUntilLeaderIsElectedOrChanged(zkClient, topic4, partitionId, 500)
debug("Leader for " + topic1 + " is elected to be: %s".format(leader1.getOrElse(-1)))
debug("Leader for " + topic2 + " is elected to be: %s".format(leader1.getOrElse(-1)))
debug("Leader for " + topic3 + "is elected to be: %s".format(leader1.getOrElse(-1)))
debug("Leader for " + topic4 + "is elected to be: %s".format(leader1.getOrElse(-1)))
assertTrue("Leader should get elected", leader1.isDefined)
assertTrue("Leader should get elected", leader2.isDefined)
assertTrue("Leader should get elected", leader3.isDefined)
assertTrue("Leader should get elected", leader4.isDefined)
assertTrue("Leader could be broker 0 or broker 1 for " + topic1, (leader1.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 1))
assertTrue("Leader could be broker 1 or broker 2 for " + topic2, (leader2.getOrElse(-1) == 1) || (leader1.getOrElse(-1) == 2))
assertTrue("Leader could be broker 2 or broker 3 for " + topic3, (leader3.getOrElse(-1) == 2) || (leader1.getOrElse(-1) == 3))
assertTrue("Leader could be broker 3 or broker 4 for " + topic4, (leader4.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 3))
// Do a rolling bounce and check if leader transitions happen correctly
// Bring down the leader for the first topic
bounceServer(topic1, 0)
// Bring down the leader for the second topic
bounceServer(topic2, 1)
// Bring down the leader for the third topic
bounceServer(topic3, 2)
// Bring down the leader for the fourth topic
bounceServer(topic4, 3)
}
private def bounceServer(topic: String, startIndex: Int) {
var prevLeader = 0
if (isLeaderLocalOnBroker(topic, partitionId, servers(startIndex))) {
servers(startIndex).shutdown()
prevLeader = startIndex
}
else {
servers((startIndex + 1) % 4).shutdown()
prevLeader = (startIndex + 1) % 4
}
var newleader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 1500)
// Ensure the new leader is different from the old
assertTrue("Leader transition did not happen for " + topic, newleader.getOrElse(-1) != -1 && (newleader.getOrElse(-1) != prevLeader))
// Start the server back up again
servers(prevLeader).startup()
}
}

View File

@ -22,26 +22,27 @@ import kafka.zk.ZooKeeperTestHarness
import kafka.admin.AdminUtils
import java.nio.ByteBuffer
import junit.framework.Assert._
import org.easymock.EasyMock
import kafka.network._
import kafka.cluster.Broker
import kafka.utils.TestUtils
import kafka.utils.TestUtils._
import kafka.server.{ReplicaManager, KafkaApis, KafkaConfig}
import kafka.server.{KafkaServer, KafkaConfig}
import kafka.api.TopicMetadataRequest
import kafka.common.ErrorMapping
import kafka.api.{RequestKeys, TopicMetadata, TopicMetadataResponse, TopicMetadataRequest}
import kafka.client.ClientUtils
class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
val props = createBrokerConfigs(1)
val configs = props.map(p => new KafkaConfig(p))
var brokers: Seq[Broker] = null
private var server1: KafkaServer = null
val brokers = configs.map(c => new Broker(c.brokerId,c.hostName,c.port))
override def setUp() {
super.setUp()
brokers = TestUtils.createBrokersInZk(zkClient, configs.map(config => config.brokerId))
server1 = TestUtils.createServer(configs.head)
}
override def tearDown() {
server1.shutdown()
super.tearDown()
}
@ -65,16 +66,15 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
// create topic
val topic = "test"
AdminUtils.createTopic(zkClient, topic, 1, 1)
// set up leader for topic partition 0
val leaderForPartitionMap = Map(
0 -> configs.head.brokerId
)
TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1)
val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0)
val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest)
assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size)
assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic)
val partitionMetadata = topicMetadata.head.partitionsMetadata
TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0, 1000)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata",
2000,0).topicsMetadata
assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode)
assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode)
assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size)
assertEquals("Expecting metadata for the test topic", "test", topicsMetadata.head.topic)
var partitionMetadata = topicsMetadata.head.partitionsMetadata
assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size)
assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId)
assertEquals(1, partitionMetadata.head.replicas.size)
@ -82,60 +82,55 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
def testGetAllTopicMetadata {
// create topic
val topic = "test"
AdminUtils.createTopic(zkClient, topic, 1, 1)
// set up leader for topic partition 0
val leaderForPartitionMap = Map(
0 -> configs.head.brokerId
)
TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1)
val topicMetadataRequest = new TopicMetadataRequest(List(), 0)
val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest)
assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size)
assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic)
val partitionMetadata = topicMetadata.head.partitionsMetadata
assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size)
assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId)
assertEquals(1, partitionMetadata.head.replicas.size)
val topic1 = "testGetAllTopicMetadata1"
val topic2 = "testGetAllTopicMetadata2"
AdminUtils.createTopic(zkClient, topic1, 1, 1)
AdminUtils.createTopic(zkClient, topic2, 1, 1)
// wait for leader to be elected for both topics
TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic1, 0, 1000)
TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic2, 0, 1000)
// issue metadata request with empty list of topics
var topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokers, "TopicMetadataTest-testGetAllTopicMetadata",
2000, 0).topicsMetadata
assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode)
assertEquals(2, topicsMetadata.size)
assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode)
assertEquals(ErrorMapping.NoError, topicsMetadata.last.partitionsMetadata.head.errorCode)
val partitionMetadataTopic1 = topicsMetadata.head.partitionsMetadata
val partitionMetadataTopic2 = topicsMetadata.last.partitionsMetadata
assertEquals("Expecting metadata for 1 partition", 1, partitionMetadataTopic1.size)
assertEquals("Expecting partition id to be 0", 0, partitionMetadataTopic1.head.partitionId)
assertEquals(1, partitionMetadataTopic1.head.replicas.size)
assertEquals("Expecting metadata for 1 partition", 1, partitionMetadataTopic2.size)
assertEquals("Expecting partition id to be 0", 0, partitionMetadataTopic2.head.partitionId)
assertEquals(1, partitionMetadataTopic2.head.replicas.size)
}
def testAutoCreateTopic {
// auto create topic
val topic = "test"
val topic = "testAutoCreateTopic"
var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testAutoCreateTopic",
2000,0).topicsMetadata
assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode)
assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size)
assertEquals("Expecting metadata for the test topic", topic, topicsMetadata.head.topic)
assertEquals(0, topicsMetadata.head.partitionsMetadata.size)
val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0)
val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest)
assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size)
assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic)
val partitionMetadata = topicMetadata.head.partitionsMetadata
// wait for leader to be elected
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0, 1000)
// retry the metadata for the auto created topic
topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata",
2000,0).topicsMetadata
assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode)
assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode)
var partitionMetadata = topicsMetadata.head.partitionsMetadata
assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size)
assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId)
assertEquals(0, partitionMetadata.head.replicas.size)
assertEquals(None, partitionMetadata.head.leader)
assertEquals(ErrorMapping.LeaderNotAvailableCode, partitionMetadata.head.errorCode)
}
private def mockLogManagerAndTestTopic(request: TopicMetadataRequest): Seq[TopicMetadata] = {
// topic metadata request only requires 1 call from the replica manager
val replicaManager = EasyMock.createMock(classOf[ReplicaManager])
EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes()
EasyMock.replay(replicaManager)
val serializedMetadataRequest = TestUtils.createRequestByteBuffer(request)
// create the kafka request handler
val requestChannel = new RequestChannel(2, 5)
val apis = new KafkaApis(requestChannel, replicaManager, zkClient, 1, configs.head)
// call the API (to be tested) to get metadata
apis.handleTopicMetadataRequest(new RequestChannel.Request
(processor=0, requestKey=RequestKeys.MetadataKey, buffer=serializedMetadataRequest, startTimeMs=1))
val metadataResponse = requestChannel.receiveResponse(0).responseSend.asInstanceOf[BoundedByteBufferSend].buffer
// check assertions
val topicMetadata = TopicMetadataResponse.readFrom(metadataResponse).topicsMetadata
topicMetadata
assertEquals(1, partitionMetadata.head.replicas.size)
assertTrue(partitionMetadata.head.leader.isDefined)
}
}

View File

@ -1,4 +1,20 @@
package kafka.log
/**
* 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.log
import junit.framework.Assert._
import java.util.concurrent.atomic._

View File

@ -25,6 +25,7 @@ import org.scalatest.junit.JUnitSuite
import scala.collection._
import scala.util.Random
import kafka.utils.TestUtils
import kafka.common.InvalidOffsetException
class OffsetIndexTest extends JUnitSuite {
@ -89,7 +90,7 @@ class OffsetIndexTest extends JUnitSuite {
assertWriteFails("Append should fail on a full index", idx, idx.maxEntries + 1, classOf[IllegalArgumentException])
}
@Test(expected = classOf[IllegalArgumentException])
@Test(expected = classOf[InvalidOffsetException])
def appendOutOfOrder() {
idx.append(51, 0)
idx.append(50, 1)

View File

@ -260,10 +260,11 @@ class AsyncProducerTest extends JUnit3Suite {
topicPartitionInfos = topicPartitionInfos)
try {
handler.partitionAndCollate(producerDataList)
fail("Should fail with UnknownTopicOrPartitionException")
}
catch {
case e: UnknownTopicOrPartitionException => // expected, do nothing
// should not throw any exception
case e => fail("Should not throw any exception")
}
}
@ -291,10 +292,10 @@ class AsyncProducerTest extends JUnit3Suite {
topicPartitionInfos = topicPartitionInfos)
try {
handler.handle(producerDataList)
fail("Should fail with NoBrokersForPartitionException")
fail("Should fail with FailedToSendMessageException")
}
catch {
case e: NoBrokersForPartitionException => // expected, do nothing
case e: FailedToSendMessageException => // we retry on any exception now
}
}
@ -419,6 +420,8 @@ class AsyncProducerTest extends JUnit3Suite {
val response2 = ProducerResponse(0,
Map((TopicAndPartition("topic1", 0), ProducerResponseStatus(ErrorMapping.NoError, 0L))))
val mockSyncProducer = EasyMock.createMock(classOf[SyncProducer])
// don't care about config mock
EasyMock.expect(mockSyncProducer.config).andReturn(EasyMock.anyObject()).anyTimes()
EasyMock.expect(mockSyncProducer.send(request1)).andThrow(new RuntimeException) // simulate SocketTimeoutException
EasyMock.expect(mockSyncProducer.send(request2)).andReturn(response1)
EasyMock.expect(mockSyncProducer.send(request3)).andReturn(response2)

View File

@ -23,14 +23,16 @@ import kafka.message.Message
import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer}
import kafka.zk.ZooKeeperTestHarness
import org.apache.log4j.{Level, Logger}
import org.junit.Assert._
import org.junit.Test
import kafka.utils._
import java.util
import kafka.admin.AdminUtils
import util.Properties
import kafka.api.FetchRequestBuilder
import kafka.common.{KafkaException, ErrorMapping, FailedToSendMessageException}
import org.junit.Assert.assertTrue
import org.junit.Assert.assertFalse
import org.junit.Assert.assertEquals
import kafka.common.{ErrorMapping, FailedToSendMessageException}
class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
@ -43,6 +45,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
private var consumer1: SimpleConsumer = null
private var consumer2: SimpleConsumer = null
private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler])
private var servers = List.empty[KafkaServer]
private val props1 = TestUtils.createBrokerConfig(brokerId1, port1)
private val config1 = new KafkaConfig(props1) {
@ -60,6 +63,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
// set up 2 brokers with 4 partitions each
server1 = TestUtils.createServer(config1)
server2 = TestUtils.createServer(config2)
servers = List(server1,server2)
val props = new Properties()
props.put("host", "localhost")
@ -68,7 +72,6 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
consumer1 = new SimpleConsumer("localhost", port1, 1000000, 64*1024, "")
consumer2 = new SimpleConsumer("localhost", port2, 100, 64*1024, "")
// temporarily set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.FATAL)
}
@ -85,10 +88,11 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
@Test
def testUpdateBrokerPartitionInfo() {
AdminUtils.createTopic(zkClient, "new-topic", 1, 2)
assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() =>
AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime))
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500)
val topic = "new-topic"
AdminUtils.createTopic(zkClient, topic, 1, 2)
// wait until the update metadata request for new topic reaches all servers
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 500)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
val props1 = new util.Properties()
props1.put("metadata.broker.list", "localhost:80,localhost:81")
@ -96,10 +100,10 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
val producerConfig1 = new ProducerConfig(props1)
val producer1 = new Producer[String, String](producerConfig1)
try{
producer1.send(new KeyedMessage[String, String]("new-topic", "test", "test1"))
producer1.send(new KeyedMessage[String, String](topic, "test", "test1"))
fail("Test should fail because the broker list provided are not valid")
} catch {
case e: KafkaException =>
case e: FailedToSendMessageException =>
case oe => fail("fails with exception", oe)
} finally {
producer1.close()
@ -111,7 +115,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
val producerConfig2= new ProducerConfig(props2)
val producer2 = new Producer[String, String](producerConfig2)
try{
producer2.send(new KeyedMessage[String, String]("new-topic", "test", "test1"))
producer2.send(new KeyedMessage[String, String](topic, "test", "test1"))
} catch {
case e => fail("Should succeed sending the message", e)
} finally {
@ -124,7 +128,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
val producerConfig3 = new ProducerConfig(props3)
val producer3 = new Producer[String, String](producerConfig3)
try{
producer3.send(new KeyedMessage[String, String]("new-topic", "test", "test1"))
producer3.send(new KeyedMessage[String, String](topic, "test", "test1"))
} catch {
case e => fail("Should succeed sending the message", e)
} finally {
@ -149,27 +153,27 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
val producerConfig1 = new ProducerConfig(props1)
val producerConfig2 = new ProducerConfig(props2)
val topic = "new-topic"
// create topic with 1 partition and await leadership
AdminUtils.createTopic(zkClient, "new-topic", 1, 2)
assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() =>
AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime))
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500)
AdminUtils.createTopic(zkClient, topic, 1, 2)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
val producer1 = new Producer[String, String](producerConfig1)
val producer2 = new Producer[String, String](producerConfig2)
// Available partition ids should be 0.
producer1.send(new KeyedMessage[String, String]("new-topic", "test", "test1"))
producer1.send(new KeyedMessage[String, String]("new-topic", "test", "test2"))
producer1.send(new KeyedMessage[String, String](topic, "test", "test1"))
producer1.send(new KeyedMessage[String, String](topic, "test", "test2"))
// get the leader
val leaderOpt = ZkUtils.getLeaderForPartition(zkClient, "new-topic", 0)
val leaderOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0)
assertTrue("Leader for topic new-topic partition 0 should exist", leaderOpt.isDefined)
val leader = leaderOpt.get
val messageSet = if(leader == server1.config.brokerId) {
val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
response1.messageSet("new-topic", 0).iterator.toBuffer
}else {
val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
response2.messageSet("new-topic", 0).iterator.toBuffer
}
assertEquals("Should have fetched 2 messages", 2, messageSet.size)
@ -178,7 +182,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
producer1.close()
try {
producer2.send(new KeyedMessage[String, String]("new-topic", "test", "test2"))
producer2.send(new KeyedMessage[String, String](topic, "test", "test2"))
fail("Should have timed out for 3 acks.")
}
catch {
@ -200,19 +204,25 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
props.put("request.required.acks", "1")
props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
val topic = "new-topic"
// create topic
AdminUtils.createTopicWithAssignment(zkClient, "new-topic", Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0)))
assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() =>
AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime))
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 1, 500)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 2, 500)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 3, 500)
AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0)))
// waiting for 1 partition is enough
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 2, 500)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 3, 500)
val config = new ProducerConfig(props)
val producer = new Producer[String, String](config)
// Available partition ids should be 0, 1, 2 and 3, all lead and hosted only on broker 0
producer.send(new KeyedMessage[String, String]("new-topic", "test", "test1"))
try {
// Available partition ids should be 0, 1, 2 and 3, all lead and hosted only
// on broker 0
producer.send(new KeyedMessage[String, String](topic, "test", "test1"))
} catch {
case e => fail("Unexpected exception: " + e)
}
// kill the broker
server1.shutdown
@ -220,7 +230,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
try {
// These sends should fail since there are no available brokers
producer.send(new KeyedMessage[String, String]("new-topic", "test", "test1"))
producer.send(new KeyedMessage[String, String](topic, "test", "test1"))
fail("Should fail since no leader exists for the partition.")
} catch {
case e => // success
@ -228,12 +238,12 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
// restart server 1
server1.startup()
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
try {
// cross check if broker 1 got the messages
val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val messageSet1 = response1.messageSet("new-topic", 0).iterator
val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
val messageSet1 = response1.messageSet(topic, 0).iterator
assertTrue("Message set should have 1 message", messageSet1.hasNext)
assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes), messageSet1.next.message)
assertFalse("Message set should have another message", messageSet1.hasNext)
@ -252,22 +262,22 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
props.put("request.timeout.ms", String.valueOf(timeoutMs))
props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
props.put("request.required.acks", "1")
props.put("client.id","ProducerTest-testAsyncSendCanCorrectlyFailWithTimeout")
val config = new ProducerConfig(props)
val producer = new Producer[String, String](config)
val topic = "new-topic"
// create topics in ZK
AdminUtils.createTopicWithAssignment(zkClient, "new-topic", Map(0->Seq(0,1)))
assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() =>
AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime))
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500)
AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0,1)))
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500)
// do a simple test to make sure plumbing is okay
try {
// this message should be assigned to partition 0 whose leader is on broker 0
producer.send(new KeyedMessage[String, String]("new-topic", "test", "test"))
producer.send(new KeyedMessage[String, String](topic, "test", "test"))
// cross check if brokers got the messages
val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(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)
@ -283,7 +293,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
try {
// this message should be assigned to partition 0 whose leader is on broker 0, but
// broker 0 will not response within timeoutMs millis.
producer.send(new KeyedMessage[String, String]("new-topic", "test", "test"))
producer.send(new KeyedMessage[String, String](topic, "test", "test"))
} catch {
case e: FailedToSendMessageException => /* success */
case e: Exception => fail("Not expected", e)
@ -320,4 +330,3 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
}
}
}

View File

@ -25,7 +25,8 @@ import org.junit._
import org.junit.Assert._
import kafka.common._
import kafka.cluster.Replica
import kafka.utils._
import kafka.utils.{SystemTime, KafkaScheduler, TestUtils, MockTime, Utils}
import java.util.concurrent.atomic.AtomicBoolean
class HighwatermarkPersistenceTest extends JUnit3Suite {
@ -55,7 +56,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
val scheduler = new KafkaScheduler(2)
scheduler.startup
// create replica manager
val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0))
val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0), new AtomicBoolean(false))
replicaManager.startup()
replicaManager.checkpointHighWatermarks()
var fooPartition0Hw = hwmFor(replicaManager, topic, 0)
@ -94,7 +95,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
val scheduler = new KafkaScheduler(2)
scheduler.startup
// create replica manager
val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0))
val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0), new AtomicBoolean(false))
replicaManager.startup()
replicaManager.checkpointHighWatermarks()
var topic1Partition0Hw = hwmFor(replicaManager, topic1, 0)

View File

@ -24,6 +24,7 @@ import org.easymock.EasyMock
import kafka.log.Log
import org.junit.Assert._
import kafka.utils._
import java.util.concurrent.atomic.AtomicBoolean
class IsrExpirationTest extends JUnit3Suite {
@ -80,7 +81,7 @@ class IsrExpirationTest extends JUnit3Suite {
private def getPartitionWithAllReplicasInIsr(topic: String, partitionId: Int, time: Time, config: KafkaConfig,
localLog: Log): Partition = {
val leaderId=config.brokerId
val replicaManager = new ReplicaManager(config, time, null, null, null)
val replicaManager = new ReplicaManager(config, time, null, null, null, new AtomicBoolean(false))
val partition = replicaManager.getOrCreatePartition(topic, partitionId, 1)
val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog))

View File

@ -132,7 +132,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness {
val leaderAndIsr = new collection.mutable.HashMap[(String, Int), LeaderIsrAndControllerEpoch]
leaderAndIsr.put((topic, partitionId),
new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2))
val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, 1)).toMap
val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, Set(0,1))).toMap
val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId,
staleControllerEpoch, 0, "")

View File

@ -50,6 +50,8 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
// create topic
AdminUtils.createTopic(zkClient, topic, 1, 1)
TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000)
// send some messages
producer.send(sent1.map(m => new KeyedMessage[Int, String](topic, 0, m)):_*)
@ -65,11 +67,12 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
server = new KafkaServer(config)
server.startup()
// wait for the broker to receive the update metadata request after startup
TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000)
producer = new Producer[Int, String](new ProducerConfig(producerConfig))
val consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "")
waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
var fetchedMessage: ByteBufferMessageSet = null
while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).maxWait(0).build())

View File

@ -20,7 +20,7 @@ import kafka.cluster.{Partition, Replica}
import kafka.log.Log
import kafka.message.{ByteBufferMessageSet, Message}
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.utils.{Time, TestUtils, MockTime}
import kafka.utils.{ZkUtils, Time, TestUtils, MockTime}
import org.easymock.EasyMock
import org.I0Itec.zkclient.ZkClient
import org.scalatest.junit.JUnit3Suite
@ -57,7 +57,9 @@ class SimpleFetchTest extends JUnit3Suite {
val fetchSize = 100
val messages = new Message("test-message".getBytes())
val zkClient = EasyMock.createMock(classOf[ZkClient])
// create nice mock since we don't particularly care about zkclient calls
val zkClient = EasyMock.createNiceMock(classOf[ZkClient])
EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false)
EasyMock.replay(zkClient)
val log = EasyMock.createMock(classOf[kafka.log.Log])
@ -85,10 +87,12 @@ class SimpleFetchTest extends JUnit3Suite {
EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes()
EasyMock.replay(replicaManager)
val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController])
// start a request channel with 2 processors and a queue size of 5 (this is more or less arbitrary)
// don't provide replica or leader callbacks since they will not be tested here
val requestChannel = new RequestChannel(2, 5)
val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head)
val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller)
// This request (from a follower) wants to read up to 2*HW but should only get back up to HW bytes into the log
val goodFetch = new FetchRequestBuilder()
@ -124,7 +128,8 @@ class SimpleFetchTest extends JUnit3Suite {
val followerReplicaId = configs(1).brokerId
val followerLEO = 15
val zkClient = EasyMock.createMock(classOf[ZkClient])
val zkClient = EasyMock.createNiceMock(classOf[ZkClient])
EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false)
EasyMock.replay(zkClient)
val log = EasyMock.createMock(classOf[kafka.log.Log])
@ -153,8 +158,10 @@ class SimpleFetchTest extends JUnit3Suite {
EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes()
EasyMock.replay(replicaManager)
val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController])
val requestChannel = new RequestChannel(2, 5)
val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head)
val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller)
/**
* This fetch, coming from a replica, requests all data at offset "15". Because the request is coming

View File

@ -38,6 +38,7 @@ import kafka.api._
import collection.mutable.Map
import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder}
import kafka.common.TopicAndPartition
import junit.framework.Assert
/**
@ -148,6 +149,7 @@ object TestUtils extends Logging {
props.put("zookeeper.sync.time.ms", "200")
props.put("auto.commit.interval.ms", "1000")
props.put("rebalance.max.retries", "4")
props.put("auto.offset.reset", "smallest")
props
}
@ -506,6 +508,12 @@ object TestUtils extends Logging {
byteBuffer.rewind()
byteBuffer
}
def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, timeout: Long) = {
Assert.assertTrue("Partition [%s,%d] metadata not propagated after timeout".format(topic, partition),
TestUtils.waitUntilTrue(() =>
servers.foldLeft(true)(_ && _.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition))), timeout))
}
}

View File

@ -26,11 +26,26 @@ object KafkaBuild extends Build {
val buildNumber = SettingKey[String]("build-number", "Build number defaults to $BUILD_NUMBER environment variable")
val releaseName = SettingKey[String]("release-name", "the full name of this release")
val commonSettings = Seq(
organization := "org.apache",
organization := "org.apache.kafka",
pomExtra :=
<parent>
<groupId>org.apache</groupId>
<artifactId>apache</artifactId>
<version>10</version>
</parent>
<licenses>
<license>
<name>Apache 2</name>
<url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
<distribution>repo</distribution>
</license>
</licenses>,
scalacOptions ++= Seq("-deprecation", "-unchecked", "-g:none"),
crossScalaVersions := Seq("2.8.0","2.8.2", "2.9.1", "2.9.2"),
scalaVersion := "2.8.0",
version := "0.8.0-SNAPSHOT",
version := "0.8.0-beta1",
publishTo := Some("Apache Maven Repo" at "https://repository.apache.org/service/local/staging/deploy/maven2"),
credentials += Credentials(Path.userHome / ".m2" / ".credentials"),
buildNumber := System.getProperty("build.number", ""),
version <<= (buildNumber, version) { (build, version) => if (build == "") version else version + "+" + build},
releaseName <<= (name, version, scalaVersion) {(name, version, scalaVersion) => name + "_" + scalaVersion + "-" + version},

View File

@ -3,3 +3,7 @@ resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.
addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.8")
addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0")
resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns)
addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6")

View File

@ -70,6 +70,7 @@
"message-size": "500",
"message": "100",
"request-num-acks": "-1",
"producer-retry-backoff-ms": "300",
"sync":"true",
"log_filename": "producer_performance.log",
"config_filename": "producer_performance.properties"