mirror of https://github.com/apache/kafka.git
kafka-965; merge c39d37e9dd
from 0.8 to trunk; patched by Jun Rao; reviewed by Jay Kreps
This commit is contained in:
commit
db37ed0054
|
@ -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
|
||||
|
||||
|
||||
|
|
|
@ -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 #############################
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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._
|
||||
|
|
|
@ -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)))
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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]]))
|
||||
}
|
||||
|
|
|
@ -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)))
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 =>
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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]
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -74,7 +74,8 @@ 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.
|
||||
*/
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -58,5 +58,5 @@ trait SyncProducerConfigShared {
|
|||
object SyncProducerConfig {
|
||||
val DefaultClientId = ""
|
||||
val DefaultRequiredAcks : Short = 0
|
||||
val DefaultAckTimeoutMs = 1500
|
||||
val DefaultAckTimeoutMs = 10000
|
||||
}
|
|
@ -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]
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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)))
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
||||
}
|
||||
}
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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) = {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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}
|
||||
|
||||
|
|
|
@ -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 = {
|
||||
|
|
|
@ -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._
|
||||
|
|
|
@ -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._
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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())
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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, "")
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
@ -173,6 +176,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))
|
||||
|
@ -243,6 +249,9 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
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())
|
||||
|
@ -266,6 +275,9 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
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,6 +335,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
|
||||
val receivedMessages1 = getMessages(nMessages, topicMessageStreams1)
|
||||
assertEquals(sentMessages1, receivedMessages1)
|
||||
zkConsumerConnector1.shutdown()
|
||||
zkClient.close()
|
||||
}
|
||||
|
||||
def sendMessagesToBrokerPartition(config: KafkaConfig,
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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._
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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{
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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))
|
||||
|
||||
|
|
|
@ -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, "")
|
||||
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
@ -507,6 +509,12 @@ object TestUtils extends Logging {
|
|||
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))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object TestZKUtils {
|
||||
|
|
|
@ -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},
|
||||
|
|
|
@ -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")
|
|
@ -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"
|
||||
|
|
Loading…
Reference in New Issue