mirror of https://github.com/apache/kafka.git
KAFKA-385 Fix race condition between checkSatisfied and expire in RequestPurgatory; fixed constant expiration of follower fetch requests as checkSatisfied was not getting called; add metrics to the RequestPurgatory; add a KafkaTimer convenience class; patched by Joel Koshy; reviewed by Jun Rao and Jay Kreps.
git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/branches/0.8@1374069 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0fe89f7a49
commit
190bd3ff71
|
@ -17,6 +17,8 @@
|
||||||
|
|
||||||
package kafka
|
package kafka
|
||||||
|
|
||||||
|
|
||||||
|
import metrics.{KafkaMetricsReporterMBean, KafkaMetricsReporter, KafkaMetricsConfig}
|
||||||
import server.{KafkaConfig, KafkaServerStartable, KafkaServer}
|
import server.{KafkaConfig, KafkaServerStartable, KafkaServer}
|
||||||
import utils.{Utils, Logging}
|
import utils.{Utils, Logging}
|
||||||
import org.apache.log4j.jmx.LoggerDynamicMBean
|
import org.apache.log4j.jmx.LoggerDynamicMBean
|
||||||
|
@ -36,6 +38,13 @@ object Kafka extends Logging {
|
||||||
try {
|
try {
|
||||||
val props = Utils.loadProps(args(0))
|
val props = Utils.loadProps(args(0))
|
||||||
val serverConfig = new KafkaConfig(props)
|
val serverConfig = new KafkaConfig(props)
|
||||||
|
val metricsConfig = new KafkaMetricsConfig(props)
|
||||||
|
metricsConfig.reporters.foreach(reporterType => {
|
||||||
|
val reporter = Utils.getObject[KafkaMetricsReporter](reporterType)
|
||||||
|
reporter.init(props)
|
||||||
|
if (reporter.isInstanceOf[KafkaMetricsReporterMBean])
|
||||||
|
Utils.registerMBean(reporter, reporter.asInstanceOf[KafkaMetricsReporterMBean].getMBeanName)
|
||||||
|
})
|
||||||
|
|
||||||
val kafkaServerStartble = new KafkaServerStartable(serverConfig)
|
val kafkaServerStartble = new KafkaServerStartable(serverConfig)
|
||||||
|
|
||||||
|
|
|
@ -73,7 +73,6 @@ class PartitionDataSend(val partitionData: PartitionData) extends Send {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
object TopicData {
|
object TopicData {
|
||||||
def readFrom(buffer: ByteBuffer): TopicData = {
|
def readFrom(buffer: ByteBuffer): TopicData = {
|
||||||
val topic = Utils.readShortString(buffer, "UTF-8")
|
val topic = Utils.readShortString(buffer, "UTF-8")
|
||||||
|
|
|
@ -0,0 +1,85 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* 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.metrics
|
||||||
|
|
||||||
|
import java.util.Properties
|
||||||
|
import com.yammer.metrics.Metrics
|
||||||
|
import java.io.File
|
||||||
|
import com.yammer.metrics.reporting.CsvReporter
|
||||||
|
import kafka.utils.{Logging, Utils}
|
||||||
|
import java.util.concurrent.TimeUnit
|
||||||
|
|
||||||
|
|
||||||
|
private trait KafkaCSVMetricsReporterMBean extends KafkaMetricsReporterMBean
|
||||||
|
|
||||||
|
private class KafkaCSVMetricsReporter extends KafkaMetricsReporter
|
||||||
|
with KafkaCSVMetricsReporterMBean
|
||||||
|
with Logging {
|
||||||
|
|
||||||
|
private var csvDir: File = null
|
||||||
|
private var underlying: CsvReporter = null
|
||||||
|
private var running = false
|
||||||
|
private var initialized = false
|
||||||
|
|
||||||
|
|
||||||
|
override def getMBeanName = "kafka:type=kafka.metrics.KafkaCSVMetricsReporter"
|
||||||
|
|
||||||
|
|
||||||
|
override def init(props: Properties) {
|
||||||
|
synchronized {
|
||||||
|
if (!initialized) {
|
||||||
|
val metricsConfig = new KafkaMetricsConfig(props)
|
||||||
|
csvDir = new File(Utils.getString(props, "kafka.csv.metrics.dir", "kafka_metrics"))
|
||||||
|
if (!csvDir.exists())
|
||||||
|
csvDir.mkdirs()
|
||||||
|
underlying = new CsvReporter(Metrics.defaultRegistry(), csvDir)
|
||||||
|
if (Utils.getBoolean(props, "kafka.csv.metrics.reporter.enabled", false))
|
||||||
|
startReporter(metricsConfig.pollingIntervalSecs)
|
||||||
|
initialized = true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
override def startReporter(pollingPeriodSecs: Long) {
|
||||||
|
synchronized {
|
||||||
|
if (initialized && !running) {
|
||||||
|
underlying.start(pollingPeriodSecs, TimeUnit.SECONDS)
|
||||||
|
running = true
|
||||||
|
info("Started Kafka CSV metrics reporter with polling period %d seconds".format(pollingPeriodSecs))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
override def stopReporter() {
|
||||||
|
synchronized {
|
||||||
|
if (initialized && running) {
|
||||||
|
underlying.shutdown()
|
||||||
|
running = false
|
||||||
|
info("Stopped Kafka CSV metrics reporter")
|
||||||
|
underlying = new CsvReporter(Metrics.defaultRegistry(), csvDir)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
|
@ -0,0 +1,38 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* 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.metrics
|
||||||
|
|
||||||
|
import java.util.Properties
|
||||||
|
import kafka.utils.Utils
|
||||||
|
|
||||||
|
class KafkaMetricsConfig(props: Properties) {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Comma-separated list of reporter types. These classes should be on the
|
||||||
|
* classpath and will be instantiated at run-time.
|
||||||
|
*/
|
||||||
|
val reporters = Utils.getCSVList(Utils.getString(props, "kafka.metrics.reporters", ""))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The metrics polling interval (in seconds).
|
||||||
|
*/
|
||||||
|
val pollingIntervalSecs = Utils.getInt(props, "kafka.metrics.polling.interval.secs", 10)
|
||||||
|
}
|
|
@ -0,0 +1,77 @@
|
||||||
|
/**
|
||||||
|
* 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.metrics
|
||||||
|
|
||||||
|
|
||||||
|
import com.yammer.metrics.core.{Gauge, MetricName}
|
||||||
|
import kafka.utils.Logging
|
||||||
|
import java.util.concurrent.TimeUnit
|
||||||
|
import com.yammer.metrics.Metrics
|
||||||
|
|
||||||
|
|
||||||
|
trait KafkaMetricsGroup extends Logging {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method enables the user to form logical sub-groups of this
|
||||||
|
* KafkaMetricsGroup by inserting a sub-group identifier in the package
|
||||||
|
* string.
|
||||||
|
*
|
||||||
|
* @return The sub-group identifier.
|
||||||
|
*/
|
||||||
|
def metricsGroupIdent: String
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a new MetricName object for gauges, meters, etc. created for this
|
||||||
|
* metrics group. It uses the metricsGroupIdent to create logical sub-groups.
|
||||||
|
* This is currently specifically of use to classes under kafka, with
|
||||||
|
* broker-id being the most common metrics grouping strategy.
|
||||||
|
*
|
||||||
|
* @param name Descriptive name of the metric.
|
||||||
|
* @return Sanitized metric name object.
|
||||||
|
*/
|
||||||
|
private def metricName(name: String) = {
|
||||||
|
val ident = metricsGroupIdent
|
||||||
|
val klass = this.getClass
|
||||||
|
val pkg = {
|
||||||
|
val actualPkg = if (klass.getPackage == null) "" else klass.getPackage.getName
|
||||||
|
if (ident.nonEmpty) {
|
||||||
|
// insert the sub-group identifier after the top-level package
|
||||||
|
if (actualPkg.contains("."))
|
||||||
|
actualPkg.replaceFirst("""\.""", ".%s.".format(ident))
|
||||||
|
else
|
||||||
|
actualPkg + "." + ident
|
||||||
|
}
|
||||||
|
else
|
||||||
|
actualPkg
|
||||||
|
}
|
||||||
|
val simpleName = klass.getSimpleName.replaceAll("\\$$", "")
|
||||||
|
new MetricName(pkg, simpleName, name)
|
||||||
|
}
|
||||||
|
|
||||||
|
def newGauge[T](name: String, metric: Gauge[T]) =
|
||||||
|
Metrics.newGauge(metricName(name), metric)
|
||||||
|
|
||||||
|
def newMeter(name: String, eventType: String, timeUnit: TimeUnit) =
|
||||||
|
Metrics.newMeter(metricName(name), eventType, timeUnit)
|
||||||
|
|
||||||
|
def newHistogram(name: String, biased: Boolean = false) = Metrics.newHistogram(metricName(name), biased)
|
||||||
|
|
||||||
|
def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit) =
|
||||||
|
Metrics.newTimer(metricName(name), durationUnit, rateUnit)
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,47 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* 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.metrics
|
||||||
|
|
||||||
|
import java.util.Properties
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base trait for reporter MBeans. If a client wants to expose these JMX
|
||||||
|
* operations on a custom reporter (that implements
|
||||||
|
* [[kafka.metrics.KafkaMetricsReporter]]), the custom reporter needs to
|
||||||
|
* additionally implement an MBean trait that extends this trait so that the
|
||||||
|
* registered MBean is compliant with the standard MBean convention.
|
||||||
|
*/
|
||||||
|
trait KafkaMetricsReporterMBean {
|
||||||
|
def startReporter(pollingPeriodInSeconds: Long)
|
||||||
|
def stopReporter()
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* @return The name with which the MBean will be registered.
|
||||||
|
*/
|
||||||
|
def getMBeanName: String
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
trait KafkaMetricsReporter {
|
||||||
|
def init(props: Properties)
|
||||||
|
}
|
||||||
|
|
|
@ -0,0 +1,40 @@
|
||||||
|
/**
|
||||||
|
* 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.metrics
|
||||||
|
|
||||||
|
import com.yammer.metrics.core.Timer
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A wrapper around metrics timer object that provides a convenient mechanism
|
||||||
|
* to time code blocks. This pattern was borrowed from the metrics-scala_2.9.1
|
||||||
|
* package.
|
||||||
|
* @param metric The underlying timer object.
|
||||||
|
*/
|
||||||
|
class KafkaTimer(metric: Timer) {
|
||||||
|
|
||||||
|
def time[A](f: => A): A = {
|
||||||
|
val ctx = metric.time
|
||||||
|
try {
|
||||||
|
f
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
ctx.stop()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -18,21 +18,23 @@
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import java.io.IOException
|
import java.io.IOException
|
||||||
import java.util.concurrent.atomic._
|
|
||||||
import kafka.admin.{CreateTopicCommand, AdminUtils}
|
import kafka.admin.{CreateTopicCommand, AdminUtils}
|
||||||
import kafka.api._
|
import kafka.api._
|
||||||
import kafka.common._
|
import kafka.common._
|
||||||
import kafka.log._
|
import kafka.log._
|
||||||
import kafka.message._
|
import kafka.message._
|
||||||
import kafka.network._
|
import kafka.network._
|
||||||
|
import kafka.utils.{ZkUtils, Pool, SystemTime, Logging}
|
||||||
import org.apache.log4j.Logger
|
import org.apache.log4j.Logger
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
import mutable.HashMap
|
import mutable.HashMap
|
||||||
import scala.math._
|
import scala.math._
|
||||||
import kafka.network.RequestChannel.Response
|
import kafka.network.RequestChannel.Response
|
||||||
import kafka.utils.{ZkUtils, SystemTime, Logging}
|
import java.util.concurrent.TimeUnit
|
||||||
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.cluster.Replica
|
import kafka.cluster.Replica
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Logic to handle the various Kafka requests
|
* Logic to handle the various Kafka requests
|
||||||
*/
|
*/
|
||||||
|
@ -44,10 +46,13 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
becomeFollower: (Replica, LeaderAndISR) => Short,
|
becomeFollower: (Replica, LeaderAndISR) => Short,
|
||||||
brokerId: Int) extends Logging {
|
brokerId: Int) extends Logging {
|
||||||
|
|
||||||
private val produceRequestPurgatory = new ProducerRequestPurgatory(brokerId)
|
private val metricsGroup = brokerId.toString
|
||||||
private val fetchRequestPurgatory = new FetchRequestPurgatory(brokerId, requestChannel)
|
private val producerRequestPurgatory = new ProducerRequestPurgatory
|
||||||
|
private val fetchRequestPurgatory = new FetchRequestPurgatory(requestChannel)
|
||||||
|
private val delayedRequestMetrics = new DelayedRequestMetrics
|
||||||
|
|
||||||
private val requestLogger = Logger.getLogger("kafka.request.logger")
|
private val requestLogger = Logger.getLogger("kafka.request.logger")
|
||||||
this.logIdent = "KafkaApi on Broker " + brokerId + ", "
|
this.logIdent = "KafkaApis-%d ".format(brokerId)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Top-level method that handles all requests and multiplexes to the right api
|
* Top-level method that handles all requests and multiplexes to the right api
|
||||||
|
@ -69,7 +74,9 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
def handleLeaderAndISRRequest(request: RequestChannel.Request){
|
def handleLeaderAndISRRequest(request: RequestChannel.Request){
|
||||||
val responseMap = new HashMap[(String, Int), Short]
|
val responseMap = new HashMap[(String, Int), Short]
|
||||||
val leaderAndISRRequest = LeaderAndISRRequest.readFrom(request.request.buffer)
|
val leaderAndISRRequest = LeaderAndISRRequest.readFrom(request.request.buffer)
|
||||||
info("handling leader and isr request " + leaderAndISRRequest)
|
if(requestLogger.isTraceEnabled)
|
||||||
|
requestLogger.trace("Handling leader and isr request " + leaderAndISRRequest)
|
||||||
|
trace("Handling leader and isr request " + leaderAndISRRequest)
|
||||||
|
|
||||||
for((partitionInfo, leaderAndISR) <- leaderAndISRRequest.leaderAndISRInfos){
|
for((partitionInfo, leaderAndISR) <- leaderAndISRRequest.leaderAndISRInfos){
|
||||||
var errorCode = ErrorMapping.NoError
|
var errorCode = ErrorMapping.NoError
|
||||||
|
@ -78,12 +85,12 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
|
|
||||||
// If the partition does not exist locally, create it
|
// If the partition does not exist locally, create it
|
||||||
if(replicaManager.getPartition(topic, partition) == None){
|
if(replicaManager.getPartition(topic, partition) == None){
|
||||||
trace("the partition (%s, %d) does not exist locally, check if current broker is in assigned replicas, if so, start the local replica".format(topic, partition))
|
trace("The partition (%s, %d) does not exist locally, check if current broker is in assigned replicas, if so, start the local replica".format(topic, partition))
|
||||||
val assignedReplicas = ZkUtils.getReplicasForPartition(kafkaZookeeper.getZookeeperClient, topic, partition)
|
val assignedReplicas = ZkUtils.getReplicasForPartition(kafkaZookeeper.getZookeeperClient, topic, partition)
|
||||||
trace("assigned replicas list for topic [%s] partition [%d] is [%s]".format(topic, partition, assignedReplicas.toString))
|
trace("Assigned replicas list for topic [%s] partition [%d] is [%s]".format(topic, partition, assignedReplicas.toString))
|
||||||
if(assignedReplicas.contains(brokerId)) {
|
if(assignedReplicas.contains(brokerId)) {
|
||||||
val replica = addReplicaCbk(topic, partition, assignedReplicas.toSet)
|
val replica = addReplicaCbk(topic, partition, assignedReplicas.toSet)
|
||||||
info("starting replica for topic [%s] partition [%d]".format(replica.topic, replica.partition.partitionId))
|
info("Starting replica for topic [%s] partition [%d]".format(replica.topic, replica.partition.partitionId))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
val replica = replicaManager.getReplica(topic, partition).get
|
val replica = replicaManager.getReplica(topic, partition).get
|
||||||
|
@ -91,11 +98,11 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
val requestedLeaderId = leaderAndISR.leader
|
val requestedLeaderId = leaderAndISR.leader
|
||||||
// If the broker is requested to be the leader and it's not current the leader (the leader id is set and not equal to broker id)
|
// If the broker is requested to be the leader and it's not current the leader (the leader id is set and not equal to broker id)
|
||||||
if(requestedLeaderId == brokerId && (!replica.partition.leaderId().isDefined || replica.partition.leaderId().get != brokerId)){
|
if(requestedLeaderId == brokerId && (!replica.partition.leaderId().isDefined || replica.partition.leaderId().get != brokerId)){
|
||||||
info("becoming the leader for partition [%s, %d] at the leader and isr request %s".format(topic, partition, leaderAndISRRequest))
|
info("Becoming the leader for partition [%s, %d] at the leader and isr request %s".format(topic, partition, leaderAndISRRequest))
|
||||||
errorCode = becomeLeader(replica, leaderAndISR)
|
errorCode = becomeLeader(replica, leaderAndISR)
|
||||||
}
|
}
|
||||||
else if (requestedLeaderId != brokerId) {
|
else if (requestedLeaderId != brokerId) {
|
||||||
info("becoming the follower for partition [%s, %d] at the leader and isr request %s".format(topic, partition, leaderAndISRRequest))
|
info("Becoming the follower for partition [%s, %d] at the leader and isr request %s".format(topic, partition, leaderAndISRRequest))
|
||||||
errorCode = becomeFollower(replica, leaderAndISR)
|
errorCode = becomeFollower(replica, leaderAndISR)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -105,7 +112,7 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
if(leaderAndISRRequest.isInit == LeaderAndISRRequest.IsInit){
|
if(leaderAndISRRequest.isInit == LeaderAndISRRequest.IsInit){
|
||||||
replicaManager.startHighWaterMarksCheckPointThread
|
replicaManager.startHighWaterMarksCheckPointThread
|
||||||
val partitionsToRemove = replicaManager.allPartitions.filter(p => !leaderAndISRRequest.leaderAndISRInfos.contains(p._1)).keySet
|
val partitionsToRemove = replicaManager.allPartitions.filter(p => !leaderAndISRRequest.leaderAndISRInfos.contains(p._1)).keySet
|
||||||
info("init flag is set in leaderAndISR request, partitions to remove: %s".format(partitionsToRemove))
|
info("Init flag is set in leaderAndISR request, partitions to remove: %s".format(partitionsToRemove))
|
||||||
partitionsToRemove.foreach(p => stopReplicaCbk(p._1, p._2))
|
partitionsToRemove.foreach(p => stopReplicaCbk(p._1, p._2))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -116,6 +123,10 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
|
|
||||||
def handleStopReplicaRequest(request: RequestChannel.Request){
|
def handleStopReplicaRequest(request: RequestChannel.Request){
|
||||||
val stopReplicaRequest = StopReplicaRequest.readFrom(request.request.buffer)
|
val stopReplicaRequest = StopReplicaRequest.readFrom(request.request.buffer)
|
||||||
|
if(requestLogger.isTraceEnabled)
|
||||||
|
requestLogger.trace("Handling stop replica request " + stopReplicaRequest)
|
||||||
|
trace("Handling stop replica request " + stopReplicaRequest)
|
||||||
|
|
||||||
val responseMap = new HashMap[(String, Int), Short]
|
val responseMap = new HashMap[(String, Int), Short]
|
||||||
|
|
||||||
for((topic, partition) <- stopReplicaRequest.stopReplicaSet){
|
for((topic, partition) <- stopReplicaRequest.stopReplicaSet){
|
||||||
|
@ -133,12 +144,17 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
def maybeUnblockDelayedFetchRequests(topic: String, partitionDatas: Array[PartitionData]) {
|
def maybeUnblockDelayedFetchRequests(topic: String, partitionDatas: Array[PartitionData]) {
|
||||||
var satisfied = new mutable.ArrayBuffer[DelayedFetch]
|
var satisfied = new mutable.ArrayBuffer[DelayedFetch]
|
||||||
for(partitionData <- partitionDatas)
|
for(partitionData <- partitionDatas)
|
||||||
satisfied ++= fetchRequestPurgatory.update((topic, partitionData.partition), partitionData)
|
satisfied ++= fetchRequestPurgatory.update(RequestKey(topic, partitionData.partition), null)
|
||||||
trace("produce request to %s unblocked %d DelayedFetchRequests.".format(topic, satisfied.size))
|
trace("Producer request to %s unblocked %d fetch requests.".format(topic, satisfied.size))
|
||||||
// send any newly unblocked responses
|
// send any newly unblocked responses
|
||||||
for(fetchReq <- satisfied) {
|
for(fetchReq <- satisfied) {
|
||||||
val topicData = readMessageSets(fetchReq.fetch)
|
val topicData = readMessageSets(fetchReq.fetch)
|
||||||
val response = new FetchResponse(FetchRequest.CurrentVersion, fetchReq.fetch.correlationId, topicData)
|
val response = new FetchResponse(FetchRequest.CurrentVersion, fetchReq.fetch.correlationId, topicData)
|
||||||
|
|
||||||
|
val fromFollower = fetchReq.fetch.replicaId != FetchRequest.NonFollowerId
|
||||||
|
delayedRequestMetrics.recordDelayedFetchSatisfied(
|
||||||
|
fromFollower, SystemTime.nanoseconds - fetchReq.creationTimeNs, response)
|
||||||
|
|
||||||
requestChannel.sendResponse(new RequestChannel.Response(fetchReq.request, new FetchResponseSend(response)))
|
requestChannel.sendResponse(new RequestChannel.Response(fetchReq.request, new FetchResponseSend(response)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -150,43 +166,45 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
val produceRequest = ProducerRequest.readFrom(request.request.buffer)
|
val produceRequest = ProducerRequest.readFrom(request.request.buffer)
|
||||||
val sTime = SystemTime.milliseconds
|
val sTime = SystemTime.milliseconds
|
||||||
if(requestLogger.isTraceEnabled)
|
if(requestLogger.isTraceEnabled)
|
||||||
requestLogger.trace("producer request %s".format(produceRequest.toString))
|
requestLogger.trace("Handling producer request " + request.toString)
|
||||||
trace("Broker %s received produce request %s".format(brokerId, produceRequest.toString))
|
trace("Handling producer request " + request.toString)
|
||||||
|
|
||||||
val response = produceToLocalLog(produceRequest)
|
val response = produceToLocalLog(produceRequest)
|
||||||
debug("produce to local log in %d ms".format(SystemTime.milliseconds - sTime))
|
debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime))
|
||||||
|
|
||||||
|
for (topicData <- produceRequest.data)
|
||||||
|
maybeUnblockDelayedFetchRequests(topicData.topic, topicData.partitionDataArray)
|
||||||
|
|
||||||
if (produceRequest.requiredAcks == 0 ||
|
if (produceRequest.requiredAcks == 0 ||
|
||||||
produceRequest.requiredAcks == 1 ||
|
produceRequest.requiredAcks == 1 ||
|
||||||
produceRequest.data.size <= 0) {
|
produceRequest.data.size <= 0)
|
||||||
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
|
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
|
||||||
|
|
||||||
for (topicData <- produceRequest.data)
|
|
||||||
maybeUnblockDelayedFetchRequests(topicData.topic, topicData.partitionDataArray)
|
|
||||||
}
|
|
||||||
else {
|
else {
|
||||||
// create a list of (topic, partition) pairs to use as keys for this delayed request
|
// create a list of (topic, partition) pairs to use as keys for this delayed request
|
||||||
val topicPartitionPairs = produceRequest.data.flatMap(topicData => {
|
val producerRequestKeys = produceRequest.data.flatMap(topicData => {
|
||||||
val topic = topicData.topic
|
val topic = topicData.topic
|
||||||
topicData.partitionDataArray.map(partitionData => {
|
topicData.partitionDataArray.map(partitionData => {
|
||||||
(topic, partitionData.partition)
|
RequestKey(topic, partitionData.partition)
|
||||||
})
|
})
|
||||||
})
|
})
|
||||||
|
|
||||||
val delayedProduce = new DelayedProduce(
|
val delayedProduce = new DelayedProduce(
|
||||||
topicPartitionPairs, request,
|
producerRequestKeys, request,
|
||||||
response.errors, response.offsets,
|
response.errors, response.offsets,
|
||||||
produceRequest, produceRequest.ackTimeoutMs.toLong)
|
produceRequest, produceRequest.ackTimeoutMs.toLong)
|
||||||
produceRequestPurgatory.watch(delayedProduce)
|
producerRequestPurgatory.watch(delayedProduce)
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Replica fetch requests may have arrived (and potentially satisfied)
|
* Replica fetch requests may have arrived (and potentially satisfied)
|
||||||
* delayedProduce requests before they even made it to the purgatory.
|
* delayedProduce requests while they were being added to the purgatory.
|
||||||
* Here, we explicitly check if any of them can be satisfied.
|
* Here, we explicitly check if any of them can be satisfied.
|
||||||
*/
|
*/
|
||||||
var satisfiedProduceRequests = new mutable.ArrayBuffer[DelayedProduce]
|
var satisfiedProduceRequests = new mutable.ArrayBuffer[DelayedProduce]
|
||||||
topicPartitionPairs.foreach(topicPartition =>
|
producerRequestKeys.foreach(key =>
|
||||||
satisfiedProduceRequests ++=
|
satisfiedProduceRequests ++=
|
||||||
produceRequestPurgatory.update(topicPartition, topicPartition))
|
producerRequestPurgatory.update(key, key))
|
||||||
debug("%d DelayedProduce requests unblocked after produce to local log.".format(satisfiedProduceRequests.size))
|
debug(satisfiedProduceRequests.size +
|
||||||
|
" producer requests unblocked during produce to local log.")
|
||||||
satisfiedProduceRequests.foreach(_.respond())
|
satisfiedProduceRequests.foreach(_.respond())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -195,10 +213,11 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
* Helper method for handling a parsed producer request
|
* Helper method for handling a parsed producer request
|
||||||
*/
|
*/
|
||||||
private def produceToLocalLog(request: ProducerRequest): ProducerResponse = {
|
private def produceToLocalLog(request: ProducerRequest): ProducerResponse = {
|
||||||
trace("produce [%s] to local log ".format(request.toString))
|
trace("Produce [%s] to local log ".format(request.toString))
|
||||||
val requestSize = request.topicPartitionCount
|
val requestSize = request.topicPartitionCount
|
||||||
val errors = new Array[Short](requestSize)
|
val errors = new Array[Short](requestSize)
|
||||||
val offsets = new Array[Long](requestSize)
|
val offsets = new Array[Long](requestSize)
|
||||||
|
|
||||||
var msgIndex = -1
|
var msgIndex = -1
|
||||||
for(topicData <- request.data) {
|
for(topicData <- request.data) {
|
||||||
for(partitionData <- topicData.partitionDataArray) {
|
for(partitionData <- topicData.partitionDataArray) {
|
||||||
|
@ -212,12 +231,13 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
replicaManager.recordLeaderLogEndOffset(topicData.topic, partitionData.partition, log.logEndOffset)
|
replicaManager.recordLeaderLogEndOffset(topicData.topic, partitionData.partition, log.logEndOffset)
|
||||||
offsets(msgIndex) = log.logEndOffset
|
offsets(msgIndex) = log.logEndOffset
|
||||||
errors(msgIndex) = ErrorMapping.NoError.toShort
|
errors(msgIndex) = ErrorMapping.NoError.toShort
|
||||||
trace(partitionData.messages.sizeInBytes + " bytes written to logs.")
|
trace("%d bytes written to logs, nextAppendOffset = %d"
|
||||||
|
.format(partitionData.messages.sizeInBytes, offsets(msgIndex)))
|
||||||
} catch {
|
} catch {
|
||||||
case e =>
|
case e =>
|
||||||
BrokerTopicStat.getBrokerTopicStat(topicData.topic).recordFailedProduceRequest
|
BrokerTopicStat.getBrokerTopicStat(topicData.topic).recordFailedProduceRequest
|
||||||
BrokerTopicStat.getBrokerAllTopicStat.recordFailedProduceRequest
|
BrokerTopicStat.getBrokerAllTopicStat.recordFailedProduceRequest
|
||||||
error("error processing ProducerRequest on " + topicData.topic + ":" + partitionData.partition, e)
|
error("Error processing ProducerRequest on %s:%d".format(topicData.topic, partitionData.partition), e)
|
||||||
e match {
|
e match {
|
||||||
case _: IOException =>
|
case _: IOException =>
|
||||||
fatal("Halting due to unrecoverable I/O error while handling producer request: " + e.getMessage, e)
|
fatal("Halting due to unrecoverable I/O error while handling producer request: " + e.getMessage, e)
|
||||||
|
@ -229,8 +249,7 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
val ret = new ProducerResponse(request.versionId, request.correlationId, errors, offsets)
|
new ProducerResponse(request.versionId, request.correlationId, errors, offsets)
|
||||||
ret
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -238,7 +257,10 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
*/
|
*/
|
||||||
def handleFetchRequest(request: RequestChannel.Request) {
|
def handleFetchRequest(request: RequestChannel.Request) {
|
||||||
val fetchRequest = FetchRequest.readFrom(request.request.buffer)
|
val fetchRequest = FetchRequest.readFrom(request.request.buffer)
|
||||||
trace("handling fetch request: " + fetchRequest.toString)
|
if(requestLogger.isTraceEnabled)
|
||||||
|
requestLogger.trace("Handling fetch request " + fetchRequest.toString)
|
||||||
|
trace("Handling fetch request " + fetchRequest.toString)
|
||||||
|
|
||||||
// validate the request
|
// validate the request
|
||||||
try {
|
try {
|
||||||
fetchRequest.validate()
|
fetchRequest.validate()
|
||||||
|
@ -255,12 +277,12 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
var satisfiedProduceRequests = new mutable.ArrayBuffer[DelayedProduce]
|
var satisfiedProduceRequests = new mutable.ArrayBuffer[DelayedProduce]
|
||||||
fetchRequest.offsetInfo.foreach(topicOffsetInfo => {
|
fetchRequest.offsetInfo.foreach(topicOffsetInfo => {
|
||||||
topicOffsetInfo.partitions.foreach(partition => {
|
topicOffsetInfo.partitions.foreach(partition => {
|
||||||
satisfiedProduceRequests ++= produceRequestPurgatory.update(
|
val key = RequestKey(topicOffsetInfo.topic, partition)
|
||||||
(topicOffsetInfo.topic, partition), (topicOffsetInfo.topic, partition)
|
satisfiedProduceRequests ++= producerRequestPurgatory.update(key, key)
|
||||||
)
|
|
||||||
})
|
})
|
||||||
})
|
})
|
||||||
debug("replica %d fetch unblocked %d DelayedProduce requests.".format(fetchRequest.replicaId, satisfiedProduceRequests.size))
|
debug("Replica %d fetch unblocked %d producer requests."
|
||||||
|
.format(fetchRequest.replicaId, satisfiedProduceRequests.size))
|
||||||
satisfiedProduceRequests.foreach(_.respond())
|
satisfiedProduceRequests.foreach(_.respond())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -270,14 +292,15 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
availableBytes >= fetchRequest.minBytes ||
|
availableBytes >= fetchRequest.minBytes ||
|
||||||
fetchRequest.numPartitions <= 0) {
|
fetchRequest.numPartitions <= 0) {
|
||||||
val topicData = readMessageSets(fetchRequest)
|
val topicData = readMessageSets(fetchRequest)
|
||||||
debug("returning fetch response %s for fetch request with correlation id %d".format(topicData.map(_.partitionDataArray.map(_.error).mkString(",")).mkString(","), fetchRequest.correlationId))
|
debug("Returning fetch response %s for fetch request with correlation id %d".format(
|
||||||
|
topicData.map(_.partitionDataArray.map(_.error).mkString(",")).mkString(","), fetchRequest.correlationId))
|
||||||
val response = new FetchResponse(FetchRequest.CurrentVersion, fetchRequest.correlationId, topicData)
|
val response = new FetchResponse(FetchRequest.CurrentVersion, fetchRequest.correlationId, topicData)
|
||||||
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response)))
|
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response)))
|
||||||
} else {
|
} else {
|
||||||
debug("putting fetch request into purgatory")
|
debug("Putting fetch request into purgatory")
|
||||||
// create a list of (topic, partition) pairs to use as keys for this delayed request
|
// create a list of (topic, partition) pairs to use as keys for this delayed request
|
||||||
val topicPartitionPairs: Seq[Any] = fetchRequest.offsetInfo.flatMap(o => o.partitions.map((o.topic, _)))
|
val delayedFetchKeys = fetchRequest.offsetInfo.flatMap(o => o.partitions.map(RequestKey(o.topic, _)))
|
||||||
val delayedFetch = new DelayedFetch(topicPartitionPairs, request, fetchRequest, fetchRequest.maxWait, availableBytes)
|
val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest, fetchRequest.maxWait)
|
||||||
fetchRequestPurgatory.watch(delayedFetch)
|
fetchRequestPurgatory.watch(delayedFetch)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -298,16 +321,18 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
totalBytes += math.min(offsetDetail.fetchSizes(i), available)
|
totalBytes += math.min(offsetDetail.fetchSizes(i), available)
|
||||||
} catch {
|
} catch {
|
||||||
case e: InvalidPartitionException =>
|
case e: InvalidPartitionException =>
|
||||||
info("invalid partition " + offsetDetail.partitions(i) + "in fetch request from client '" + fetchRequest.clientId + "'")
|
info("Invalid partition %d in fetch request from client %d."
|
||||||
|
.format(offsetDetail.partitions(i), fetchRequest.clientId))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
trace(totalBytes + " available bytes for fetch request.")
|
||||||
totalBytes
|
totalBytes
|
||||||
}
|
}
|
||||||
|
|
||||||
private def maybeUpdatePartitionHW(fetchRequest: FetchRequest) {
|
private def maybeUpdatePartitionHW(fetchRequest: FetchRequest) {
|
||||||
val offsets = fetchRequest.offsetInfo
|
val offsets = fetchRequest.offsetInfo
|
||||||
debug("act on update partition HW, check offset detail: %s ".format(offsets))
|
debug("Act on update partition HW, check offset detail: %s ".format(offsets))
|
||||||
for(offsetDetail <- offsets) {
|
for(offsetDetail <- offsets) {
|
||||||
val topic = offsetDetail.topic
|
val topic = offsetDetail.topic
|
||||||
val (partitions, offsets) = (offsetDetail.partitions, offsetDetail.offsets)
|
val (partitions, offsets) = (offsetDetail.partitions, offsetDetail.offsets)
|
||||||
|
@ -343,17 +368,20 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
BrokerTopicStat.getBrokerTopicStat(topic).recordBytesOut(messages.sizeInBytes)
|
BrokerTopicStat.getBrokerTopicStat(topic).recordBytesOut(messages.sizeInBytes)
|
||||||
BrokerTopicStat.getBrokerAllTopicStat.recordBytesOut(messages.sizeInBytes)
|
BrokerTopicStat.getBrokerAllTopicStat.recordBytesOut(messages.sizeInBytes)
|
||||||
val leaderReplicaOpt = replicaManager.getReplica(topic, partition, brokerId)
|
val leaderReplicaOpt = replicaManager.getReplica(topic, partition, brokerId)
|
||||||
assert(leaderReplicaOpt.isDefined, "Leader replica for topic %s partition %d".format(topic, partition) + " must exist on leader broker %d".format(brokerId))
|
assert(leaderReplicaOpt.isDefined, "Leader replica for topic %s partition %d must exist on leader broker %d".format(topic, partition, brokerId))
|
||||||
val leaderReplica = leaderReplicaOpt.get
|
val leaderReplica = leaderReplicaOpt.get
|
||||||
fetchRequest.replicaId match {
|
fetchRequest.replicaId match {
|
||||||
case FetchRequest.NonFollowerId => // replica id value of -1 signifies a fetch request from an external client, not from one of the replicas
|
case FetchRequest.NonFollowerId =>
|
||||||
|
// replica id value of -1 signifies a fetch request from an external client, not from one of the replicas
|
||||||
new PartitionData(partition, ErrorMapping.NoError, offset, leaderReplica.highWatermark(), messages)
|
new PartitionData(partition, ErrorMapping.NoError, offset, leaderReplica.highWatermark(), messages)
|
||||||
case _ => // fetch request from a follower
|
case _ => // fetch request from a follower
|
||||||
val replicaOpt = replicaManager.getReplica(topic, partition, fetchRequest.replicaId)
|
val replicaOpt = replicaManager.getReplica(topic, partition, fetchRequest.replicaId)
|
||||||
assert(replicaOpt.isDefined, "No replica %d in replica manager on %d".format(fetchRequest.replicaId, brokerId))
|
assert(replicaOpt.isDefined, "No replica %d in replica manager on %d".format(fetchRequest.replicaId, brokerId))
|
||||||
val replica = replicaOpt.get
|
val replica = replicaOpt.get
|
||||||
debug("leader [%d] for topic [%s] partition [%d] received fetch request from follower [%d]".format(brokerId, replica.topic, replica.partition.partitionId, fetchRequest.replicaId))
|
debug("Leader for topic [%s] partition [%d] received fetch request from follower [%d]"
|
||||||
debug("Leader %d returning %d messages for topic %s partition %d to follower %d".format(brokerId, messages.sizeInBytes, replica.topic, replica.partition.partitionId, fetchRequest.replicaId))
|
.format(replica.topic, replica.partition.partitionId, fetchRequest.replicaId))
|
||||||
|
debug("Leader returning %d messages for topic %s partition %d to follower %d"
|
||||||
|
.format(messages.sizeInBytes, replica.topic, replica.partition.partitionId, fetchRequest.replicaId))
|
||||||
new PartitionData(partition, ErrorMapping.NoError, offset, leaderReplica.highWatermark(), messages)
|
new PartitionData(partition, ErrorMapping.NoError, offset, leaderReplica.highWatermark(), messages)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -372,7 +400,7 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
try {
|
try {
|
||||||
// check if the current broker is the leader for the partitions
|
// check if the current broker is the leader for the partitions
|
||||||
kafkaZookeeper.ensurePartitionLeaderOnThisBroker(topic, partition)
|
kafkaZookeeper.ensurePartitionLeaderOnThisBroker(topic, partition)
|
||||||
trace("fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize))
|
trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize))
|
||||||
val log = logManager.getLog(topic, partition)
|
val log = logManager.getLog(topic, partition)
|
||||||
response = Right(log match { case Some(l) => l.read(offset, maxSize) case None => MessageSet.Empty })
|
response = Right(log match { case Some(l) => l.read(offset, maxSize) case None => MessageSet.Empty })
|
||||||
} catch {
|
} catch {
|
||||||
|
@ -389,7 +417,9 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
def handleOffsetRequest(request: RequestChannel.Request) {
|
def handleOffsetRequest(request: RequestChannel.Request) {
|
||||||
val offsetRequest = OffsetRequest.readFrom(request.request.buffer)
|
val offsetRequest = OffsetRequest.readFrom(request.request.buffer)
|
||||||
if(requestLogger.isTraceEnabled)
|
if(requestLogger.isTraceEnabled)
|
||||||
requestLogger.trace("offset request " + offsetRequest.toString)
|
requestLogger.trace("Handling offset request " + offsetRequest.toString)
|
||||||
|
trace("Handling offset request " + offsetRequest.toString)
|
||||||
|
|
||||||
var response: OffsetResponse = null
|
var response: OffsetResponse = null
|
||||||
try {
|
try {
|
||||||
kafkaZookeeper.ensurePartitionLeaderOnThisBroker(offsetRequest.topic, offsetRequest.partition)
|
kafkaZookeeper.ensurePartitionLeaderOnThisBroker(offsetRequest.topic, offsetRequest.partition)
|
||||||
|
@ -412,11 +442,14 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
def handleTopicMetadataRequest(request: RequestChannel.Request) {
|
def handleTopicMetadataRequest(request: RequestChannel.Request) {
|
||||||
val metadataRequest = TopicMetadataRequest.readFrom(request.request.buffer)
|
val metadataRequest = TopicMetadataRequest.readFrom(request.request.buffer)
|
||||||
if(requestLogger.isTraceEnabled)
|
if(requestLogger.isTraceEnabled)
|
||||||
requestLogger.trace("topic metadata request " + metadataRequest.toString())
|
requestLogger.trace("Handling topic metadata request " + metadataRequest.toString())
|
||||||
|
trace("Handling topic metadata request " + metadataRequest.toString())
|
||||||
|
|
||||||
val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]()
|
val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]()
|
||||||
val zkClient = kafkaZookeeper.getZookeeperClient
|
val zkClient = kafkaZookeeper.getZookeeperClient
|
||||||
var errorCode = ErrorMapping.NoError
|
var errorCode = ErrorMapping.NoError
|
||||||
val config = logManager.config
|
val config = logManager.config
|
||||||
|
|
||||||
try {
|
try {
|
||||||
val topicMetadataList = AdminUtils.getTopicMetaDataFromZK(metadataRequest.topics, zkClient)
|
val topicMetadataList = AdminUtils.getTopicMetaDataFromZK(metadataRequest.topics, zkClient)
|
||||||
metadataRequest.topics.zip(topicMetadataList).foreach(
|
metadataRequest.topics.zip(topicMetadataList).foreach(
|
||||||
|
@ -452,33 +485,43 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
}
|
}
|
||||||
|
|
||||||
def close() {
|
def close() {
|
||||||
debug("shut down")
|
debug("Shutting down.")
|
||||||
fetchRequestPurgatory.shutdown()
|
fetchRequestPurgatory.shutdown()
|
||||||
produceRequestPurgatory.shutdown()
|
producerRequestPurgatory.shutdown()
|
||||||
debug("shutted down completely")
|
debug("Shut down complete.")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private [kafka] trait MetricKey {
|
||||||
|
def keyLabel: String
|
||||||
|
}
|
||||||
|
private [kafka] object MetricKey {
|
||||||
|
val globalLabel = "all"
|
||||||
|
}
|
||||||
|
|
||||||
|
private [kafka] case class RequestKey(topic: String, partition: Int)
|
||||||
|
extends MetricKey {
|
||||||
|
override def keyLabel = "%s-%d".format(topic, partition)
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* A delayed fetch request
|
* A delayed fetch request
|
||||||
*/
|
*/
|
||||||
class DelayedFetch(keys: Seq[Any], request: RequestChannel.Request, val fetch: FetchRequest, delayMs: Long, initialSize: Long) extends DelayedRequest(keys, request, delayMs) {
|
class DelayedFetch(keys: Seq[RequestKey], request: RequestChannel.Request, val fetch: FetchRequest, delayMs: Long)
|
||||||
val bytesAccumulated = new AtomicLong(initialSize)
|
extends DelayedRequest(keys, request, delayMs)
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A holding pen for fetch requests waiting to be satisfied
|
* A holding pen for fetch requests waiting to be satisfied
|
||||||
*/
|
*/
|
||||||
class FetchRequestPurgatory(brokerId: Int, requestChannel: RequestChannel) extends RequestPurgatory[DelayedFetch, PartitionData]("Fetch Request Purgatory on Broker " + brokerId + ", ") {
|
class FetchRequestPurgatory(requestChannel: RequestChannel) extends RequestPurgatory[DelayedFetch, Null](brokerId) {
|
||||||
|
|
||||||
|
this.logIdent = "FetchRequestPurgatory-%d ".format(brokerId)
|
||||||
|
|
||||||
|
override def metricsGroupIdent = metricsGroup
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A fetch request is satisfied when it has accumulated enough data to meet the min_bytes field
|
* A fetch request is satisfied when it has accumulated enough data to meet the min_bytes field
|
||||||
*/
|
*/
|
||||||
def checkSatisfied(partitionData: PartitionData, delayedFetch: DelayedFetch): Boolean = {
|
def checkSatisfied(n: Null, delayedFetch: DelayedFetch): Boolean =
|
||||||
val messageDataSize = partitionData.messages.sizeInBytes
|
availableFetchBytes(delayedFetch.fetch) >= delayedFetch.fetch.minBytes
|
||||||
val accumulatedSize = delayedFetch.bytesAccumulated.addAndGet(messageDataSize)
|
|
||||||
debug("fetch request check, accm size: " + accumulatedSize + " delay fetch min bytes: " + delayedFetch.fetch.minBytes)
|
|
||||||
accumulatedSize >= delayedFetch.fetch.minBytes
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* When a request expires just answer it with whatever data is present
|
* When a request expires just answer it with whatever data is present
|
||||||
|
@ -486,11 +529,13 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
def expire(delayed: DelayedFetch) {
|
def expire(delayed: DelayedFetch) {
|
||||||
val topicData = readMessageSets(delayed.fetch)
|
val topicData = readMessageSets(delayed.fetch)
|
||||||
val response = new FetchResponse(FetchRequest.CurrentVersion, delayed.fetch.correlationId, topicData)
|
val response = new FetchResponse(FetchRequest.CurrentVersion, delayed.fetch.correlationId, topicData)
|
||||||
|
val fromFollower = delayed.fetch.replicaId != FetchRequest.NonFollowerId
|
||||||
|
delayedRequestMetrics.recordDelayedFetchExpired(fromFollower, response)
|
||||||
requestChannel.sendResponse(new RequestChannel.Response(delayed.request, new FetchResponseSend(response)))
|
requestChannel.sendResponse(new RequestChannel.Response(delayed.request, new FetchResponseSend(response)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class DelayedProduce(keys: Seq[Any],
|
class DelayedProduce(keys: Seq[RequestKey],
|
||||||
request: RequestChannel.Request,
|
request: RequestChannel.Request,
|
||||||
localErrors: Array[Short],
|
localErrors: Array[Short],
|
||||||
requiredOffsets: Array[Long],
|
requiredOffsets: Array[Long],
|
||||||
|
@ -504,7 +549,7 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
* values are effectively synchronized by the ProducerRequestPurgatory's
|
* values are effectively synchronized by the ProducerRequestPurgatory's
|
||||||
* update method
|
* update method
|
||||||
*/
|
*/
|
||||||
private val partitionStatus = keys.map(key => {
|
private [kafka] val partitionStatus = keys.map(key => {
|
||||||
val keyIndex = keys.indexOf(key)
|
val keyIndex = keys.indexOf(key)
|
||||||
// if there was an error in writing to the local replica's log, then don't
|
// if there was an error in writing to the local replica's log, then don't
|
||||||
// wait for acks on this partition
|
// wait for acks on this partition
|
||||||
|
@ -525,13 +570,13 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
|
|
||||||
def respond() {
|
def respond() {
|
||||||
val errorsAndOffsets: (List[Short], List[Long]) = (
|
val errorsAndOffsets: (List[Short], List[Long]) = (
|
||||||
keys.foldRight
|
keys.foldRight
|
||||||
((List[Short](), List[Long]()))
|
((List[Short](), List[Long]()))
|
||||||
((key: Any, result: (List[Short], List[Long])) => {
|
((key: RequestKey, result: (List[Short], List[Long])) => {
|
||||||
val status = partitionStatus(key)
|
val status = partitionStatus(key)
|
||||||
(status.error :: result._1, status.requiredOffset :: result._2)
|
(status.error :: result._1, status.requiredOffset :: result._2)
|
||||||
})
|
})
|
||||||
)
|
)
|
||||||
val response = new ProducerResponse(produce.versionId, produce.correlationId,
|
val response = new ProducerResponse(produce.versionId, produce.correlationId,
|
||||||
errorsAndOffsets._1.toArray, errorsAndOffsets._2.toArray)
|
errorsAndOffsets._1.toArray, errorsAndOffsets._2.toArray)
|
||||||
|
|
||||||
|
@ -550,9 +595,14 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
* As partitions become acknowledged, we may be able to unblock
|
* As partitions become acknowledged, we may be able to unblock
|
||||||
* DelayedFetchRequests that are pending on those partitions.
|
* DelayedFetchRequests that are pending on those partitions.
|
||||||
*/
|
*/
|
||||||
def isSatisfied(followerFetchPartition: (String, Int)) = {
|
def isSatisfied(followerFetchRequestKey: RequestKey) = {
|
||||||
val (topic, partitionId) = followerFetchPartition
|
val topic = followerFetchRequestKey.topic
|
||||||
val fetchPartitionStatus = partitionStatus(followerFetchPartition)
|
val partitionId = followerFetchRequestKey.partition
|
||||||
|
val key = RequestKey(topic, partitionId)
|
||||||
|
val fetchPartitionStatus = partitionStatus(key)
|
||||||
|
val durationNs = SystemTime.nanoseconds - creationTimeNs
|
||||||
|
trace("Checking producer request satisfaction for %s-%d, acksPending = %b"
|
||||||
|
.format(topic, partitionId, fetchPartitionStatus.acksPending))
|
||||||
if (fetchPartitionStatus.acksPending) {
|
if (fetchPartitionStatus.acksPending) {
|
||||||
val leaderReplica = replicaManager.getLeaderReplica(topic, partitionId)
|
val leaderReplica = replicaManager.getLeaderReplica(topic, partitionId)
|
||||||
leaderReplica match {
|
leaderReplica match {
|
||||||
|
@ -560,14 +610,16 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
if (leader.isLocal) {
|
if (leader.isLocal) {
|
||||||
val isr = leader.partition.inSyncReplicas
|
val isr = leader.partition.inSyncReplicas
|
||||||
val numAcks = isr.count(r => {
|
val numAcks = isr.count(r => {
|
||||||
if (!r.isLocal)
|
if (!r.isLocal) {
|
||||||
r.logEndOffset() >= partitionStatus(followerFetchPartition).requiredOffset
|
r.logEndOffset() >= partitionStatus(key).requiredOffset
|
||||||
|
}
|
||||||
else
|
else
|
||||||
true /* also count the local (leader) replica */
|
true /* also count the local (leader) replica */
|
||||||
})
|
})
|
||||||
trace("Received %d/%d acks for produce request to %s-%d".format(
|
|
||||||
|
trace("Received %d/%d acks for producer request to %s-%d; isr size = %d".format(
|
||||||
numAcks, produce.requiredAcks,
|
numAcks, produce.requiredAcks,
|
||||||
topic, partitionId))
|
topic, partitionId, isr.size))
|
||||||
if ((produce.requiredAcks < 0 && numAcks >= isr.size) ||
|
if ((produce.requiredAcks < 0 && numAcks >= isr.size) ||
|
||||||
(produce.requiredAcks > 0 && numAcks >= produce.requiredAcks)) {
|
(produce.requiredAcks > 0 && numAcks >= produce.requiredAcks)) {
|
||||||
/*
|
/*
|
||||||
|
@ -575,12 +627,16 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
* are fully caught up to the (local) leader's offset
|
* are fully caught up to the (local) leader's offset
|
||||||
* corresponding to this produce request.
|
* corresponding to this produce request.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
fetchPartitionStatus.acksPending = false
|
fetchPartitionStatus.acksPending = false
|
||||||
fetchPartitionStatus.error = ErrorMapping.NoError
|
fetchPartitionStatus.error = ErrorMapping.NoError
|
||||||
val topicData =
|
val topicData =
|
||||||
produce.data.find(_.topic == topic).get
|
produce.data.find(_.topic == topic).get
|
||||||
val partitionData =
|
val partitionData =
|
||||||
topicData.partitionDataArray.find(_.partition == partitionId).get
|
topicData.partitionDataArray.find(_.partition == partitionId).get
|
||||||
|
delayedRequestMetrics.recordDelayedProducerKeyCaughtUp(key,
|
||||||
|
durationNs,
|
||||||
|
partitionData.sizeInBytes)
|
||||||
maybeUnblockDelayedFetchRequests(
|
maybeUnblockDelayedFetchRequests(
|
||||||
topic, Array(partitionData))
|
topic, Array(partitionData))
|
||||||
}
|
}
|
||||||
|
@ -597,7 +653,10 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
}
|
}
|
||||||
|
|
||||||
// unblocked if there are no partitions with pending acks
|
// unblocked if there are no partitions with pending acks
|
||||||
! partitionStatus.exists(p => p._2.acksPending)
|
val satisfied = ! partitionStatus.exists(p => p._2.acksPending)
|
||||||
|
if (satisfied)
|
||||||
|
delayedRequestMetrics.recordDelayedProduceSatisfied(durationNs)
|
||||||
|
satisfied
|
||||||
}
|
}
|
||||||
|
|
||||||
class PartitionStatus(var acksPending: Boolean,
|
class PartitionStatus(var acksPending: Boolean,
|
||||||
|
@ -618,18 +677,159 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||||
/**
|
/**
|
||||||
* A holding pen for produce requests waiting to be satisfied.
|
* A holding pen for produce requests waiting to be satisfied.
|
||||||
*/
|
*/
|
||||||
private [kafka] class ProducerRequestPurgatory(brokerId: Int) extends RequestPurgatory[DelayedProduce, (String, Int)]("Producer Request Purgatory on Broker " + brokerId + ", ") {
|
private [kafka] class ProducerRequestPurgatory extends RequestPurgatory[DelayedProduce, RequestKey](brokerId) {
|
||||||
|
|
||||||
protected def checkSatisfied(fetchRequestPartition: (String, Int),
|
|
||||||
|
this.logIdent = "ProducerRequestPurgatory-%d ".format(brokerId)
|
||||||
|
|
||||||
|
override def metricsGroupIdent = metricsGroup
|
||||||
|
|
||||||
|
protected def checkSatisfied(followerFetchRequestKey: RequestKey,
|
||||||
delayedProduce: DelayedProduce) =
|
delayedProduce: DelayedProduce) =
|
||||||
delayedProduce.isSatisfied(fetchRequestPartition)
|
delayedProduce.isSatisfied(followerFetchRequestKey)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handle an expired delayed request
|
* Handle an expired delayed request
|
||||||
*/
|
*/
|
||||||
protected def expire(delayedProduce: DelayedProduce) {
|
protected def expire(delayedProduce: DelayedProduce) {
|
||||||
|
for (partitionStatus <- delayedProduce.partitionStatus if partitionStatus._2.acksPending)
|
||||||
|
delayedRequestMetrics.recordDelayedProducerKeyExpired(partitionStatus._1)
|
||||||
|
|
||||||
delayedProduce.respond()
|
delayedProduce.respond()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private class DelayedRequestMetrics {
|
||||||
|
|
||||||
|
|
||||||
|
private class DelayedProducerRequestMetrics(keyLabel: String = MetricKey.globalLabel) extends KafkaMetricsGroup {
|
||||||
|
override def metricsGroupIdent = metricsGroup
|
||||||
|
val caughtUpFollowerFetchRequestMeter =
|
||||||
|
newMeter("CaughtUpFollowerFetchRequestsPerSecond-" + keyLabel, "requests", TimeUnit.SECONDS)
|
||||||
|
val followerCatchUpTimeHistogram = if (keyLabel == MetricKey.globalLabel)
|
||||||
|
Some(newHistogram("FollowerCatchUpTimeInNs", biased = true))
|
||||||
|
else None
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Note that throughput is updated on individual key satisfaction.
|
||||||
|
* Therefore, it is an upper bound on throughput since the
|
||||||
|
* DelayedProducerRequest may get expired.
|
||||||
|
*/
|
||||||
|
val throughputMeter = newMeter("Throughput-" + keyLabel, "bytes", TimeUnit.SECONDS)
|
||||||
|
val expiredRequestMeter = newMeter("ExpiredRequestsPerSecond-" + keyLabel, "requests", TimeUnit.SECONDS)
|
||||||
|
|
||||||
|
val satisfiedRequestMeter = if (keyLabel == MetricKey.globalLabel)
|
||||||
|
Some(newMeter("SatisfiedRequestsPerSecond", "requests", TimeUnit.SECONDS))
|
||||||
|
else None
|
||||||
|
val satisfactionTimeHistogram = if (keyLabel == MetricKey.globalLabel)
|
||||||
|
Some(newHistogram("SatisfactionTimeInNs", biased = true))
|
||||||
|
else None
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private class DelayedFetchRequestMetrics(forFollower: Boolean,
|
||||||
|
keyLabel: String = MetricKey.globalLabel) extends KafkaMetricsGroup {
|
||||||
|
private val metricPrefix = if (forFollower) "Follower" else "NonFollower"
|
||||||
|
|
||||||
|
override def metricsGroupIdent = metricsGroup
|
||||||
|
val satisfiedRequestMeter = if (keyLabel == MetricKey.globalLabel)
|
||||||
|
Some(newMeter(metricPrefix + "-SatisfiedRequestsPerSecond",
|
||||||
|
"requests", TimeUnit.SECONDS))
|
||||||
|
else None
|
||||||
|
|
||||||
|
val satisfactionTimeHistogram = if (keyLabel == MetricKey.globalLabel)
|
||||||
|
Some(newHistogram(metricPrefix + "-SatisfactionTimeInNs", biased = true))
|
||||||
|
else None
|
||||||
|
|
||||||
|
val expiredRequestMeter = if (keyLabel == MetricKey.globalLabel)
|
||||||
|
Some(newMeter(metricPrefix + "-ExpiredRequestsPerSecond",
|
||||||
|
"requests", TimeUnit.SECONDS))
|
||||||
|
else None
|
||||||
|
|
||||||
|
val throughputMeter = newMeter("%s-Throughput-%s".format(metricPrefix, keyLabel),
|
||||||
|
"bytes", TimeUnit.SECONDS)
|
||||||
|
}
|
||||||
|
|
||||||
|
private val producerRequestMetricsForKey = {
|
||||||
|
val valueFactory = (k: MetricKey) => new DelayedProducerRequestMetrics(k.keyLabel)
|
||||||
|
new Pool[MetricKey, DelayedProducerRequestMetrics](Some(valueFactory))
|
||||||
|
}
|
||||||
|
|
||||||
|
private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics
|
||||||
|
|
||||||
|
private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true)
|
||||||
|
private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false)
|
||||||
|
|
||||||
|
private val followerFetchRequestMetricsForKey = {
|
||||||
|
val valueFactory = (k: MetricKey) => new DelayedFetchRequestMetrics(forFollower = true, k.keyLabel)
|
||||||
|
new Pool[MetricKey, DelayedFetchRequestMetrics](Some(valueFactory))
|
||||||
|
}
|
||||||
|
|
||||||
|
private val nonFollowerFetchRequestMetricsForKey = {
|
||||||
|
val valueFactory = (k: MetricKey) => new DelayedFetchRequestMetrics(forFollower = false, k.keyLabel)
|
||||||
|
new Pool[MetricKey, DelayedFetchRequestMetrics](Some(valueFactory))
|
||||||
|
}
|
||||||
|
|
||||||
|
def recordDelayedProducerKeyExpired(key: MetricKey) {
|
||||||
|
val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key)
|
||||||
|
List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark())
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def recordDelayedProducerKeyCaughtUp(key: MetricKey, timeToCatchUpNs: Long, bytes: Int) {
|
||||||
|
val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key)
|
||||||
|
List(keyMetrics, aggregateProduceRequestMetrics).foreach(m => {
|
||||||
|
m.caughtUpFollowerFetchRequestMeter.mark()
|
||||||
|
m.followerCatchUpTimeHistogram.foreach(_.update(timeToCatchUpNs))
|
||||||
|
m.throughputMeter.mark(bytes)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def recordDelayedProduceSatisfied(timeToSatisfyNs: Long) {
|
||||||
|
aggregateProduceRequestMetrics.satisfiedRequestMeter.foreach(_.mark())
|
||||||
|
aggregateProduceRequestMetrics.satisfactionTimeHistogram.foreach(_.update(timeToSatisfyNs))
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private def recordDelayedFetchThroughput(forFollower: Boolean, response: FetchResponse) {
|
||||||
|
val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics
|
||||||
|
else aggregateNonFollowerFetchRequestMetrics
|
||||||
|
metrics.throughputMeter.mark(response.sizeInBytes)
|
||||||
|
|
||||||
|
response.topicMap.foreach(topicAndData => {
|
||||||
|
val topic = topicAndData._1
|
||||||
|
topicAndData._2.partitionDataArray.foreach(partitionData => {
|
||||||
|
val key = RequestKey(topic, partitionData.partition)
|
||||||
|
val keyMetrics = if (forFollower)
|
||||||
|
followerFetchRequestMetricsForKey.getAndMaybePut(key)
|
||||||
|
else
|
||||||
|
nonFollowerFetchRequestMetricsForKey.getAndMaybePut(key)
|
||||||
|
keyMetrics.throughputMeter.mark(partitionData.sizeInBytes)
|
||||||
|
})
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def recordDelayedFetchExpired(forFollower: Boolean, response: FetchResponse) {
|
||||||
|
val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics
|
||||||
|
else aggregateNonFollowerFetchRequestMetrics
|
||||||
|
|
||||||
|
metrics.expiredRequestMeter.foreach(_.mark())
|
||||||
|
|
||||||
|
recordDelayedFetchThroughput(forFollower, response)
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def recordDelayedFetchSatisfied(forFollower: Boolean, timeToSatisfyNs: Long, response: FetchResponse) {
|
||||||
|
val aggregateMetrics = if (forFollower) aggregateFollowerFetchRequestMetrics
|
||||||
|
else aggregateNonFollowerFetchRequestMetrics
|
||||||
|
|
||||||
|
aggregateMetrics.satisfactionTimeHistogram.foreach(_.update(timeToSatisfyNs))
|
||||||
|
aggregateMetrics.satisfiedRequestMeter.foreach(_.mark())
|
||||||
|
|
||||||
|
recordDelayedFetchThroughput(forFollower, response)
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,6 +24,7 @@ import kafka.consumer.ConsumerConfig
|
||||||
import java.net.InetAddress
|
import java.net.InetAddress
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Configuration settings for the kafka server
|
* Configuration settings for the kafka server
|
||||||
*/
|
*/
|
||||||
|
@ -138,7 +139,7 @@ class KafkaConfig(props: Properties) extends ZKConfig(props) {
|
||||||
val replicaMaxWaitTimeMs = Utils.getInt(props, "replica.fetch.wait.time.ms", 500)
|
val replicaMaxWaitTimeMs = Utils.getInt(props, "replica.fetch.wait.time.ms", 500)
|
||||||
|
|
||||||
/** minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs */
|
/** minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs */
|
||||||
val replicaMinBytes = Utils.getInt(props, "replica.fetch.min.bytes", 4086)
|
val replicaMinBytes = Utils.getInt(props, "replica.fetch.min.bytes", 4096)
|
||||||
|
|
||||||
/* number of fetcher threads used to replicate messages from a source broker.
|
/* number of fetcher threads used to replicate messages from a source broker.
|
||||||
* Increasing this value can increase the degree of I/O parallelism in the follower broker. */
|
* Increasing this value can increase the degree of I/O parallelism in the follower broker. */
|
||||||
|
|
|
@ -5,7 +5,7 @@
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
* 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 not use this file except in compliance with
|
||||||
* the License. You may obtain a copy of the License at
|
* the License. You may obtain a copy of the License at
|
||||||
*
|
*
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
*
|
*
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
@ -23,6 +23,9 @@ import java.util.concurrent._
|
||||||
import java.util.concurrent.atomic._
|
import java.util.concurrent.atomic._
|
||||||
import kafka.network._
|
import kafka.network._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
|
import com.yammer.metrics.core.Gauge
|
||||||
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A request whose processing needs to be delayed for at most the given delayMs
|
* A request whose processing needs to be delayed for at most the given delayMs
|
||||||
|
@ -30,6 +33,7 @@ import kafka.utils._
|
||||||
* for example a key could be a (topic, partition) pair.
|
* for example a key could be a (topic, partition) pair.
|
||||||
*/
|
*/
|
||||||
class DelayedRequest(val keys: Seq[Any], val request: RequestChannel.Request, delayMs: Long) extends DelayedItem[RequestChannel.Request](request, delayMs) {
|
class DelayedRequest(val keys: Seq[Any], val request: RequestChannel.Request, delayMs: Long) extends DelayedItem[RequestChannel.Request](request, delayMs) {
|
||||||
|
val creationTimeNs = SystemTime.nanoseconds
|
||||||
val satisfied = new AtomicBoolean(false)
|
val satisfied = new AtomicBoolean(false)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -58,13 +62,41 @@ class DelayedRequest(val keys: Seq[Any], val request: RequestChannel.Request, de
|
||||||
* this function handles delayed requests that have hit their time limit without being satisfied.
|
* this function handles delayed requests that have hit their time limit without being satisfied.
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) extends Logging{
|
abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) extends Logging with KafkaMetricsGroup {
|
||||||
this.logIdent = logPrefix
|
|
||||||
/* a list of requests watching each key */
|
/* a list of requests watching each key */
|
||||||
private val watchersForKey = new ConcurrentHashMap[Any, Watchers]
|
private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers))
|
||||||
|
|
||||||
|
private val numDelayedRequestsBeanName = "NumDelayedRequests"
|
||||||
|
private val timeToSatisfyHistogramBeanName = "TimeToSatisfyInNs"
|
||||||
|
private val satisfactionRateBeanName = "SatisfactionRate"
|
||||||
|
private val expirationRateBeanName = "ExpirationRate"
|
||||||
|
|
||||||
|
override def metricsGroupIdent = ""
|
||||||
|
|
||||||
|
val satisfactionRateMeter = newMeter(
|
||||||
|
satisfactionRateBeanName,
|
||||||
|
"requests",
|
||||||
|
TimeUnit.SECONDS
|
||||||
|
)
|
||||||
|
|
||||||
|
val timeToSatisfyHistogram = newHistogram(timeToSatisfyHistogramBeanName, biased = true)
|
||||||
|
|
||||||
|
newGauge(
|
||||||
|
numDelayedRequestsBeanName,
|
||||||
|
new Gauge[Int] {
|
||||||
|
def value() = expiredRequestReaper.unsatisfied.get()
|
||||||
|
}
|
||||||
|
)
|
||||||
|
|
||||||
|
val expirationRateMeter = newMeter(
|
||||||
|
expirationRateBeanName,
|
||||||
|
"requests",
|
||||||
|
TimeUnit.SECONDS
|
||||||
|
)
|
||||||
|
|
||||||
/* background thread expiring requests that have been waiting too long */
|
/* background thread expiring requests that have been waiting too long */
|
||||||
private val expiredRequestReaper = new ExpiredRequestReaper(logPrefix)
|
private val expiredRequestReaper = new ExpiredRequestReaper
|
||||||
private val expirationThread = Utils.daemonThread("request-expiration-task", expiredRequestReaper)
|
private val expirationThread = Utils.daemonThread("request-expiration-task", expiredRequestReaper)
|
||||||
expirationThread.start()
|
expirationThread.start()
|
||||||
|
|
||||||
|
@ -90,15 +122,8 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) exten
|
||||||
w.collectSatisfiedRequests(request)
|
w.collectSatisfiedRequests(request)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def watchersFor(key: Any): Watchers = {
|
private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key)
|
||||||
var lst = watchersForKey.get(key)
|
|
||||||
if(lst == null) {
|
|
||||||
watchersForKey.putIfAbsent(key, new Watchers)
|
|
||||||
lst = watchersForKey.get(key)
|
|
||||||
}
|
|
||||||
lst
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if this request satisfied this delayed request
|
* Check if this request satisfied this delayed request
|
||||||
*/
|
*/
|
||||||
|
@ -117,7 +142,8 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) exten
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A linked list of DelayedRequests watching some key with some associated bookeeping logic
|
* A linked list of DelayedRequests watching some key with some associated
|
||||||
|
* bookkeeping logic.
|
||||||
*/
|
*/
|
||||||
private class Watchers {
|
private class Watchers {
|
||||||
|
|
||||||
|
@ -132,10 +158,10 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) exten
|
||||||
|
|
||||||
def add(t: T) {
|
def add(t: T) {
|
||||||
synchronized {
|
synchronized {
|
||||||
requests.add(t)
|
requests.add(t)
|
||||||
liveCount += 1
|
liveCount += 1
|
||||||
maybePurge()
|
maybePurge()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def maybePurge() {
|
private def maybePurge() {
|
||||||
|
@ -151,32 +177,39 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) exten
|
||||||
|
|
||||||
def decLiveCount() {
|
def decLiveCount() {
|
||||||
synchronized {
|
synchronized {
|
||||||
liveCount -= 1
|
liveCount -= 1
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def collectSatisfiedRequests(request: R): Seq[T] = {
|
def collectSatisfiedRequests(request: R): Seq[T] = {
|
||||||
val response = new mutable.ArrayBuffer[T]
|
val response = new mutable.ArrayBuffer[T]
|
||||||
synchronized {
|
synchronized {
|
||||||
val iter = requests.iterator()
|
val iter = requests.iterator()
|
||||||
while(iter.hasNext) {
|
while(iter.hasNext) {
|
||||||
val curr = iter.next
|
val curr = iter.next
|
||||||
if(curr.satisfied.get) {
|
if(curr.satisfied.get) {
|
||||||
// another thread has satisfied this request, remove it
|
// another thread has satisfied this request, remove it
|
||||||
iter.remove()
|
iter.remove()
|
||||||
} else {
|
} else {
|
||||||
if(checkSatisfied(request, curr)) {
|
// synchronize on curr to avoid any race condition with expire
|
||||||
iter.remove()
|
// on client-side.
|
||||||
val updated = curr.satisfied.compareAndSet(false, true)
|
val satisfied = curr synchronized checkSatisfied(request, curr)
|
||||||
if(updated == true) {
|
if(satisfied) {
|
||||||
response += curr
|
iter.remove()
|
||||||
liveCount -= 1
|
val updated = curr.satisfied.compareAndSet(false, true)
|
||||||
expiredRequestReaper.satisfyRequest()
|
if(updated == true) {
|
||||||
}
|
val requestNs = SystemTime.nanoseconds - curr.creationTimeNs
|
||||||
}
|
satisfactionRateMeter.mark()
|
||||||
}
|
timeToSatisfyHistogram.update(requestNs)
|
||||||
}
|
|
||||||
}
|
response += curr
|
||||||
|
liveCount -= 1
|
||||||
|
expiredRequestReaper.satisfyRequest()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
response
|
response
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -184,9 +217,8 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) exten
|
||||||
/**
|
/**
|
||||||
* Runnable to expire requests that have sat unfullfilled past their deadline
|
* Runnable to expire requests that have sat unfullfilled past their deadline
|
||||||
*/
|
*/
|
||||||
private class ExpiredRequestReaper(logPrefix: String) extends Runnable with Logging {
|
private class ExpiredRequestReaper extends Runnable with Logging {
|
||||||
this.logIdent = "ExpiredRequestReaper for " + logPrefix
|
this.logIdent = "ExpiredRequestReaper-%d ".format(brokerId)
|
||||||
|
|
||||||
/* a few magic parameters to help do cleanup to avoid accumulating old watchers */
|
/* a few magic parameters to help do cleanup to avoid accumulating old watchers */
|
||||||
private val CleanupThresholdSize = 100
|
private val CleanupThresholdSize = 100
|
||||||
private val CleanupThresholdPrct = 0.5
|
private val CleanupThresholdPrct = 0.5
|
||||||
|
@ -196,14 +228,16 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) exten
|
||||||
private val shutdownLatch = new CountDownLatch(1)
|
private val shutdownLatch = new CountDownLatch(1)
|
||||||
private val needsPurge = new AtomicBoolean(false)
|
private val needsPurge = new AtomicBoolean(false)
|
||||||
/* The count of elements in the delay queue that are unsatisfied */
|
/* The count of elements in the delay queue that are unsatisfied */
|
||||||
private val unsatisfied = new AtomicInteger(0)
|
private [kafka] val unsatisfied = new AtomicInteger(0)
|
||||||
|
|
||||||
/** Main loop for the expiry thread */
|
/** Main loop for the expiry thread */
|
||||||
def run() {
|
def run() {
|
||||||
while(running.get) {
|
while(running.get) {
|
||||||
try {
|
try {
|
||||||
val curr = pollExpired()
|
val curr = pollExpired()
|
||||||
expire(curr)
|
curr synchronized {
|
||||||
|
expire(curr)
|
||||||
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case ie: InterruptedException =>
|
case ie: InterruptedException =>
|
||||||
if(needsPurge.getAndSet(false)) {
|
if(needsPurge.getAndSet(false)) {
|
||||||
|
@ -232,11 +266,11 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) exten
|
||||||
|
|
||||||
/** Shutdown the expiry thread*/
|
/** Shutdown the expiry thread*/
|
||||||
def shutdown() {
|
def shutdown() {
|
||||||
debug("shutting down")
|
debug("Shutting down.")
|
||||||
running.set(false)
|
running.set(false)
|
||||||
expirationThread.interrupt()
|
expirationThread.interrupt()
|
||||||
shutdownLatch.await()
|
shutdownLatch.await()
|
||||||
debug("shut down completely")
|
debug("Shut down complete.")
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Record the fact that we satisfied a request in the stats for the expiry queue */
|
/** Record the fact that we satisfied a request in the stats for the expiry queue */
|
||||||
|
@ -250,6 +284,7 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) exten
|
||||||
val curr = delayed.take()
|
val curr = delayed.take()
|
||||||
val updated = curr.satisfied.compareAndSet(false, true)
|
val updated = curr.satisfied.compareAndSet(false, true)
|
||||||
if(updated) {
|
if(updated) {
|
||||||
|
expirationRateMeter.mark()
|
||||||
unsatisfied.getAndDecrement()
|
unsatisfied.getAndDecrement()
|
||||||
for(key <- curr.keys)
|
for(key <- curr.keys)
|
||||||
watchersFor(key).decLiveCount()
|
watchersFor(key).decLiveCount()
|
||||||
|
|
|
@ -20,21 +20,45 @@ package kafka.utils
|
||||||
import java.util.ArrayList
|
import java.util.ArrayList
|
||||||
import java.util.concurrent._
|
import java.util.concurrent._
|
||||||
import collection.JavaConversions
|
import collection.JavaConversions
|
||||||
|
import kafka.common.KafkaException
|
||||||
|
|
||||||
class Pool[K,V] extends Iterable[(K, V)] {
|
|
||||||
|
class Pool[K,V](valueFactory: Option[(K) => V] = None) extends Iterable[(K, V)] {
|
||||||
|
|
||||||
private val pool = new ConcurrentHashMap[K, V]
|
private val pool = new ConcurrentHashMap[K, V]
|
||||||
|
|
||||||
def this(m: collection.Map[K, V]) {
|
def this(m: collection.Map[K, V]) {
|
||||||
this()
|
this()
|
||||||
for((k,v) <- m.elements)
|
m.foreach(kv => pool.put(kv._1, kv._2))
|
||||||
pool.put(k, v)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
def put(k: K, v: V) = pool.put(k, v)
|
def put(k: K, v: V) = pool.put(k, v)
|
||||||
|
|
||||||
def putIfNotExists(k: K, v: V) = pool.putIfAbsent(k, v)
|
def putIfNotExists(k: K, v: V) = pool.putIfAbsent(k, v)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the value associated with the given key. If there is no associated
|
||||||
|
* value, then create the value using the pool's value factory and return the
|
||||||
|
* value associated with the key. The user should declare the factory method
|
||||||
|
* as lazy if its side-effects need to be avoided.
|
||||||
|
*
|
||||||
|
* @param key The key to lookup.
|
||||||
|
* @return The final value associated with the key. This may be different from
|
||||||
|
* the value created by the factory if another thread successfully
|
||||||
|
* put a value.
|
||||||
|
*/
|
||||||
|
def getAndMaybePut(key: K) = {
|
||||||
|
if (valueFactory.isEmpty)
|
||||||
|
throw new KafkaException("Empty value factory in pool.")
|
||||||
|
val curr = pool.get(key)
|
||||||
|
if (curr == null) {
|
||||||
|
pool.putIfAbsent(key, valueFactory.get(key))
|
||||||
|
pool.get(key)
|
||||||
|
}
|
||||||
|
else
|
||||||
|
curr
|
||||||
|
}
|
||||||
|
|
||||||
def contains(id: K) = pool.containsKey(id)
|
def contains(id: K) = pool.containsKey(id)
|
||||||
|
|
||||||
def get(key: K): V = pool.get(key)
|
def get(key: K): V = pool.get(key)
|
||||||
|
@ -46,7 +70,7 @@ class Pool[K,V] extends Iterable[(K, V)] {
|
||||||
def values: Iterable[V] =
|
def values: Iterable[V] =
|
||||||
JavaConversions.asIterable(new ArrayList[V](pool.values()))
|
JavaConversions.asIterable(new ArrayList[V](pool.values()))
|
||||||
|
|
||||||
def clear: Unit = pool.clear()
|
def clear() { pool.clear() }
|
||||||
|
|
||||||
override def size = pool.size
|
override def size = pool.size
|
||||||
|
|
||||||
|
|
|
@ -76,7 +76,7 @@ class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness wit
|
||||||
fail("shouldn't reach here in SimpleConsumer since log file is corrupted.")
|
fail("shouldn't reach here in SimpleConsumer since log file is corrupted.")
|
||||||
fail("shouldn't reach here in SimpleConsumer since log file is corrupted.")
|
fail("shouldn't reach here in SimpleConsumer since log file is corrupted.")
|
||||||
} catch {
|
} catch {
|
||||||
case e: InvalidMessageSizeException => println("This is good")
|
case e: InvalidMessageSizeException => "This is good"
|
||||||
}
|
}
|
||||||
|
|
||||||
// test ZookeeperConsumer
|
// test ZookeeperConsumer
|
||||||
|
|
|
@ -103,8 +103,10 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
val logManager = EasyMock.createMock(classOf[LogManager])
|
val logManager = EasyMock.createMock(classOf[LogManager])
|
||||||
val kafkaZookeeper = EasyMock.createMock(classOf[KafkaZooKeeper])
|
val kafkaZookeeper = EasyMock.createMock(classOf[KafkaZooKeeper])
|
||||||
val replicaManager = EasyMock.createMock(classOf[ReplicaManager])
|
val replicaManager = EasyMock.createMock(classOf[ReplicaManager])
|
||||||
|
EasyMock.expect(replicaManager.config).andReturn(configs.head)
|
||||||
EasyMock.expect(kafkaZookeeper.getZookeeperClient).andReturn(zkClient)
|
EasyMock.expect(kafkaZookeeper.getZookeeperClient).andReturn(zkClient)
|
||||||
EasyMock.expect(logManager.config).andReturn(configs.head)
|
EasyMock.expect(logManager.config).andReturn(configs.head)
|
||||||
|
EasyMock.replay(replicaManager)
|
||||||
EasyMock.replay(logManager)
|
EasyMock.replay(logManager)
|
||||||
EasyMock.replay(kafkaZookeeper)
|
EasyMock.replay(kafkaZookeeper)
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,60 @@
|
||||||
|
package unit.kafka.metrics
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
import org.junit.Test
|
||||||
|
import org.scalatest.junit.JUnit3Suite
|
||||||
|
import java.util.concurrent.TimeUnit
|
||||||
|
import junit.framework.Assert._
|
||||||
|
import kafka.metrics.KafkaTimer
|
||||||
|
import com.yammer.metrics.core.{MetricsRegistry, Clock}
|
||||||
|
|
||||||
|
class KafkaTimerTest extends JUnit3Suite {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testKafkaTimer() {
|
||||||
|
val clock = new ManualClock
|
||||||
|
val testRegistry = new MetricsRegistry(clock)
|
||||||
|
val metric = testRegistry.newTimer(this.getClass, "TestTimer")
|
||||||
|
|
||||||
|
val timer = new KafkaTimer(metric)
|
||||||
|
timer.time {
|
||||||
|
clock.addMillis(1000)
|
||||||
|
}
|
||||||
|
assertEquals(1, metric.count())
|
||||||
|
assertTrue((metric.max() - 1000).abs <= Double.Epsilon)
|
||||||
|
assertTrue((metric.min() - 1000).abs <= Double.Epsilon)
|
||||||
|
}
|
||||||
|
|
||||||
|
private class ManualClock extends Clock {
|
||||||
|
|
||||||
|
private var ticksInNanos = 0L
|
||||||
|
|
||||||
|
override def tick() = {
|
||||||
|
ticksInNanos
|
||||||
|
}
|
||||||
|
|
||||||
|
override def time() = {
|
||||||
|
TimeUnit.NANOSECONDS.toMillis(ticksInNanos)
|
||||||
|
}
|
||||||
|
|
||||||
|
def addMillis(millis: Long) {
|
||||||
|
ticksInNanos += TimeUnit.MILLISECONDS.toNanos(millis)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,26 +18,28 @@
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
import org.junit.{After, Before, Test}
|
import org.junit.Test
|
||||||
import junit.framework.Assert._
|
import junit.framework.Assert._
|
||||||
import kafka.message._
|
import kafka.message._
|
||||||
import kafka.api._
|
import kafka.api._
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
|
import org.scalatest.junit.JUnit3Suite
|
||||||
|
|
||||||
class RequestPurgatoryTest {
|
|
||||||
|
class RequestPurgatoryTest extends JUnit3Suite {
|
||||||
|
|
||||||
val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes)))
|
val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes)))
|
||||||
val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes)))
|
val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes)))
|
||||||
var purgatory: MockRequestPurgatory = null
|
var purgatory: MockRequestPurgatory = null
|
||||||
|
|
||||||
@Before
|
override def setUp() {
|
||||||
def setup() {
|
super.setUp()
|
||||||
purgatory = new MockRequestPurgatory()
|
purgatory = new MockRequestPurgatory()
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
override def tearDown() {
|
||||||
def teardown() {
|
|
||||||
purgatory.shutdown()
|
purgatory.shutdown()
|
||||||
|
super.tearDown()
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -54,7 +56,7 @@ class RequestPurgatoryTest {
|
||||||
assertEquals("Nothing satisfied", 0, purgatory.update("test1", producerRequest2).size)
|
assertEquals("Nothing satisfied", 0, purgatory.update("test1", producerRequest2).size)
|
||||||
purgatory.satisfied += r2
|
purgatory.satisfied += r2
|
||||||
assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2", producerRequest2))
|
assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2", producerRequest2))
|
||||||
assertEquals("Nothing satisfied", 0, purgatory.update("test2", producerRequest2).size)
|
assertEquals("Nothing satisfied", 0, purgatory.update("test2", producerRequest2).size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -73,7 +75,7 @@ class RequestPurgatoryTest {
|
||||||
assertTrue("Time for expiration was about 20ms", (elapsed - expiration).abs < 10L)
|
assertTrue("Time for expiration was about 20ms", (elapsed - expiration).abs < 10L)
|
||||||
}
|
}
|
||||||
|
|
||||||
class MockRequestPurgatory extends RequestPurgatory[DelayedRequest, ProducerRequest]("Mock Request Purgatory") {
|
class MockRequestPurgatory extends RequestPurgatory[DelayedRequest, ProducerRequest] {
|
||||||
val satisfied = mutable.Set[DelayedRequest]()
|
val satisfied = mutable.Set[DelayedRequest]()
|
||||||
val expired = mutable.Set[DelayedRequest]()
|
val expired = mutable.Set[DelayedRequest]()
|
||||||
def awaitExpiration(delayed: DelayedRequest) = {
|
def awaitExpiration(delayed: DelayedRequest) = {
|
||||||
|
|
|
@ -16,7 +16,7 @@
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import sbt._
|
import sbt._
|
||||||
import scala.xml.{Node, Elem, NodeSeq}
|
import scala.xml.{Node, Elem}
|
||||||
import scala.xml.transform.{RewriteRule, RuleTransformer}
|
import scala.xml.transform.{RewriteRule, RuleTransformer}
|
||||||
|
|
||||||
class KafkaProject(info: ProjectInfo) extends ParentProject(info) with IdeaProject {
|
class KafkaProject(info: ProjectInfo) extends ParentProject(info) with IdeaProject {
|
||||||
|
@ -60,11 +60,11 @@ class KafkaProject(info: ProjectInfo) extends ParentProject(info) with IdeaProje
|
||||||
|
|
||||||
def zkClientDep =
|
def zkClientDep =
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>zkclient</groupId>
|
<groupId>zkclient</groupId>
|
||||||
<artifactId>zkclient</artifactId>
|
<artifactId>zkclient</artifactId>
|
||||||
<version>20120522</version>
|
<version>20120522</version>
|
||||||
<scope>compile</scope>
|
<scope>compile</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
object ZkClientDepAdder extends RuleTransformer(new RewriteRule() {
|
object ZkClientDepAdder extends RuleTransformer(new RewriteRule() {
|
||||||
override def transform(node: Node): Seq[Node] = node match {
|
override def transform(node: Node): Seq[Node] = node match {
|
||||||
|
@ -251,6 +251,8 @@ class KafkaProject(info: ProjectInfo) extends ParentProject(info) with IdeaProje
|
||||||
trait CoreDependencies {
|
trait CoreDependencies {
|
||||||
val log4j = "log4j" % "log4j" % "1.2.15"
|
val log4j = "log4j" % "log4j" % "1.2.15"
|
||||||
val jopt = "net.sf.jopt-simple" % "jopt-simple" % "3.2"
|
val jopt = "net.sf.jopt-simple" % "jopt-simple" % "3.2"
|
||||||
|
val metricsCore = "com.yammer.metrics" % "metrics-core" % "latest.release"
|
||||||
|
val slf4jSimple = "org.slf4j" % "slf4j-simple" % "latest.release"
|
||||||
}
|
}
|
||||||
|
|
||||||
trait HadoopDependencies {
|
trait HadoopDependencies {
|
||||||
|
@ -264,5 +266,4 @@ class KafkaProject(info: ProjectInfo) extends ParentProject(info) with IdeaProje
|
||||||
trait CompressionDependencies {
|
trait CompressionDependencies {
|
||||||
val snappy = "org.xerial.snappy" % "snappy-java" % "1.0.4.1"
|
val snappy = "org.xerial.snappy" % "snappy-java" % "1.0.4.1"
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue