Merge branch '0.8' into trunk

Conflicts:
	core/src/main/scala/kafka/log/Log.scala
	core/src/test/scala/unit/kafka/admin/AdminTest.scala
	core/src/test/scala/unit/kafka/log/LogTest.scala
	core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
This commit is contained in:
Jay Kreps 2012-12-18 09:43:41 -08:00
commit 32dae955d5
122 changed files with 1573 additions and 1017 deletions

View File

@ -1,17 +0,0 @@
#!/bin/bash
# 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.
$(dirname $0)/kafka-run-class.sh kafka.tools.ProducerShell $@

View File

@ -0,0 +1,18 @@
@echo off
rem Licensed to the Apache Software Foundation (ASF) under one or more
rem contributor license agreements. See the NOTICE file distributed with
rem this work for additional information regarding copyright ownership.
rem The ASF licenses this file to You under the Apache License, Version 2.0
rem (the "License"); you may not use this file except in compliance with
rem the License. You may obtain a copy of the License at
rem
rem http://www.apache.org/licenses/LICENSE-2.0
rem
rem Unless required by applicable law or agreed to in writing, software
rem distributed under the License is distributed on an "AS IS" BASIS,
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
rem See the License for the specific language governing permissions and
rem limitations under the License.
wmic process where (commandline like "%%kafka.Kafka%%" and not name="wmic.exe") delete
rem ps ax | grep -i 'kafka.Kafka' | grep -v grep | awk '{print $1}' | xargs kill -SIGTERM

View File

@ -0,0 +1,17 @@
@echo off
rem Licensed to the Apache Software Foundation (ASF) under one or more
rem contributor license agreements. See the NOTICE file distributed with
rem this work for additional information regarding copyright ownership.
rem The ASF licenses this file to You under the Apache License, Version 2.0
rem (the "License"); you may not use this file except in compliance with
rem the License. You may obtain a copy of the License at
rem
rem http://www.apache.org/licenses/LICENSE-2.0
rem
rem Unless required by applicable law or agreed to in writing, software
rem distributed under the License is distributed on an "AS IS" BASIS,
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
rem See the License for the specific language governing permissions and
rem limitations under the License.
wmic process where (commandline like "%%zookeeper%%" and not name="wmic.exe") delete

View File

@ -109,7 +109,7 @@ public class KafkaETLContext {
// read data from queue
URI uri = _request.getURI();
_consumer = new SimpleConsumer(uri.getHost(), uri.getPort(), _timeout, _bufferSize);
_consumer = new SimpleConsumer(uri.getHost(), uri.getPort(), _timeout, _bufferSize, "KafkaETLContext");
// get available offset range
_offsetRange = getOffsetRange();

View File

@ -25,7 +25,8 @@ import org.I0Itec.zkclient.ZkClient
import org.I0Itec.zkclient.exception.ZkNodeExistsException
import scala.collection._
import scala.collection.mutable
import kafka.common.{BrokerNotAvailableException, LeaderNotAvailableException, ReplicaNotAvailableException, ErrorMapping}
import kafka.common._
import scala.Some
object AdminUtils extends Logging {
val rand = new Random
@ -82,7 +83,7 @@ object AdminUtils extends Logging {
ZkUtils.createPersistentPath(zkClient, zkPath, jsonPartitionMap)
debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionMap))
} catch {
case e: ZkNodeExistsException => throw new AdministrationException("topic %s already exists".format(topic))
case e: ZkNodeExistsException => throw new TopicExistsException("topic %s already exists".format(topic))
case e2 => throw new AdministrationException(e2.toString)
}
}

View File

@ -21,6 +21,7 @@ import joptsimple.OptionParser
import kafka.utils._
import org.I0Itec.zkclient.ZkClient
import scala.collection.mutable
import kafka.common.Topic
object CreateTopicCommand extends Logging {

View File

@ -30,10 +30,10 @@ case class PartitionFetchInfo(offset: Long, fetchSize: Int)
object FetchRequest {
val CurrentVersion = 1.shortValue()
val CurrentVersion = 0.shortValue
val DefaultMaxWait = 0
val DefaultMinBytes = 0
val ReplicaFetcherClientId = "replica fetcher"
val ReplicaFetcherClientId = "replica-fetcher"
val DefaultCorrelationId = 0
def readFrom(buffer: ByteBuffer): FetchRequest = {

View File

@ -27,29 +27,25 @@ import kafka.api.ApiUtils._
object FetchResponsePartitionData {
def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = {
val error = buffer.getShort
val initialOffset = buffer.getLong
val hw = buffer.getLong
val messageSetSize = buffer.getInt
val messageSetBuffer = buffer.slice()
messageSetBuffer.limit(messageSetSize)
buffer.position(buffer.position + messageSetSize)
new FetchResponsePartitionData(error, initialOffset,
hw, new ByteBufferMessageSet(messageSetBuffer))
new FetchResponsePartitionData(error, hw, new ByteBufferMessageSet(messageSetBuffer))
}
val headerSize =
2 + /* error code */
8 + /* initialOffset */
8 + /* high watermark */
4 /* messageSetSize */
}
case class FetchResponsePartitionData(error: Short = ErrorMapping.NoError,
initialOffset:Long = 0L, hw: Long = -1L, messages: MessageSet) {
case class FetchResponsePartitionData(error: Short = ErrorMapping.NoError, hw: Long = -1L, messages: MessageSet) {
val sizeInBytes = FetchResponsePartitionData.headerSize + messages.sizeInBytes
def this(messages: MessageSet) = this(ErrorMapping.NoError, 0L, -1L, messages)
def this(messages: MessageSet) = this(ErrorMapping.NoError, -1L, messages)
}
@ -63,7 +59,6 @@ class PartitionDataSend(val partitionId: Int,
private val buffer = ByteBuffer.allocate( 4 /** partitionId **/ + FetchResponsePartitionData.headerSize)
buffer.putInt(partitionId)
buffer.putShort(partitionData.error)
buffer.putLong(partitionData.initialOffset)
buffer.putLong(partitionData.hw)
buffer.putInt(partitionData.messages.sizeInBytes)
buffer.rewind()
@ -141,12 +136,10 @@ class TopicDataSend(val topicData: TopicData) extends Send {
object FetchResponse {
val headerSize =
2 + /* versionId */
4 + /* correlationId */
4 /* topic count */
def readFrom(buffer: ByteBuffer): FetchResponse = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
@ -156,13 +149,12 @@ object FetchResponse {
(TopicAndPartition(topicData.topic, partitionId), partitionData)
}
})
FetchResponse(versionId, correlationId, Map(pairs:_*))
FetchResponse(correlationId, Map(pairs:_*))
}
}
case class FetchResponse(versionId: Short,
correlationId: Int,
case class FetchResponse(correlationId: Int,
data: Map[TopicAndPartition, FetchResponsePartitionData]) {
/**
@ -211,7 +203,6 @@ class FetchResponseSend(val fetchResponse: FetchResponse) extends Send {
private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize)
buffer.putInt(size)
buffer.putShort(fetchResponse.versionId)
buffer.putInt(fetchResponse.correlationId)
buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count
buffer.rewind()

View File

@ -79,7 +79,7 @@ case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndContr
}
object LeaderAndIsrRequest {
val CurrentVersion = 1.shortValue()
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
val IsInit: Boolean = true
val NotInit: Boolean = false
@ -87,6 +87,7 @@ object LeaderAndIsrRequest {
def readFrom(buffer: ByteBuffer): LeaderAndIsrRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val ackTimeoutMs = buffer.getInt
val controllerEpoch = buffer.getInt
@ -106,11 +107,12 @@ object LeaderAndIsrRequest {
for (i <- 0 until leadersCount)
leaders += Broker.readFrom(buffer)
new LeaderAndIsrRequest(versionId, clientId, ackTimeoutMs, partitionStateInfos.toMap, leaders, controllerEpoch)
new LeaderAndIsrRequest(versionId, correlationId, clientId, ackTimeoutMs, partitionStateInfos.toMap, leaders, controllerEpoch)
}
}
case class LeaderAndIsrRequest (versionId: Short,
correlationId: Int,
clientId: String,
ackTimeoutMs: Int,
partitionStateInfos: Map[(String, Int), PartitionStateInfo],
@ -119,12 +121,13 @@ case class LeaderAndIsrRequest (versionId: Short,
extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) {
def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], liveBrokers: Set[Broker], controllerEpoch: Int) = {
this(LeaderAndIsrRequest.CurrentVersion, LeaderAndIsrRequest.DefaultClientId, LeaderAndIsrRequest.DefaultAckTimeout,
this(LeaderAndIsrRequest.CurrentVersion, 0, LeaderAndIsrRequest.DefaultClientId, LeaderAndIsrRequest.DefaultAckTimeout,
partitionStateInfos, liveBrokers, controllerEpoch)
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(ackTimeoutMs)
buffer.putInt(controllerEpoch)
@ -141,6 +144,7 @@ case class LeaderAndIsrRequest (versionId: Short,
def sizeInBytes(): Int = {
var size =
2 /* version id */ +
4 /* correlation id */ +
(2 + clientId.length) /* client id */ +
4 /* ack timeout */ +
4 /* controller epoch */ +

View File

@ -26,7 +26,7 @@ import collection.Map
object LeaderAndIsrResponse {
def readFrom(buffer: ByteBuffer): LeaderAndIsrResponse = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val errorCode = buffer.getShort
val numEntries = buffer.getInt
val responseMap = new HashMap[(String, Int), Short]()
@ -36,18 +36,18 @@ object LeaderAndIsrResponse {
val partitionErrorCode = buffer.getShort
responseMap.put((topic, partition), partitionErrorCode)
}
new LeaderAndIsrResponse(versionId, responseMap, errorCode)
new LeaderAndIsrResponse(correlationId, responseMap, errorCode)
}
}
case class LeaderAndIsrResponse(versionId: Short,
case class LeaderAndIsrResponse(correlationId: Int,
responseMap: Map[(String, Int), Short],
errorCode: Short = ErrorMapping.NoError)
extends RequestOrResponse {
def sizeInBytes(): Int ={
var size =
2 /* version id */ +
4 /* correlation id */ +
2 /* error code */ +
4 /* number of responses */
for ((key, value) <- responseMap) {
@ -60,7 +60,7 @@ case class LeaderAndIsrResponse(versionId: Short,
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
buffer.putShort(errorCode)
buffer.putInt(responseMap.size)
for ((key:(String, Int), value) <- responseMap){

View File

@ -23,7 +23,7 @@ import kafka.api.ApiUtils._
object OffsetRequest {
val CurrentVersion = 1.shortValue()
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
val SmallestTimeString = "smallest"
@ -33,6 +33,7 @@ object OffsetRequest {
def readFrom(buffer: ByteBuffer): OffsetRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val replicaId = buffer.getInt
val topicCount = buffer.getInt
@ -54,16 +55,18 @@ case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int)
case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo],
versionId: Short = OffsetRequest.CurrentVersion,
correlationId: Int = 0,
clientId: String = OffsetRequest.DefaultClientId,
replicaId: Int = Request.OrdinaryConsumerId)
extends RequestOrResponse(Some(RequestKeys.OffsetsKey)) {
def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, OffsetRequest.DefaultClientId, replicaId)
def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], correlationId: Int, replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, correlationId, OffsetRequest.DefaultClientId, replicaId)
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(replicaId)
@ -83,6 +86,7 @@ case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequ
def sizeInBytes =
2 + /* versionId */
4 + /* correlationId */
shortStringLength(clientId) +
4 + /* replicaId */
4 + /* topic count */

View File

@ -25,7 +25,7 @@ import kafka.api.ApiUtils._
object OffsetResponse {
def readFrom(buffer: ByteBuffer): OffsetResponse = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val numTopics = buffer.getInt
val pairs = (1 to numTopics).flatMap(_ => {
val topic = readShortString(buffer)
@ -38,7 +38,7 @@ object OffsetResponse {
(TopicAndPartition(topic, partition), PartitionOffsetsResponse(error, offsets))
})
})
OffsetResponse(versionId, Map(pairs:_*))
OffsetResponse(correlationId, Map(pairs:_*))
}
}
@ -47,7 +47,7 @@ object OffsetResponse {
case class PartitionOffsetsResponse(error: Short, offsets: Seq[Long])
case class OffsetResponse(versionId: Short,
case class OffsetResponse(correlationId: Int,
partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse])
extends RequestOrResponse {
@ -56,7 +56,7 @@ case class OffsetResponse(versionId: Short,
def hasError = partitionErrorAndOffsets.values.exists(_.error != ErrorMapping.NoError)
val sizeInBytes = {
2 + /* versionId */
4 + /* correlation id */
4 + /* topic count */
offsetsGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => {
val (topic, errorAndOffsetsMap) = currTopic
@ -74,7 +74,7 @@ case class OffsetResponse(versionId: Short,
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
buffer.putInt(offsetsGroupedByTopic.size) // topic count
offsetsGroupedByTopic.foreach {
case((topic, errorAndOffsetsMap)) =>

View File

@ -25,7 +25,7 @@ import kafka.api.ApiUtils._
object ProducerRequest {
val CurrentVersion: Short = 0
val CurrentVersion = 0.shortValue
def readFrom(buffer: ByteBuffer): ProducerRequest = {
val versionId: Short = buffer.getShort
@ -57,7 +57,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
clientId: String,
requiredAcks: Short,
ackTimeoutMs: Int,
data: Map[TopicAndPartition, MessageSet])
data: Map[TopicAndPartition, ByteBufferMessageSet])
extends RequestOrResponse(Some(RequestKeys.ProduceKey)) {
/**
@ -69,7 +69,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
clientId: String,
requiredAcks: Short,
ackTimeoutMs: Int,
data: Map[TopicAndPartition, MessageSet]) =
data: Map[TopicAndPartition, ByteBufferMessageSet]) =
this(ProducerRequest.CurrentVersion, correlationId, clientId, requiredAcks, ackTimeoutMs, data)
def writeTo(buffer: ByteBuffer) {
@ -88,7 +88,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
topicAndPartitionData.foreach(partitionAndData => {
val partition = partitionAndData._1.partition
val partitionMessageData = partitionAndData._2
val bytes = partitionMessageData.asInstanceOf[ByteBufferMessageSet].buffer
val bytes = partitionMessageData.buffer
buffer.putInt(partition)
buffer.putInt(bytes.limit)
buffer.put(bytes)

View File

@ -25,7 +25,6 @@ import kafka.api.ApiUtils._
object ProducerResponse {
def readFrom(buffer: ByteBuffer): ProducerResponse = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val topicCount = buffer.getInt
val statusPairs = (1 to topicCount).flatMap(_ => {
@ -39,15 +38,14 @@ object ProducerResponse {
})
})
ProducerResponse(versionId, correlationId, Map(statusPairs:_*))
ProducerResponse(correlationId, Map(statusPairs:_*))
}
}
case class ProducerResponseStatus(error: Short, offset: Long)
case class ProducerResponse(versionId: Short,
correlationId: Int,
case class ProducerResponse(correlationId: Int,
status: Map[TopicAndPartition, ProducerResponseStatus]) extends RequestOrResponse {
/**
@ -59,7 +57,6 @@ case class ProducerResponse(versionId: Short,
val sizeInBytes = {
val groupedStatus = statusGroupedByTopic
2 + /* version id */
4 + /* correlation id */
4 + /* topic count */
groupedStatus.foldLeft (0) ((foldedTopics, currTopic) => {
@ -76,8 +73,6 @@ case class ProducerResponse(versionId: Short,
def writeTo(buffer: ByteBuffer) {
val groupedStatus = statusGroupedByTopic
buffer.putShort(versionId)
buffer.putInt(correlationId)
buffer.putInt(groupedStatus.size) // topic count

View File

@ -25,12 +25,13 @@ import kafka.network.InvalidRequestException
object StopReplicaRequest extends Logging {
val CurrentVersion = 1.shortValue()
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
val DefaultAckTimeout = 100
def readFrom(buffer: ByteBuffer): StopReplicaRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val ackTimeoutMs = buffer.getInt
val controllerEpoch = buffer.getInt
@ -45,11 +46,12 @@ object StopReplicaRequest extends Logging {
(1 to topicPartitionPairCount) foreach { _ =>
topicPartitionPairSet.add(readShortString(buffer), buffer.getInt)
}
StopReplicaRequest(versionId, clientId, ackTimeoutMs, deletePartitions, topicPartitionPairSet.toSet, controllerEpoch)
StopReplicaRequest(versionId, correlationId, clientId, ackTimeoutMs, deletePartitions, topicPartitionPairSet.toSet, controllerEpoch)
}
}
case class StopReplicaRequest(versionId: Short,
correlationId: Int,
clientId: String,
ackTimeoutMs: Int,
deletePartitions: Boolean,
@ -58,12 +60,13 @@ case class StopReplicaRequest(versionId: Short,
extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) {
def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerEpoch: Int) = {
this(StopReplicaRequest.CurrentVersion, StopReplicaRequest.DefaultClientId, StopReplicaRequest.DefaultAckTimeout,
this(StopReplicaRequest.CurrentVersion, 0, StopReplicaRequest.DefaultClientId, StopReplicaRequest.DefaultAckTimeout,
deletePartitions, partitions, controllerEpoch)
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(ackTimeoutMs)
buffer.putInt(controllerEpoch)
@ -78,6 +81,7 @@ case class StopReplicaRequest(versionId: Short,
def sizeInBytes(): Int = {
var size =
2 + /* versionId */
4 + /* correlation id */
ApiUtils.shortStringLength(clientId) +
4 + /* ackTimeoutMs */
4 + /* controller epoch */

View File

@ -26,7 +26,7 @@ import kafka.api.ApiUtils._
object StopReplicaResponse {
def readFrom(buffer: ByteBuffer): StopReplicaResponse = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val errorCode = buffer.getShort
val numEntries = buffer.getInt
@ -37,17 +37,17 @@ object StopReplicaResponse {
val partitionErrorCode = buffer.getShort()
responseMap.put((topic, partition), partitionErrorCode)
}
new StopReplicaResponse(versionId, responseMap.toMap, errorCode)
new StopReplicaResponse(correlationId, responseMap.toMap, errorCode)
}
}
case class StopReplicaResponse(val versionId: Short,
case class StopReplicaResponse(val correlationId: Int,
val responseMap: Map[(String, Int), Short],
val errorCode: Short = ErrorMapping.NoError) extends RequestOrResponse{
def sizeInBytes(): Int ={
var size =
2 /* version id */ +
4 /* correlation id */ +
2 /* error code */ +
4 /* number of responses */
for ((key, value) <- responseMap) {
@ -60,7 +60,7 @@ case class StopReplicaResponse(val versionId: Short,
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
buffer.putShort(errorCode)
buffer.putInt(responseMap.size)
for ((key:(String, Int), value) <- responseMap){

View File

@ -21,57 +21,29 @@ import kafka.cluster.Broker
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.utils.Logging
import collection.mutable.ListBuffer
import kafka.common.{KafkaException, ErrorMapping}
/**
* topic (2 bytes + topic.length)
* number of partitions (4 bytes)
*
* partition id (4 bytes)
*
* does leader exist (1 byte)
* leader info (4 + creator.length + host.length + 4 (port) + 4 (id))
* number of replicas (2 bytes)
* replica info (4 + creator.length + host.length + 4 (port) + 4 (id))
* number of in sync replicas (2 bytes)
* replica info (4 + creator.length + host.length + 4 (port) + 4 (id))
*
* does log metadata exist (1 byte)
* number of log segments (4 bytes)
* total size of log in bytes (8 bytes)
*
* number of log segments (4 bytes)
* beginning offset (8 bytes)
* last modified timestamp (8 bytes)
* size of log segment (8 bytes)
*
*/
sealed trait LeaderRequest { def requestId: Byte }
case object LeaderExists extends LeaderRequest { val requestId: Byte = 1 }
case object LeaderDoesNotExist extends LeaderRequest { val requestId: Byte = 0 }
import collection.mutable.ArrayBuffer
import kafka.common._
object TopicMetadata {
def readFrom(buffer: ByteBuffer): TopicMetadata = {
val NoLeaderNodeId = -1
def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): TopicMetadata = {
val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue))
val topic = readShortString(buffer)
val numPartitions = readIntInRange(buffer, "number of partitions", (0, Int.MaxValue))
val partitionsMetadata = new ListBuffer[PartitionMetadata]()
val partitionsMetadata = new ArrayBuffer[PartitionMetadata]()
for(i <- 0 until numPartitions)
partitionsMetadata += PartitionMetadata.readFrom(buffer)
partitionsMetadata += PartitionMetadata.readFrom(buffer, brokers)
new TopicMetadata(topic, partitionsMetadata, errorCode)
}
}
case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError) extends Logging {
def sizeInBytes: Int = {
var size: Int = 2 /* error code */
size += shortStringLength(topic)
size += partitionsMetadata.foldLeft(4 /* number of partitions */)(_ + _.sizeInBytes)
debug("Size of topic metadata = " + size)
size
2 /* error code */ +
shortStringLength(topic) +
4 + partitionsMetadata.map(_.sizeInBytes).sum /* size and partition data array */
}
def writeTo(buffer: ByteBuffer) {
@ -87,40 +59,24 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat
object PartitionMetadata {
def readFrom(buffer: ByteBuffer): PartitionMetadata = {
def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): PartitionMetadata = {
val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue))
val partitionId = readIntInRange(buffer, "partition id", (0, Int.MaxValue)) /* partition id */
val doesLeaderExist = getLeaderRequest(buffer.get)
val leader = doesLeaderExist match {
case LeaderExists => /* leader exists */
Some(Broker.readFrom(buffer))
case LeaderDoesNotExist => None
}
val leaderId = buffer.getInt
val leader = brokers.get(leaderId)
/* list of all replicas */
val numReplicas = readShortInRange(buffer, "number of all replicas", (0, Short.MaxValue))
val replicas = new Array[Broker](numReplicas)
for(i <- 0 until numReplicas) {
replicas(i) = Broker.readFrom(buffer)
}
val numReplicas = readIntInRange(buffer, "number of all replicas", (0, Int.MaxValue))
val replicaIds = (0 until numReplicas).map(_ => buffer.getInt)
val replicas = replicaIds.map(brokers)
/* list of in-sync replicas */
val numIsr = readShortInRange(buffer, "number of in-sync replicas", (0, Short.MaxValue))
val isr = new Array[Broker](numIsr)
for(i <- 0 until numIsr) {
isr(i) = Broker.readFrom(buffer)
}
val numIsr = readIntInRange(buffer, "number of in-sync replicas", (0, Int.MaxValue))
val isrIds = (0 until numIsr).map(_ => buffer.getInt)
val isr = isrIds.map(brokers)
new PartitionMetadata(partitionId, leader, replicas, isr, errorCode)
}
private def getLeaderRequest(requestId: Byte): LeaderRequest = {
requestId match {
case LeaderExists.requestId => LeaderExists
case LeaderDoesNotExist.requestId => LeaderDoesNotExist
case _ => throw new KafkaException("Unknown leader request id " + requestId)
}
}
}
case class PartitionMetadata(partitionId: Int,
@ -129,42 +85,28 @@ case class PartitionMetadata(partitionId: Int,
isr: Seq[Broker] = Seq.empty,
errorCode: Short = ErrorMapping.NoError) extends Logging {
def sizeInBytes: Int = {
var size: Int = 2 /* error code */ + 4 /* partition id */ + 1 /* if leader exists*/
leader match {
case Some(l) => size += l.sizeInBytes
case None =>
}
size += 2 /* number of replicas */
size += replicas.foldLeft(0)(_ + _.sizeInBytes)
size += 2 /* number of in sync replicas */
size += isr.foldLeft(0)(_ + _.sizeInBytes)
debug("Size of partition metadata = " + size)
size
2 /* error code */ +
4 /* partition id */ +
4 /* leader */ +
4 + 4 * replicas.size /* replica array */ +
4 + 4 * isr.size /* isr array */
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(errorCode)
buffer.putInt(partitionId)
/* if leader exists*/
leader match {
case Some(l) =>
buffer.put(LeaderExists.requestId)
/* leader id host_name port */
l.writeTo(buffer)
case None => buffer.put(LeaderDoesNotExist.requestId)
}
/* leader */
val leaderId = if(leader.isDefined) leader.get.id else TopicMetadata.NoLeaderNodeId
buffer.putInt(leaderId)
/* number of replicas */
buffer.putShort(replicas.size.toShort)
replicas.foreach(r => r.writeTo(buffer))
buffer.putInt(replicas.size)
replicas.foreach(r => buffer.putInt(r.id))
/* number of in-sync replicas */
buffer.putShort(isr.size.toShort)
isr.foreach(r => r.writeTo(buffer))
buffer.putInt(isr.size)
isr.foreach(r => buffer.putInt(r.id))
}
}

View File

@ -23,7 +23,7 @@ import collection.mutable.ListBuffer
import kafka.utils.Logging
object TopicMetadataRequest extends Logging {
val CurrentVersion = 1.shortValue()
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
/**
@ -33,6 +33,7 @@ object TopicMetadataRequest extends Logging {
def readFrom(buffer: ByteBuffer): TopicMetadataRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val numTopics = readIntInRange(buffer, "number of topics", (0, Int.MaxValue))
val topics = new ListBuffer[String]()
@ -40,26 +41,28 @@ object TopicMetadataRequest extends Logging {
topics += readShortString(buffer)
val topicsList = topics.toList
debug("topic = %s".format(topicsList.head))
new TopicMetadataRequest(versionId, clientId, topics.toList)
new TopicMetadataRequest(versionId, clientId, topics.toList, correlationId)
}
}
case class TopicMetadataRequest(val versionId: Short,
val clientId: String,
val topics: Seq[String])
val topics: Seq[String],
val correlationId: Int)
extends RequestOrResponse(Some(RequestKeys.MetadataKey)){
def this(topics: Seq[String]) =
this(TopicMetadataRequest.CurrentVersion, TopicMetadataRequest.DefaultClientId, topics)
this(TopicMetadataRequest.CurrentVersion, TopicMetadataRequest.DefaultClientId, topics, 0)
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId) // correlation id not set yet
writeShortString(buffer, clientId)
buffer.putInt(topics.size)
topics.foreach(topic => writeShortString(buffer, topic))
}
def sizeInBytes(): Int = {
2 + (2 + clientId.length) + 4 /* number of topics */ + topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */
2 + 4 + shortStringLength(clientId) + 4 /* number of topics */ + topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */
}
}

View File

@ -17,30 +17,43 @@
package kafka.api
import kafka.cluster.Broker
import java.nio.ByteBuffer
object TopicMetadataResponse {
def readFrom(buffer: ByteBuffer): TopicMetadataResponse = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val brokerCount = buffer.getInt
val brokers = (0 until brokerCount).map(_ => Broker.readFrom(buffer))
val brokerMap = brokers.map(b => (b.id, b)).toMap
val topicCount = buffer.getInt
val topicsMetadata = new Array[TopicMetadata](topicCount)
for( i <- 0 until topicCount) {
topicsMetadata(i) = TopicMetadata.readFrom(buffer)
}
new TopicMetadataResponse(versionId, topicsMetadata.toSeq)
val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap))
new TopicMetadataResponse(topicsMetadata, correlationId)
}
}
case class TopicMetadataResponse(versionId: Short,
topicsMetadata: Seq[TopicMetadata]) extends RequestOrResponse
{
val sizeInBytes = 2 + topicsMetadata.foldLeft(4)(_ + _.sizeInBytes)
case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata],
correlationId: Int) extends RequestOrResponse {
val sizeInBytes: Int = {
val brokers = extractBrokers(topicsMetadata).values
4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
/* brokers */
val brokers = extractBrokers(topicsMetadata).values
buffer.putInt(brokers.size)
brokers.foreach(_.writeTo(buffer))
/* topic metadata */
buffer.putInt(topicsMetadata.length)
topicsMetadata.foreach(_.writeTo(buffer))
}
def extractBrokers(topicMetadatas: Seq[TopicMetadata]): Map[Int, Broker] = {
val parts = topicsMetadata.flatMap(_.partitionsMetadata)
val brokers = (parts.flatMap(_.replicas)) ++ (parts.map(_.leader).collect{case Some(l) => l})
brokers.map(b => (b.id, b)).toMap
}
}

View File

@ -6,20 +6,28 @@ import kafka.api._
import kafka.producer._
import kafka.common.KafkaException
import kafka.utils.{Utils, Logging}
import java.util.Properties
/**
* Helper functions common to clients (producer, consumer, or admin)
*/
object ClientUtils extends Logging{
def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker]): TopicMetadataResponse = {
/**
* Used by the producer to send a metadata request since it has access to the ProducerConfig
* @param topics The topics for which the metadata needs to be fetched
* @param brokers The brokers in the cluster as configured on the producer through broker.list
* @param producerConfig The producer's config
* @return topic metadata response
*/
def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], producerConfig: ProducerConfig): TopicMetadataResponse = {
var fetchMetaDataSucceeded: Boolean = false
var i: Int = 0
val topicMetadataRequest = new TopicMetadataRequest(topics.toSeq)
var topicMetadataResponse: TopicMetadataResponse = null
var t: Throwable = null
while(i < brokers.size && !fetchMetaDataSucceeded) {
val producer: SyncProducer = ProducerPool.createSyncProducer(None, brokers(i))
val producer: SyncProducer = ProducerPool.createSyncProducer(producerConfig, brokers(i))
info("Fetching metadata for topic %s".format(topics))
try {
topicMetadataResponse = producer.send(topicMetadataRequest)
@ -40,6 +48,21 @@ object ClientUtils extends Logging{
return topicMetadataResponse
}
/**
* Used by a non-producer client to send a metadata request
* @param topics The topics for which the metadata needs to be fetched
* @param brokers The brokers in the cluster as configured on the client
* @param clientId The client's identifier
* @return topic metadata response
*/
def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String): TopicMetadataResponse = {
val props = new Properties()
props.put("broker.list", brokers.map(_.getConnectionString()).mkString(","))
props.put("clientid", clientId)
val producerConfig = new ProducerConfig(props)
fetchTopicMetadata(topics, brokers, producerConfig)
}
/**
* Parse a list of broker urls in the form host1:port1, host2:port2, ...
*/
@ -52,8 +75,7 @@ object ClientUtils extends Logging{
val brokerInfos = brokerStr.split(":")
val hostName = brokerInfos(0)
val port = brokerInfos(1).toInt
val creatorId = hostName + "-" + System.currentTimeMillis()
new Broker(brokerId, creatorId, hostName, port)
new Broker(brokerId, hostName, port)
})
}

View File

@ -31,36 +31,32 @@ private[kafka] object Broker {
if(brokerInfoString == null)
throw new BrokerNotAvailableException("Broker id %s does not exist".format(id))
val brokerInfo = brokerInfoString.split(":")
new Broker(id, brokerInfo(0), brokerInfo(1), brokerInfo(2).toInt)
new Broker(id, brokerInfo(0), brokerInfo(1).toInt)
}
def readFrom(buffer: ByteBuffer): Broker = {
val id = buffer.getInt
val creatorId = readShortString(buffer)
val host = readShortString(buffer)
val port = buffer.getInt
new Broker(id, creatorId, host, port)
new Broker(id, host, port)
}
}
private[kafka] case class Broker(val id: Int, val creatorId: String, val host: String, val port: Int) {
private[kafka] case class Broker(val id: Int, val host: String, val port: Int) {
override def toString(): String = new String("id:" + id + ",creatorId:" + creatorId + ",host:" + host + ",port:" + port)
override def toString(): String = new String("id:" + id + ",host:" + host + ",port:" + port)
def getZKString(): String = new String(creatorId + ":" + host + ":" + port)
def getZkString(): String = host + ":" + port
def getConnectionString(): String = host + ":" + port
def writeTo(buffer: ByteBuffer) {
buffer.putInt(id)
writeShortString(buffer, creatorId)
writeShortString(buffer, host)
buffer.putInt(port)
}
def sizeInBytes: Int = {
val size = shortStringLength(creatorId) + shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/
debug("Size of broker info = " + size)
size
}
def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/
override def equals(obj: Any): Boolean = {
obj match {

View File

@ -0,0 +1,26 @@
package kafka.common
/**
* 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.
*/
/**
* Convenience case class since (clientId, brokerInfo) pairs are used to create
* SyncProducer Request Stats and SimpleConsumer Request and Response Stats.
*/
case class ClientIdAndBroker(clientId: String, brokerInfo: String) {
override def toString = "%s-%s".format(clientId, brokerInfo)
}

View File

@ -0,0 +1,27 @@
package kafka.common
/**
* 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.
*/
/**
* Convenience case class since (clientId, topic) pairs are used in the creation
* of many Stats objects.
*/
case class ClientIdAndTopic(clientId: String, topic: String) {
override def toString = "%s-%s".format(clientId, topic)
}

View File

@ -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.common
import util.matching.Regex
import kafka.utils.Logging
trait Config extends Logging {
def validateChars(prop: String, value: String) {
val legalChars = "[a-zA-Z0-9_-]"
val rgx = new Regex(legalChars + "*")
rgx.findFirstIn(value) match {
case Some(t) =>
if (!t.equals(value))
throw new InvalidConfigException(prop + " " + value + " is illegal, contains a character other than ASCII alphanumerics, _ and -")
case None => throw new InvalidConfigException(prop + " " + value + " is illegal, contains a character other than ASCII alphanumerics, _ and -")
}
}
}

View File

@ -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.common
import util.matching.Regex
object Topic {
private val legalChars = "[a-zA-Z0-9_-]"
private val maxNameLength = 255
private val rgx = new Regex(legalChars + "+")
def validate(topic: String) {
if (topic.length <= 0)
throw new InvalidTopicException("topic name is illegal, can't be empty")
else if (topic.length > maxNameLength)
throw new InvalidTopicException("topic name is illegal, can't be longer than " + maxNameLength + " characters")
rgx.findFirstIn(topic) match {
case Some(t) =>
if (!t.equals(topic))
throw new InvalidTopicException("topic name " + topic + " is illegal, contains a character other than ASCII alphanumerics, _ and -")
case None => throw new InvalidTopicException("topic name " + topic + " is illegal, contains a character other than ASCII alphanumerics, _ and -")
}
}
}

View File

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

View File

@ -89,7 +89,7 @@ object ConsoleConsumer extends Logging {
.withRequiredArg
.describedAs("class")
.ofType(classOf[String])
.defaultsTo(classOf[NewlineMessageFormatter].getName)
.defaultsTo(classOf[DefaultMessageFormatter].getName)
val messageFormatterArgOpt = parser.accepts("property")
.withRequiredArg
.describedAs("prop")
@ -176,6 +176,7 @@ object ConsoleConsumer extends Logging {
}
})
var numMessages = 0L
val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter]
formatter.init(formatterArgs)
try {
@ -188,6 +189,7 @@ object ConsoleConsumer extends Logging {
for(messageAndTopic <- iter) {
try {
formatter.writeTo(messageAndTopic.key, messageAndTopic.message, System.out)
numMessages += 1
} catch {
case e =>
if (skipMessageOnError)
@ -198,6 +200,7 @@ object ConsoleConsumer extends Logging {
if(System.out.checkError()) {
// This means no one is listening to our output stream any more, time to shutdown
System.err.println("Unable to write to standard out, closing consumer.")
System.err.println("Consumed %d messages".format(numMessages))
formatter.close()
connector.shutdown()
System.exit(1)
@ -206,6 +209,7 @@ object ConsoleConsumer extends Logging {
} catch {
case e => error("Error processing message, stopping consumer: ", e)
}
System.out.println("Consumed %d messages".format(numMessages))
System.out.flush()
formatter.close()
connector.shutdown()
@ -256,10 +260,27 @@ trait MessageFormatter {
def close() {}
}
class NewlineMessageFormatter extends MessageFormatter {
class DefaultMessageFormatter extends MessageFormatter {
var printKey = false
var keySeparator = "\t".getBytes
var lineSeparator = "\n".getBytes
override def init(props: Properties) {
if(props.containsKey("print.key"))
printKey = props.getProperty("print.key").trim.toLowerCase.equals("true")
if(props.containsKey("key.separator"))
keySeparator = props.getProperty("key.separator").getBytes
if(props.containsKey("line.separator"))
lineSeparator = props.getProperty("line.separator").getBytes
}
def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {
if(printKey) {
output.write(key)
output.write(keySeparator)
}
output.write(value)
output.write('\n')
output.write(lineSeparator)
}
}

View File

@ -19,16 +19,17 @@ package kafka.consumer
import java.util.Properties
import kafka.api.OffsetRequest
import kafka.utils.{VerifiableProperties, ZKConfig}
import kafka.utils._
import kafka.common.{InvalidConfigException, Config}
object ConsumerConfig {
object ConsumerConfig extends Config {
val SocketTimeout = 30 * 1000
val SocketBufferSize = 64*1024
val FetchSize = 1024 * 1024
val MaxFetchSize = 10*FetchSize
val DefaultFetcherBackoffMs = 1000
val AutoCommit = true
val AutoCommitInterval = 10 * 1000
val AutoCommitInterval = 60 * 1000
val MaxQueuedChunks = 10
val MaxRebalanceRetries = 4
val AutoOffsetReset = OffsetRequest.SmallestTimeString
@ -43,6 +44,28 @@ object ConsumerConfig {
val MirrorTopicsBlacklistProp = "mirror.topics.blacklist"
val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads"
val DefaultClientId = ""
def validate(config: ConsumerConfig) {
validateClientId(config.clientId)
validateGroupId(config.groupId)
validateAutoOffsetReset(config.autoOffsetReset)
}
def validateClientId(clientId: String) {
validateChars("clientid", clientId)
}
def validateGroupId(groupId: String) {
validateChars("groupid", groupId)
}
def validateAutoOffsetReset(autoOffsetReset: String) {
autoOffsetReset match {
case OffsetRequest.SmallestTimeString =>
case OffsetRequest.LargestTimeString =>
case _ => throw new InvalidConfigException("Wrong value " + autoOffsetReset + " of autoOffsetReset in ConsumerConfig")
}
}
}
class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) {
@ -109,8 +132,10 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
val enableShallowIterator = props.getBoolean("shallowiterator.enable", false)
/**
* Cliient id is specified by the kafka consumer client, used to distinguish different clients
* Client id is specified by the kafka consumer client, used to distinguish different clients
*/
val clientId = props.getString("clientid", groupId)
validate(this)
}

View File

@ -54,7 +54,7 @@ class ConsumerFetcherManager(private val consumerIdString: String,
try {
trace("Partitions without leader %s".format(noLeaderPartitionSet))
val brokers = getAllBrokersInCluster(zkClient)
val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, brokers).topicsMetadata
val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, brokers, config.clientId).topicsMetadata
val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker]
topicsMetadata.foreach(
tmd => {

View File

@ -34,12 +34,14 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk
consumerTimeoutMs: Int,
private val keyDecoder: Decoder[K],
private val valueDecoder: Decoder[V],
val enableShallowIterator: Boolean)
val enableShallowIterator: Boolean,
val clientId: String)
extends IteratorTemplate[MessageAndMetadata[K, V]] with Logging {
private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null)
private var currentTopicInfo: PartitionTopicInfo = null
private var consumedOffset: Long = -1L
private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId)
override def next(): MessageAndMetadata[K, V] = {
val item = super.next()
@ -48,8 +50,8 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk
currentTopicInfo.resetConsumeOffset(consumedOffset)
val topic = currentTopicInfo.topic
trace("Setting %s consumed offset to %d".format(topic, consumedOffset))
ConsumerTopicStat.getConsumerTopicStat(topic).messageRate.mark()
ConsumerTopicStat.getConsumerAllTopicStat().messageRate.mark()
consumerTopicStats.getConsumerTopicStats(topic).messageRate.mark()
consumerTopicStats.getConsumerAllTopicStats().messageRate.mark()
item
}

View File

@ -1,40 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import kafka.utils.{Pool, threadsafe, Logging}
import java.util.concurrent.TimeUnit
import kafka.metrics.KafkaMetricsGroup
@threadsafe
class ConsumerTopicStat(name: String) extends KafkaMetricsGroup {
val messageRate = newMeter(name + "MessagesPerSec", "messages", TimeUnit.SECONDS)
val byteRate = newMeter(name + "BytesPerSec", "bytes", TimeUnit.SECONDS)
}
object ConsumerTopicStat extends Logging {
private val valueFactory = (k: String) => new ConsumerTopicStat(k)
private val stats = new Pool[String, ConsumerTopicStat](Some(valueFactory))
private val allTopicStat = new ConsumerTopicStat("AllTopics")
def getConsumerAllTopicStat(): ConsumerTopicStat = allTopicStat
def getConsumerTopicStat(topic: String): ConsumerTopicStat = {
stats.getAndMaybePut(topic + "-")
}
}

View File

@ -0,0 +1,57 @@
/**
* 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.consumer
import kafka.utils.{Pool, threadsafe, Logging}
import java.util.concurrent.TimeUnit
import kafka.metrics.KafkaMetricsGroup
import kafka.common.ClientIdAndTopic
@threadsafe
class ConsumerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup {
val messageRate = newMeter(metricId + "-MessagesPerSec", "messages", TimeUnit.SECONDS)
val byteRate = newMeter(metricId + "-BytesPerSec", "bytes", TimeUnit.SECONDS)
}
/**
* Tracks metrics for each topic the given consumer client has consumed data from.
* @param clientId The clientId of the given consumer client.
*/
class ConsumerTopicStats(clientId: String) extends Logging {
private val valueFactory = (k: ClientIdAndTopic) => new ConsumerTopicMetrics(k)
private val stats = new Pool[ClientIdAndTopic, ConsumerTopicMetrics](Some(valueFactory))
private val allTopicStats = new ConsumerTopicMetrics(new ClientIdAndTopic(clientId, "All.Topics")) // to differentiate from a topic named AllTopics
def getConsumerAllTopicStats(): ConsumerTopicMetrics = allTopicStats
def getConsumerTopicStats(topic: String): ConsumerTopicMetrics = {
stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic))
}
}
/**
* Stores the topic stats information of each consumer client in a (clientId -> ConsumerTopicStats) map.
*/
object ConsumerTopicStatsRegistry {
private val valueFactory = (k: String) => new ConsumerTopicStats(k)
private val globalStats = new Pool[String, ConsumerTopicStats](Some(valueFactory))
def getConsumerTopicStat(clientId: String) = {
globalStats.getAndMaybePut(clientId)
}
}

View File

@ -0,0 +1,58 @@
/**
* 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.consumer
import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
import kafka.utils.Pool
import java.util.concurrent.TimeUnit
import kafka.common.ClientIdAndBroker
class FetchRequestAndResponseMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup {
val requestTimer = new KafkaTimer(newTimer(metricId + "-FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
val requestSizeHist = newHistogram(metricId + "-FetchResponseSize")
}
/**
* Tracks metrics of the requests made by a given consumer client to all brokers, and the responses obtained from the brokers.
* @param clientId ClientId of the given consumer
*/
class FetchRequestAndResponseStats(clientId: String) {
private val valueFactory = (k: ClientIdAndBroker) => new FetchRequestAndResponseMetrics(k)
private val stats = new Pool[ClientIdAndBroker, FetchRequestAndResponseMetrics](Some(valueFactory))
private val allBrokersStats = new FetchRequestAndResponseMetrics(new ClientIdAndBroker(clientId, "All.Brokers"))
def getFetchRequestAndResponseAllBrokersStats(): FetchRequestAndResponseMetrics = allBrokersStats
def getFetchRequestAndResponseStats(brokerInfo: String): FetchRequestAndResponseMetrics = {
stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo))
}
}
/**
* Stores the fetch request and response stats information of each consumer client in a (clientId -> FetchRequestAndResponseStats) map.
*/
object FetchRequestAndResponseStatsRegistry {
private val valueFactory = (k: String) => new FetchRequestAndResponseStats(k)
private val globalStats = new Pool[String, FetchRequestAndResponseStats](Some(valueFactory))
def getFetchRequestAndResponseStats(clientId: String) = {
globalStats.getAndMaybePut(clientId)
}
}

View File

@ -26,11 +26,12 @@ class KafkaStream[K,V](private val queue: BlockingQueue[FetchedDataChunk],
consumerTimeoutMs: Int,
private val keyDecoder: Decoder[K],
private val valueDecoder: Decoder[V],
val enableShallowIterator: Boolean)
val enableShallowIterator: Boolean,
val clientId: String)
extends Iterable[MessageAndMetadata[K,V]] with java.lang.Iterable[MessageAndMetadata[K,V]] {
private val iter: ConsumerIterator[K,V] =
new ConsumerIterator[K,V](queue, consumerTimeoutMs, keyDecoder, valueDecoder, enableShallowIterator)
new ConsumerIterator[K,V](queue, consumerTimeoutMs, keyDecoder, valueDecoder, enableShallowIterator, clientId)
/**
* Create an iterator over messages in the stream.

View File

@ -28,11 +28,14 @@ class PartitionTopicInfo(val topic: String,
private val chunkQueue: BlockingQueue[FetchedDataChunk],
private val consumedOffset: AtomicLong,
private val fetchedOffset: AtomicLong,
private val fetchSize: AtomicInteger) extends Logging {
private val fetchSize: AtomicInteger,
private val clientId: String) extends Logging {
debug("initial consumer offset of " + this + " is " + consumedOffset.get)
debug("initial fetch offset of " + this + " is " + fetchedOffset.get)
private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId)
def getConsumeOffset() = consumedOffset.get
def getFetchOffset() = fetchedOffset.get
@ -58,8 +61,8 @@ class PartitionTopicInfo(val topic: String,
chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get))
fetchedOffset.set(next)
debug("updated fetch offset of (%s) to %d".format(this, next))
ConsumerTopicStat.getConsumerTopicStat(topic).byteRate.mark(size)
ConsumerTopicStat.getConsumerAllTopicStat().byteRate.mark(size)
consumerTopicStats.getConsumerTopicStats(topic).byteRate.mark(size)
consumerTopicStats.getConsumerAllTopicStats().byteRate.mark(size)
}
}

View File

@ -20,8 +20,6 @@ package kafka.consumer
import kafka.api._
import kafka.network._
import kafka.utils._
import java.util.concurrent.TimeUnit
import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
import kafka.utils.ZkUtils._
import collection.immutable
import kafka.common.{TopicAndPartition, KafkaException}
@ -30,19 +28,23 @@ import kafka.cluster.Broker
object SimpleConsumer extends Logging {
def earliestOrLatestOffset(broker: Broker, topic: String, partitionId: Int, earliestOrLatest: Long,
def earliestOrLatestOffset(broker: Broker,
topic: String,
partitionId: Int,
earliestOrLatest: Long,
clientId: String,
isFromOrdinaryConsumer: Boolean): Long = {
var simpleConsumer: SimpleConsumer = null
var producedOffset: Long = -1L
try {
simpleConsumer = new SimpleConsumer(broker.host, broker.port, ConsumerConfig.SocketTimeout,
ConsumerConfig.SocketBufferSize)
ConsumerConfig.SocketBufferSize, clientId)
val topicAndPartition = TopicAndPartition(topic, partitionId)
val request = if(isFromOrdinaryConsumer)
new OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)))
else
new OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)),
Request.DebuggingConsumerId)
0, Request.DebuggingConsumerId)
producedOffset = simpleConsumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
} catch {
case e =>
@ -55,15 +57,20 @@ object SimpleConsumer extends Logging {
producedOffset
}
def earliestOrLatestOffset(zkClient: ZkClient, topic: String, brokerId: Int, partitionId: Int,
earliestOrLatest: Long, isFromOrdinaryConsumer: Boolean = true): Long = {
def earliestOrLatestOffset(zkClient: ZkClient,
topic: String,
brokerId: Int,
partitionId: Int,
earliestOrLatest: Long,
clientId: String,
isFromOrdinaryConsumer: Boolean = true): Long = {
val cluster = getCluster(zkClient)
val broker = cluster.getBroker(brokerId) match {
case Some(b) => b
case None => throw new KafkaException("Broker " + brokerId + " is unavailable. Cannot issue " +
"getOffsetsBefore request")
}
earliestOrLatestOffset(broker, topic, partitionId, earliestOrLatest, isFromOrdinaryConsumer)
earliestOrLatestOffset(broker, topic, partitionId, earliestOrLatest, clientId, isFromOrdinaryConsumer)
}
}
@ -75,10 +82,14 @@ object SimpleConsumer extends Logging {
class SimpleConsumer(val host: String,
val port: Int,
val soTimeout: Int,
val bufferSize: Int) extends Logging {
val bufferSize: Int,
val clientId: String) extends Logging {
ConsumerConfig.validateClientId(clientId)
private val lock = new Object()
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 def connect(): BlockingChannel = {
close
@ -143,12 +154,17 @@ class SimpleConsumer(val host: String,
*/
def fetch(request: FetchRequest): FetchResponse = {
var response: Receive = null
FetchRequestAndResponseStat.requestTimer.time {
response = sendRequest(request)
val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestTimer
val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer
aggregateTimer.time {
specificTimer.time {
response = sendRequest(request)
}
}
val fetchResponse = FetchResponse.readFrom(response.buffer)
val fetchedSize = fetchResponse.sizeInBytes
FetchRequestAndResponseStat.respondSizeHist.update(fetchedSize)
fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestSizeHist.update(fetchedSize)
fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize)
fetchResponse
}
@ -166,7 +182,3 @@ class SimpleConsumer(val host: String,
}
}
object FetchRequestAndResponseStat extends KafkaMetricsGroup {
val requestTimer = new KafkaTimer(newTimer("FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
val respondSizeHist = newHistogram("FetchResponseSize")
}

View File

@ -20,7 +20,7 @@ package kafka.consumer
import java.util.concurrent._
import java.util.concurrent.atomic._
import locks.ReentrantLock
import scala.collection._
import collection._
import kafka.cluster._
import kafka.utils._
import org.I0Itec.zkclient.exception.ZkNodeExistsException
@ -35,7 +35,7 @@ import kafka.client.ClientUtils
import com.yammer.metrics.core.Gauge
import kafka.api.OffsetRequest
import kafka.metrics._
import kafka.producer.ProducerConfig
import scala.Some
/**
@ -80,6 +80,7 @@ private[kafka] object ZookeeperConsumerConnector {
private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val enableFetcher: Boolean) // for testing only
extends ConsumerConnector with Logging with KafkaMetricsGroup {
private val isShuttingDown = new AtomicBoolean(false)
private val rebalanceLock = new Object
private var fetcher: Option[ConsumerFetcherManager] = None
@ -198,7 +199,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
threadIdSet.map(_ => {
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks)
val stream = new KafkaStream[K,V](
queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.enableShallowIterator)
queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.enableShallowIterator, config.clientId)
(queue, stream)
})
).flatten.toList
@ -402,7 +403,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic
val consumersPerTopicMap = getConsumersPerTopic(zkClient, group)
val brokers = getAllBrokersInCluster(zkClient)
val topicsMetadata = ClientUtils.fetchTopicMetadata(myTopicThreadIdsMap.keySet, brokers).topicsMetadata
val topicsMetadata = ClientUtils.fetchTopicMetadata(myTopicThreadIdsMap.keySet, brokers, config.clientId).topicsMetadata
val partitionsPerTopicMap = new mutable.HashMap[String, Seq[Int]]
val leaderIdForPartitionsMap = new mutable.HashMap[(String, Int), Int]
topicsMetadata.foreach(m =>{
@ -598,11 +599,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
case None =>
config.autoOffsetReset match {
case OffsetRequest.SmallestTimeString =>
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.EarliestTime)
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.EarliestTime, config.clientId)
case OffsetRequest.LargestTimeString =>
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.LatestTime)
case _ =>
throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig")
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.LatestTime, config.clientId)
}
}
val queue = topicThreadIdAndQueues.get((topic, consumerThreadId))
@ -614,7 +613,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
queue,
consumedOffset,
fetchedOffset,
new AtomicInteger(config.fetchSize))
new AtomicInteger(config.fetchSize),
config.clientId)
partTopicInfoMap.put(partition, partTopicInfo)
debug(partTopicInfo + " selected new offset " + offset)
}
@ -670,7 +670,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val q = e._2._1
topicThreadIdAndQueues.put(topicThreadId, q)
newGauge(
config.groupId + "-" + topicThreadId._1 + "-" + topicThreadId._2 + "-FetchQueueSize",
config.clientId + "-" + config.groupId + "-" + topicThreadId._1 + "-" + topicThreadId._2 + "-FetchQueueSize",
new Gauge[Int] {
def getValue = q.size
}
@ -717,7 +717,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
config.consumerTimeoutMs,
keyDecoder,
valueDecoder,
config.enableShallowIterator)
config.enableShallowIterator,
config.clientId)
(queue, stream)
}).toList

View File

@ -961,7 +961,7 @@ case class PartitionAndReplica(topic: String, partition: Int, replica: Int)
case class LeaderIsrAndControllerEpoch(val leaderAndIsr: LeaderAndIsr, controllerEpoch: Int)
object ControllerStat extends KafkaMetricsGroup {
object ControllerStats extends KafkaMetricsGroup {
val offlinePartitionRate = newMeter("OfflinePartitionsPerSec", "partitions", TimeUnit.SECONDS)
val uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS)
val leaderElectionTimer = new KafkaTimer(newTimer("LeaderElectionRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))

View File

@ -58,12 +58,12 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten
.format(liveAssignedReplicasToThisPartition.mkString(",")))
liveAssignedReplicasToThisPartition.isEmpty match {
case true =>
ControllerStat.offlinePartitionRate.mark()
ControllerStats.offlinePartitionRate.mark()
throw new PartitionOfflineException(("No replica for partition " +
"([%s, %d]) is alive. Live brokers are: [%s],".format(topic, partition, controllerContext.liveBrokerIds)) +
" Assigned replicas are: [%s]".format(assignedReplicas))
case false =>
ControllerStat.uncleanLeaderElectionRate.mark()
ControllerStats.uncleanLeaderElectionRate.mark()
val newLeader = liveAssignedReplicasToThisPartition.head
warn("No broker in ISR is alive, elected leader from the alive replicas is [%s], ".format(newLeader) +
"There's potential data loss")
@ -78,7 +78,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten
partition))
(newLeaderAndIsr, liveAssignedReplicasToThisPartition)
case None =>
ControllerStat.offlinePartitionRate.mark()
ControllerStats.offlinePartitionRate.mark()
throw new PartitionOfflineException("Partition [%s, %d] doesn't have".format(topic, partition) +
"replicas assigned to it")
}

View File

@ -223,7 +223,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
val liveAssignedReplicas = replicaAssignment.filter(r => controllerContext.liveBrokerIds.contains(r))
liveAssignedReplicas.size match {
case 0 =>
ControllerStat.offlinePartitionRate.mark()
ControllerStats.offlinePartitionRate.mark()
throw new StateChangeFailedException(("During state change of partition %s from NEW to ONLINE, assigned replicas are " +
"[%s], live brokers are [%s]. No assigned replica is alive").format(topicAndPartition,
replicaAssignment.mkString(","), controllerContext.liveBrokerIds))
@ -249,7 +249,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
// read the controller epoch
val leaderIsrAndEpoch = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic,
topicAndPartition.partition).get
ControllerStat.offlinePartitionRate.mark()
ControllerStats.offlinePartitionRate.mark()
throw new StateChangeFailedException("Error while changing partition %s's state from New to Online"
.format(topicAndPartition) + " since Leader and isr path already exists with value " +
"%s and controller epoch %d".format(leaderIsrAndEpoch.leaderAndIsr.toString(), leaderIsrAndEpoch.controllerEpoch))

View File

@ -227,7 +227,7 @@ 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]) {
ControllerStat.leaderElectionTimer.time {
ControllerStats.leaderElectionTimer.time {
info("Broker change listener fired for path %s with children %s".format(parentPath, currentBrokerList.mkString(",")))
if(!isShuttingDown.get()) {
controllerContext.controllerLock synchronized {

View File

@ -20,14 +20,15 @@ import kafka.api._
import java.nio.ByteBuffer
import scala.collection.JavaConversions
class TopicMetadataRequest(val versionId: Short,
class TopicMetadataRequest(val correlationId: Int,
val versionId: Short,
val clientId: String,
val topics: java.util.List[String]) extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) {
val underlying: kafka.api.TopicMetadataRequest =
new kafka.api.TopicMetadataRequest(versionId, clientId, JavaConversions.asBuffer(topics))
new kafka.api.TopicMetadataRequest(versionId, clientId, JavaConversions.asBuffer(topics), correlationId)
def this(topics: java.util.List[String]) =
this(kafka.api.TopicMetadataRequest.CurrentVersion, kafka.api.TopicMetadataRequest.DefaultClientId, topics)
this(0, kafka.api.TopicMetadataRequest.CurrentVersion, kafka.api.TopicMetadataRequest.DefaultClientId, topics)
def writeTo(buffer: ByteBuffer) = underlying.writeTo(buffer)

View File

@ -29,8 +29,10 @@ import kafka.javaapi.OffsetRequest
class SimpleConsumer(val host: String,
val port: Int,
val soTimeout: Int,
val bufferSize: Int) {
private val underlying = new kafka.consumer.SimpleConsumer(host, port, soTimeout, bufferSize)
val bufferSize: Int,
val clientId: String) {
private val underlying = new kafka.consumer.SimpleConsumer(host, port, soTimeout, bufferSize, clientId)
/**
* Fetch a set of messages from a topic. This version of the fetch method

View File

@ -16,7 +16,6 @@
*/
package kafka.javaapi.consumer
import kafka.message.Message
import kafka.serializer._
import kafka.consumer._
import scala.collection.JavaConversions.asList

View File

@ -93,14 +93,23 @@ class Log(val dir: File,
val ls = dir.listFiles()
if(ls != null) {
for(file <- ls if file.isFile) {
if(!file.canRead)
throw new IOException("Could not read file " + file)
val filename = file.getName
if(filename.endsWith(DeletedFileSuffix)) {
// if the file ends in .deleted, delete it
val deleted = file.delete()
if(!deleted)
warn("Attempt to delete defunct segment file %s failed.".format(filename))
} else if(filename.endsWith(IndexFileSuffix)) {
// if it is an index file, make sure it has a corresponding .log file
val logFile = new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix))
if(!logFile.exists) {
warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath))
file.delete()
}
} else if(filename.endsWith(LogFileSuffix)) {
if(!file.canRead)
throw new IOException("Could not read file " + file)
// if its a log file, load the corresponding log segment
val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong
val hasIndex = Log.indexFilename(dir, start).exists
val segment = new LogSegment(dir = dir,

View File

@ -105,7 +105,7 @@ class LogManager(val config: KafkaConfig,
for(dir <- dirs) {
/* check if this set of logs was shut down cleanly */
val cleanShutDownFile = new File(dir, CleanShutdownFile)
val needsRecovery = cleanShutDownFile.exists
val needsRecovery = !cleanShutDownFile.exists
cleanShutDownFile.delete
/* load the logs */
val subDirs = dir.listFiles()

View File

@ -38,17 +38,20 @@ object Message {
val KeySizeOffset = AttributesOffset + AttributesLength
val KeySizeLength = 4
val KeyOffset = KeySizeOffset + KeySizeLength
val MessageOverhead = KeyOffset
val ValueSizeLength = 4
/** The amount of overhead bytes in a message */
val MessageOverhead = KeyOffset + ValueSizeLength
/**
* The minimum valid size for the message header
*/
val MinHeaderSize = CrcLength + MagicLength + AttributesLength + KeySizeLength
val MinHeaderSize = CrcLength + MagicLength + AttributesLength + KeySizeLength + ValueSizeLength
/**
* The current "magic" value
*/
val CurrentMagicValue: Byte = 2
val CurrentMagicValue: Byte = 0
/**
* Specifies the mask for the compression code. 2 bits to hold the compression codec.
@ -97,22 +100,24 @@ class Message(val buffer: ByteBuffer) {
Message.AttributesLength +
Message.KeySizeLength +
(if(key == null) 0 else key.length) +
Message.ValueSizeLength +
(if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset)))
// skip crc, we will fill that in at the end
buffer.put(MagicOffset, CurrentMagicValue)
var attributes:Byte = 0
buffer.position(MagicOffset)
buffer.put(CurrentMagicValue)
var attributes: Byte = 0
if (codec.codec > 0)
attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte
buffer.put(AttributesOffset, attributes)
buffer.put(attributes)
if(key == null) {
buffer.putInt(KeySizeOffset, -1)
buffer.position(KeyOffset)
buffer.putInt(-1)
} else {
buffer.putInt(KeySizeOffset, key.length)
buffer.position(KeyOffset)
buffer.putInt(key.length)
buffer.put(key, 0, key.length)
}
buffer.put(bytes, payloadOffset, if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset)
val size = if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset
buffer.putInt(size)
buffer.put(bytes, payloadOffset, size)
buffer.rewind()
// now compute the checksum and fill it in
@ -170,10 +175,15 @@ class Message(val buffer: ByteBuffer) {
*/
def hasKey: Boolean = keySize >= 0
/**
* The position where the payload size is stored
*/
private def payloadSizeOffset = Message.KeyOffset + max(0, keySize)
/**
* The length of the message value in bytes
*/
def payloadSize: Int = size - KeyOffset - max(0, keySize)
def payloadSize: Int = buffer.getInt(payloadSizeOffset)
/**
* The magic version of this message
@ -194,29 +204,27 @@ class Message(val buffer: ByteBuffer) {
/**
* A ByteBuffer containing the content of the message
*/
def payload: ByteBuffer = {
var payload = buffer.duplicate
payload.position(KeyOffset + max(keySize, 0))
payload = payload.slice()
payload.limit(payloadSize)
payload.rewind()
payload
}
def payload: ByteBuffer = sliceDelimited(payloadSizeOffset)
/**
* A ByteBuffer containing the message key
*/
def key: ByteBuffer = {
val s = keySize
if(s < 0) {
def key: ByteBuffer = sliceDelimited(KeySizeOffset)
/**
* Read a size-delimited byte buffer starting at the given offset
*/
private def sliceDelimited(start: Int): ByteBuffer = {
val size = buffer.getInt(start)
if(size < 0) {
null
} else {
var key = buffer.duplicate
key.position(KeyOffset)
key = key.slice()
key.limit(s)
key.rewind()
key
var b = buffer.duplicate
b.position(start + 4)
b = b.slice()
b.limit(size)
b.rewind
b
}
}

View File

@ -24,7 +24,6 @@ import com.yammer.metrics.Metrics
import java.io.File
import com.yammer.metrics.reporting.CsvReporter
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean
import kafka.utils.{Utils, VerifiableProperties, Logging}

View File

@ -13,7 +13,7 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
*/
package kafka.producer
import collection.mutable.HashMap
@ -68,11 +68,11 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig,
/**
* It updates the cache by issuing a get topic metadata request to a random broker.
* @param topic the topic for which the metadata is to be fetched
* @param topics the topics for which the metadata is to be fetched
*/
def updateInfo(topics: Set[String]) = {
def updateInfo(topics: Set[String]) {
var topicsMetadata: Seq[TopicMetadata] = Nil
val topicMetadataResponse = ClientUtils.fetchTopicMetadata(topics, brokers)
val topicMetadataResponse = ClientUtils.fetchTopicMetadata(topics, brokers, producerConfig)
topicsMetadata = topicMetadataResponse.topicsMetadata
// throw partition specific exception
topicsMetadata.foreach(tmd =>{

View File

@ -21,6 +21,7 @@ import scala.collection.JavaConversions._
import joptsimple._
import java.util.Properties
import java.io._
import kafka.common._
import kafka.message._
import kafka.serializer._
@ -49,13 +50,18 @@ object ConsoleProducer {
.describedAs("timeout_ms")
.ofType(classOf[java.lang.Long])
.defaultsTo(1000)
val messageEncoderOpt = parser.accepts("message-encoder", "The class name of the message encoder implementation to use.")
val valueEncoderOpt = parser.accepts("value-serializer", "The class name of the message encoder implementation to use for serializing values.")
.withRequiredArg
.describedAs("encoder_class")
.ofType(classOf[java.lang.String])
.defaultsTo(classOf[StringEncoder].getName)
val keyEncoderOpt = parser.accepts("key-serializer", "The class name of the message encoder implementation to use for serializing keys.")
.withRequiredArg
.describedAs("encoder_class")
.ofType(classOf[java.lang.String])
.defaultsTo(classOf[StringEncoder].getName)
val messageReaderOpt = parser.accepts("line-reader", "The class name of the class to use for reading lines from standard in. " +
"By default each line is read as a seperate message.")
"By default each line is read as a separate message.")
.withRequiredArg
.describedAs("reader_class")
.ofType(classOf[java.lang.String])
@ -82,9 +88,11 @@ object ConsoleProducer {
val compress = options.has(compressOpt)
val batchSize = options.valueOf(batchSizeOpt)
val sendTimeout = options.valueOf(sendTimeoutOpt)
val encoderClass = options.valueOf(messageEncoderOpt)
val keyEncoderClass = options.valueOf(keyEncoderOpt)
val valueEncoderClass = options.valueOf(valueEncoderOpt)
val readerClass = options.valueOf(messageReaderOpt)
val cmdLineProps = parseLineReaderArgs(options.valuesOf(propertyOpt))
cmdLineProps.put("topic", topic)
val props = new Properties()
props.put("broker.list", brokerList)
@ -94,12 +102,13 @@ object ConsoleProducer {
if(options.has(batchSizeOpt))
props.put("batch.size", batchSize.toString)
props.put("queue.time", sendTimeout.toString)
props.put("serializer.class", encoderClass)
props.put("key.serializer.class", keyEncoderClass)
props.put("serializer.class", valueEncoderClass)
val reader = Class.forName(readerClass).newInstance().asInstanceOf[MessageReader]
val reader = Class.forName(readerClass).newInstance().asInstanceOf[MessageReader[AnyRef, AnyRef]]
reader.init(System.in, cmdLineProps)
val producer = new Producer[Any, Any](new ProducerConfig(props))
val producer = new Producer[AnyRef, AnyRef](new ProducerConfig(props))
Runtime.getRuntime.addShutdownHook(new Thread() {
override def run() {
@ -107,11 +116,11 @@ object ConsoleProducer {
}
})
var message: AnyRef = null
var message: KeyedMessage[AnyRef, AnyRef] = null
do {
message = reader.readMessage()
if(message != null)
producer.send(new KeyedMessage(topic, message))
producer.send(message)
} while(message != null)
}
@ -127,19 +136,49 @@ object ConsoleProducer {
props
}
trait MessageReader {
trait MessageReader[K,V] {
def init(inputStream: InputStream, props: Properties) {}
def readMessage(): AnyRef
def readMessage(): KeyedMessage[K,V]
def close() {}
}
class LineMessageReader extends MessageReader {
class LineMessageReader extends MessageReader[String, String] {
var topic: String = null
var reader: BufferedReader = null
var parseKey = false
var keySeparator = "\t"
var ignoreError = false
var lineNumber = 0
override def init(inputStream: InputStream, props: Properties) {
topic = props.getProperty("topic")
if(props.containsKey("parse.key"))
parseKey = props.getProperty("parse.key").trim.toLowerCase.equals("true")
if(props.containsKey("key.seperator"))
keySeparator = props.getProperty("key.separator")
if(props.containsKey("ignore.error"))
ignoreError = props.getProperty("ignore.error").trim.toLowerCase.equals("true")
reader = new BufferedReader(new InputStreamReader(inputStream))
}
override def readMessage() = reader.readLine()
override def readMessage() = {
lineNumber += 1
val line = reader.readLine()
if(parseKey) {
line.indexOf(keySeparator) match {
case -1 =>
if(ignoreError)
new KeyedMessage(topic, line)
else
throw new KafkaException("No key found on line " + lineNumber + ": " + line)
case n =>
new KeyedMessage(topic,
line.substring(0, n),
if(n + keySeparator.size > line.size) "" else line.substring(n + keySeparator.size))
}
} else {
new KeyedMessage(topic, line)
}
}
}
}

View File

@ -17,7 +17,6 @@
package kafka.producer
import kafka.utils.Utils
import kafka.utils._

View File

@ -16,23 +16,21 @@
*/
package kafka.producer
import async.{AsyncProducerStats, DefaultEventHandler, ProducerSendThread, EventHandler}
import async.{DefaultEventHandler, ProducerSendThread, EventHandler}
import kafka.utils._
import java.util.Random
import java.util.concurrent.{TimeUnit, LinkedBlockingQueue}
import kafka.serializer.Encoder
import java.util.concurrent.atomic.AtomicBoolean
import kafka.common.{QueueFullException, InvalidConfigException}
import kafka.common.QueueFullException
import kafka.metrics._
class Producer[K,V](config: ProducerConfig,
private val eventHandler: EventHandler[K,V]) // for testing only
extends Logging {
private val hasShutdown = new AtomicBoolean(false)
if (config.batchSize > config.queueSize)
throw new InvalidConfigException("Batch size can't be larger than queue size.")
private val eventHandler: EventHandler[K,V]) // only for unit testing
extends Logging {
private val hasShutdown = new AtomicBoolean(false)
private val queue = new LinkedBlockingQueue[KeyedMessage[K,V]](config.queueSize)
private val random = new Random
@ -47,11 +45,14 @@ extends Logging {
queue,
eventHandler,
config.queueTime,
config.batchSize)
config.batchSize,
config.clientId)
producerSendThread.start()
case _ => throw new InvalidConfigException("Valid values for producer.type are sync/async")
}
private val producerStats = ProducerStatsRegistry.getProducerStats(config.clientId)
private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(config.clientId)
KafkaMetricsReporter.startReporters(config.props)
def this(config: ProducerConfig) =
@ -65,7 +66,7 @@ extends Logging {
/**
* Sends the data, partitioned by key to the topic using either the
* synchronous or the asynchronous producer
* @param producerData the producer data object that encapsulates the topic, key and message data
* @param messages the producer data object that encapsulates the topic, key and message data
*/
def send(messages: KeyedMessage[K,V]*) {
if (hasShutdown.get)
@ -79,8 +80,8 @@ extends Logging {
private def recordStats(messages: Seq[KeyedMessage[K,V]]) {
for (message <- messages) {
ProducerTopicStat.getProducerTopicStat(message.topic).messageRate.mark()
ProducerTopicStat.getProducerAllTopicStat.messageRate.mark()
producerTopicStats.getProducerTopicStats(message.topic).messageRate.mark()
producerTopicStats.getProducerAllTopicStats.messageRate.mark()
}
}
@ -105,7 +106,7 @@ extends Logging {
}
}
if(!added) {
AsyncProducerStats.droppedMessageRate.mark()
producerStats.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 {
@ -130,27 +131,4 @@ extends Logging {
}
}
@threadsafe
class ProducerTopicStat(name: String) extends KafkaMetricsGroup {
val messageRate = newMeter(name + "MessagesPerSec", "messages", TimeUnit.SECONDS)
val byteRate = newMeter(name + "BytesPerSec", "bytes", TimeUnit.SECONDS)
}
object ProducerTopicStat {
private val valueFactory = (k: String) => new ProducerTopicStat(k)
private val stats = new Pool[String, ProducerTopicStat](Some(valueFactory))
private val allTopicStat = new ProducerTopicStat("AllTopics")
def getProducerAllTopicStat(): ProducerTopicStat = allTopicStat
def getProducerTopicStat(topic: String): ProducerTopicStat = {
stats.getAndMaybePut(topic + "-")
}
}
object ProducerStats extends KafkaMetricsGroup {
val serializationErrorRate = newMeter("SerializationErrorsPerSec", "errors", TimeUnit.SECONDS)
val resendRate = newMeter( "ResendsPerSec", "resends", TimeUnit.SECONDS)
val failedSendRate = newMeter("FailedSendsPerSec", "failed sends", TimeUnit.SECONDS)
}

View File

@ -21,9 +21,36 @@ import async.AsyncProducerConfig
import java.util.Properties
import kafka.utils.{Utils, VerifiableProperties}
import kafka.message.{CompressionCodec, NoCompressionCodec}
import kafka.common.{InvalidConfigException, Config}
object ProducerConfig extends Config {
def validate(config: ProducerConfig) {
validateClientId(config.clientId)
validateBatchSize(config.batchSize, config.queueSize)
validateProducerType(config.producerType)
}
def validateClientId(clientId: String) {
validateChars("clientid", clientId)
}
def validateBatchSize(batchSize: Int, queueSize: Int) {
if (batchSize > queueSize)
throw new InvalidConfigException("Batch size = " + batchSize + " can't be larger than queue size = " + queueSize)
}
def validateProducerType(producerType: String) {
producerType match {
case "sync" =>
case "async"=>
case _ => throw new InvalidConfigException("Invalid value " + producerType + " for producer.type, valid values are sync/async")
}
}
}
class ProducerConfig private (val props: VerifiableProperties)
extends AsyncProducerConfig with SyncProducerConfigShared {
import ProducerConfig._
def this(originalProps: Properties) {
this(new VerifiableProperties(originalProps))
@ -33,7 +60,7 @@ class ProducerConfig private (val props: VerifiableProperties)
/** This is for bootstrapping and the producer will only use it for getting metadata
* (topics, partitions and replicas). The socket connections for sending the actual data
* will be established based on the broker information returned in the metadata. The
* format is host1:por1,host2:port2, and the list can be a subset of brokers or
* format is host1:port1,host2:port2, and the list can be a subset of brokers or
* a VIP pointing to a subset of brokers.
*/
val brokerList = props.getString("broker.list")
@ -85,4 +112,6 @@ class ProducerConfig private (val props: VerifiableProperties)
* The amount of time to wait in between retries
*/
val producerRetryBackoffMs = props.getInt("producer.retry.backoff.ms", 100)
validate(this)
}

View File

@ -26,13 +26,15 @@ import kafka.api.TopicMetadata
import kafka.common.UnavailableProducerException
object ProducerPool{
def createSyncProducer(configOpt: Option[ProducerConfig], broker: Broker): SyncProducer = {
object ProducerPool {
/**
* Used in ProducerPool to initiate a SyncProducer connection with a broker.
*/
def createSyncProducer(config: ProducerConfig, broker: Broker): SyncProducer = {
val props = new Properties()
props.put("host", broker.host)
props.put("port", broker.port.toString)
if(configOpt.isDefined)
props.putAll(configOpt.get.props.props)
props.putAll(config.props.props)
new SyncProducer(new SyncProducerConfig(props))
}
}
@ -41,9 +43,9 @@ class ProducerPool(val config: ProducerConfig) extends Logging {
private val syncProducers = new HashMap[Int, SyncProducer]
private val lock = new Object()
def updateProducer(topicMetaDatas: Seq[TopicMetadata]) {
def updateProducer(topicMetadatas: Seq[TopicMetadata]) {
val newBrokers = new collection.mutable.HashSet[Broker]
topicMetaDatas.foreach(tmd => {
topicMetadatas.foreach(tmd => {
tmd.partitionsMetadata.foreach(pmd => {
if(pmd.leader.isDefined)
newBrokers+=(pmd.leader.get)
@ -53,9 +55,9 @@ class ProducerPool(val config: ProducerConfig) extends Logging {
newBrokers.foreach(b => {
if(syncProducers.contains(b.id)){
syncProducers(b.id).close()
syncProducers.put(b.id, ProducerPool.createSyncProducer(Some(config), b))
syncProducers.put(b.id, ProducerPool.createSyncProducer(config, b))
} else
syncProducers.put(b.id, ProducerPool.createSyncProducer(Some(config), b))
syncProducers.put(b.id, ProducerPool.createSyncProducer(config, b))
})
}
}

View File

@ -0,0 +1,56 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.producer
import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
import java.util.concurrent.TimeUnit
import kafka.utils.Pool
import kafka.common.ClientIdAndBroker
class ProducerRequestMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup {
val requestTimer = new KafkaTimer(newTimer(metricId + "-ProducerRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
val requestSizeHist = newHistogram(metricId + "-ProducerRequestSize")
}
/**
* Tracks metrics of requests made by a given producer client to all brokers.
* @param clientId ClientId of the given producer
*/
class ProducerRequestStats(clientId: String) {
private val valueFactory = (k: ClientIdAndBroker) => new ProducerRequestMetrics(k)
private val stats = new Pool[ClientIdAndBroker, ProducerRequestMetrics](Some(valueFactory))
private val allBrokersStats = new ProducerRequestMetrics(new ClientIdAndBroker(clientId, "All.Brokers"))
def getProducerRequestAllBrokersStats(): ProducerRequestMetrics = allBrokersStats
def getProducerRequestStats(brokerInfo: String): ProducerRequestMetrics = {
stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo))
}
}
/**
* Stores the request stats information of each producer client in a (clientId -> ProducerRequestStats) map.
*/
object ProducerRequestStatsRegistry {
private val valueFactory = (k: String) => new ProducerRequestStats(k)
private val globalStats = new Pool[String, ProducerRequestStats](Some(valueFactory))
def getProducerRequestStats(clientId: String) = {
globalStats.getAndMaybePut(clientId)
}
}

View File

@ -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.producer
import kafka.metrics.KafkaMetricsGroup
import java.util.concurrent.TimeUnit
import kafka.utils.Pool
class ProducerStats(clientId: String) extends KafkaMetricsGroup {
val serializationErrorRate = newMeter(clientId + "-SerializationErrorsPerSec", "errors", TimeUnit.SECONDS)
val resendRate = newMeter(clientId + "-ResendsPerSec", "resends", TimeUnit.SECONDS)
val failedSendRate = newMeter(clientId + "-FailedSendsPerSec", "failed sends", TimeUnit.SECONDS)
val droppedMessageRate = newMeter(clientId + "-DroppedMessagesPerSec", "drops", TimeUnit.SECONDS)
}
/**
* Stores metrics of serialization and message sending activity of each producer client in a (clientId -> ProducerStats) map.
*/
object ProducerStatsRegistry {
private val valueFactory = (k: String) => new ProducerStats(k)
private val statsRegistry = new Pool[String, ProducerStats](Some(valueFactory))
def getProducerStats(clientId: String) = {
statsRegistry.getAndMaybePut(clientId)
}
}

View File

@ -0,0 +1,57 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.producer
import kafka.metrics.KafkaMetricsGroup
import kafka.common.ClientIdAndTopic
import kafka.utils.{Pool, threadsafe}
import java.util.concurrent.TimeUnit
@threadsafe
class ProducerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup {
val messageRate = newMeter(metricId + "-MessagesPerSec", "messages", TimeUnit.SECONDS)
val byteRate = newMeter(metricId + "-BytesPerSec", "bytes", TimeUnit.SECONDS)
}
/**
* Tracks metrics for each topic the given producer client has produced data to.
* @param clientId The clientId of the given producer client.
*/
class ProducerTopicStats(clientId: String) {
private val valueFactory = (k: ClientIdAndTopic) => new ProducerTopicMetrics(k)
private val stats = new Pool[ClientIdAndTopic, ProducerTopicMetrics](Some(valueFactory))
private val allTopicStats = new ProducerTopicMetrics(new ClientIdAndTopic(clientId, "All.Topics")) // to differentiate from a topic named AllTopics
def getProducerAllTopicStats(): ProducerTopicMetrics = allTopicStats
def getProducerTopicStats(topic: String): ProducerTopicMetrics = {
stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic))
}
}
/**
* Stores the topic stats information of each producer client in a (clientId -> ProducerTopicStats) map.
*/
object ProducerTopicStatsRegistry {
private val valueFactory = (k: String) => new ProducerTopicStats(k)
private val globalStats = new Pool[String, ProducerTopicStats](Some(valueFactory))
def getProducerTopicStats(clientId: String) = {
globalStats.getAndMaybePut(clientId)
}
}

View File

@ -21,8 +21,6 @@ import kafka.api._
import kafka.network.{BlockingChannel, BoundedByteBufferSend, Receive}
import kafka.utils._
import java.util.Random
import java.util.concurrent.TimeUnit
import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
object SyncProducer {
val RequestKey: Short = 0
@ -35,13 +33,12 @@ object SyncProducer {
@threadsafe
class SyncProducer(val config: SyncProducerConfig) extends Logging {
private val MaxConnectBackoffMs = 60000
private var sentOnConnection = 0
private val lock = new Object()
@volatile private var shutdown: Boolean = false
private val blockingChannel = new BlockingChannel(config.host, config.port, BlockingChannel.UseDefaultBufferSize,
config.bufferSize, config.requestTimeoutMs)
val brokerInfo = "host_%s-port_%s".format(config.host, config.port)
val producerRequestStats = ProducerRequestStatsRegistry.getProducerRequestStats(config.clientId)
trace("Instantiating Scala Sync Producer")
@ -89,10 +86,17 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging {
* Send a message
*/
def send(producerRequest: ProducerRequest): ProducerResponse = {
ProducerRequestStat.requestSizeHist.update(producerRequest.sizeInBytes)
val requestSize = producerRequest.sizeInBytes
producerRequestStats.getProducerRequestStats(brokerInfo).requestSizeHist.update(requestSize)
producerRequestStats.getProducerRequestAllBrokersStats.requestSizeHist.update(requestSize)
var response: Receive = null
ProducerRequestStat.requestTimer.time {
response = doSend(producerRequest)
val specificTimer = producerRequestStats.getProducerRequestStats(brokerInfo).requestTimer
val aggregateTimer = producerRequestStats.getProducerRequestAllBrokersStats.requestTimer
aggregateTimer.time {
specificTimer.time {
response = doSend(producerRequest)
}
}
ProducerResponse.readFrom(response.buffer)
}
@ -152,7 +156,3 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging {
}
}
object ProducerRequestStat extends KafkaMetricsGroup {
val requestTimer = new KafkaTimer(newTimer("ProduceRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
val requestSizeHist = newHistogram("ProducerRequestSize")
}

View File

@ -41,10 +41,7 @@ trait SyncProducerConfigShared {
val maxMessageSize = props.getInt("max.message.size", 1000000)
/* the client application sending the producer requests */
val correlationId = props.getInt("producer.request.correlation_id", SyncProducerConfig.DefaultCorrelationId)
/* the client application sending the producer requests */
val clientId = props.getString("producer.request.client_id",SyncProducerConfig.DefaultClientId)
val clientId = props.getString("clientid", SyncProducerConfig.DefaultClientId)
/*
* The required acks of the producer requests - negative value means ack
@ -61,8 +58,7 @@ trait SyncProducerConfigShared {
}
object SyncProducerConfig {
val DefaultCorrelationId = -1
val DefaultClientId = ""
val DefaultRequiredAcks : Short = 0
val DefaultAckTimeoutMs = 500
val DefaultAckTimeoutMs = 1500
}

View File

@ -1,25 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.producer.async
import kafka.metrics.KafkaMetricsGroup
import java.util.concurrent.TimeUnit
object AsyncProducerStats extends KafkaMetricsGroup {
val droppedMessageRate = newMeter("DroppedMessagesPerSec", "drops", TimeUnit.SECONDS)
}

View File

@ -37,19 +37,23 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
extends EventHandler[K,V] with Logging {
val isSync = ("sync" == config.producerType)
val counter = new AtomicInteger(0)
val partitionCounter = new AtomicInteger(0)
val correlationCounter = new AtomicInteger(0)
val brokerPartitionInfo = new BrokerPartitionInfo(config, producerPool, topicPartitionInfos)
private val lock = new Object()
private val producerStats = ProducerStatsRegistry.getProducerStats(config.clientId)
private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(config.clientId)
def handle(events: Seq[KeyedMessage[K,V]]) {
lock synchronized {
val serializedData = serialize(events)
serializedData.foreach{
keyed =>
val dataSize = keyed.message.payloadSize
ProducerTopicStat.getProducerTopicStat(keyed.topic).byteRate.mark(dataSize)
ProducerTopicStat.getProducerAllTopicStat.byteRate.mark(dataSize)
producerTopicStats.getProducerTopicStats(keyed.topic).byteRate.mark(dataSize)
producerTopicStats.getProducerAllTopicStats.byteRate.mark(dataSize)
}
var outstandingProduceRequests = serializedData
var remainingRetries = config.producerRetries + 1
@ -61,11 +65,11 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
// get topics of the outstanding produce requests and refresh metadata for those
Utils.swallowError(brokerPartitionInfo.updateInfo(outstandingProduceRequests.map(_.topic).toSet))
remainingRetries -= 1
ProducerStats.resendRate.mark()
producerStats.resendRate.mark()
}
}
if(outstandingProduceRequests.size > 0) {
ProducerStats.failedSendRate.mark()
producerStats.failedSendRate.mark()
error("Failed to send the following requests: " + outstandingProduceRequests)
throw new FailedToSendMessageException("Failed to send messages after " + config.producerRetries + " tries.", null)
}
@ -111,7 +115,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
serializedMessages += KeyedMessage[K,Message](topic = e.topic, key = null.asInstanceOf[K], message = new Message(bytes = encoder.toBytes(e.message)))
} catch {
case t =>
ProducerStats.serializationErrorRate.mark()
producerStats.serializationErrorRate.mark()
if (isSync) {
throw t
} else {
@ -189,7 +193,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
"\n Valid values are > 0")
val partition =
if(key == null)
Utils.abs(counter.getAndIncrement()) % numPartitions
Utils.abs(partitionCounter.getAndIncrement()) % numPartitions
else
partitioner.partition(key, numPartitions)
if(partition < 0 || partition >= numPartitions)
@ -210,20 +214,21 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
warn("Failed to send to broker %d with data %s".format(brokerId, messagesPerTopic))
messagesPerTopic.keys.toSeq
} else if(messagesPerTopic.size > 0) {
val topicPartitionDataPairs = messagesPerTopic.toSeq.map {
case (topicAndPartition, messages) =>
(topicAndPartition, messages)
}
val producerRequest = new ProducerRequest(config.correlationId, config.clientId, config.requiredAcks,
config.requestTimeoutMs, Map(topicPartitionDataPairs:_*))
val producerRequest = new ProducerRequest(correlationCounter.getAndIncrement(), config.clientId, config.requiredAcks,
config.requestTimeoutMs, messagesPerTopic)
try {
val syncProducer = producerPool.getProducer(brokerId)
val response = syncProducer.send(producerRequest)
trace("Producer sent messages for topics %s to broker %d on %s:%d"
debug("Producer sent messages for topics %s to broker %d on %s:%d"
.format(messagesPerTopic, brokerId, syncProducer.config.host, syncProducer.config.port))
if (response.status.size != producerRequest.data.size)
throw new KafkaException("Incomplete response (%s) for producer request (%s)"
.format(response, producerRequest))
.format(response, producerRequest))
if (logger.isTraceEnabled) {
val successfullySentData = response.status.filter(_._2.error == ErrorMapping.NoError)
successfullySentData.foreach(m => messagesPerTopic(m._1).foreach(message =>
trace("Successfully sent message: %s".format(Utils.readString(message.message.payload)))))
}
response.status.filter(_._2.error != ErrorMapping.NoError).toSeq
.map(partitionStatus => partitionStatus._1)
} catch {
@ -238,33 +243,33 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
private def groupMessagesToSet(messagesPerTopicAndPartition: Map[TopicAndPartition, Seq[KeyedMessage[K,Message]]]) = {
/** enforce the compressed.topics config here.
* If the compression codec is anything other than NoCompressionCodec,
* Enable compression only for specified topics if any
* If the list of compressed topics is empty, then enable the specified compression codec for all topics
* If the compression codec is NoCompressionCodec, compression is disabled for all topics
*/
* If the compression codec is anything other than NoCompressionCodec,
* Enable compression only for specified topics if any
* If the list of compressed topics is empty, then enable the specified compression codec for all topics
* If the compression codec is NoCompressionCodec, compression is disabled for all topics
*/
val messagesPerTopicPartition = messagesPerTopicAndPartition.map { case (topicAndPartition, messages) =>
val rawMessages = messages.map(_.message)
( topicAndPartition,
config.compressionCodec match {
case NoCompressionCodec =>
trace("Sending %d messages with no compression to %s".format(messages.size, topicAndPartition))
debug("Sending %d messages with no compression to %s".format(messages.size, topicAndPartition))
new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*)
case _ =>
config.compressedTopics.size match {
case 0 =>
trace("Sending %d messages with compression codec %d to %s"
debug("Sending %d messages with compression codec %d to %s"
.format(messages.size, config.compressionCodec.codec, topicAndPartition))
new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*)
case _ =>
if(config.compressedTopics.contains(topicAndPartition.topic)) {
trace("Sending %d messages with compression codec %d to %s"
debug("Sending %d messages with compression codec %d to %s"
.format(messages.size, config.compressionCodec.codec, topicAndPartition))
new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*)
}
else {
trace("Sending %d messages to %s with no compression as it is not in compressed.topics - %s"
debug("Sending %d messages to %s with no compression as it is not in compressed.topics - %s"
.format(messages.size, topicAndPartition, config.compressedTopics.toString))
new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*)
}

View File

@ -19,7 +19,7 @@ package kafka.producer.async
import kafka.utils.{SystemTime, Logging}
import java.util.concurrent.{TimeUnit, CountDownLatch, BlockingQueue}
import collection.mutable.ListBuffer
import collection.mutable.ArrayBuffer
import kafka.producer.KeyedMessage
import kafka.metrics.KafkaMetricsGroup
import com.yammer.metrics.core.Gauge
@ -28,12 +28,13 @@ class ProducerSendThread[K,V](val threadName: String,
val queue: BlockingQueue[KeyedMessage[K,V]],
val handler: EventHandler[K,V],
val queueTime: Long,
val batchSize: Int) extends Thread(threadName) with Logging with KafkaMetricsGroup {
val batchSize: Int,
val clientId: String) extends Thread(threadName) with Logging with KafkaMetricsGroup {
private val shutdownLatch = new CountDownLatch(1)
private val shutdownCommand = new KeyedMessage[K,V]("shutdown", null.asInstanceOf[K], null.asInstanceOf[V])
newGauge("ProducerQueueSize-" + getId,
newGauge(clientId + "-ProducerQueueSize-" + getId,
new Gauge[Int] {
def getValue = queue.size
})
@ -57,7 +58,7 @@ class ProducerSendThread[K,V](val threadName: String,
private def processEvents() {
var lastSend = SystemTime.milliseconds
var events = new ListBuffer[KeyedMessage[K,V]]
var events = new ArrayBuffer[KeyedMessage[K,V]]
var full: Boolean = false
// drain the queue until you get a shutdown command
@ -85,7 +86,7 @@ class ProducerSendThread[K,V](val threadName: String,
// if either queue time has reached or batch size has reached, dispatch to event handler
tryToHandle(events)
lastSend = SystemTime.milliseconds
events = new ListBuffer[KeyedMessage[K,V]]
events = new ArrayBuffer[KeyedMessage[K,V]]
}
}
// send the last batch of events

View File

@ -17,7 +17,6 @@
package kafka.serializer
import kafka.message._
import kafka.utils.VerifiableProperties
/**
@ -36,13 +35,6 @@ class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[B
def fromBytes(bytes: Array[Byte]): Array[Byte] = bytes
}
/**
* Decode messages without any key
*/
class KeylessMessageDecoder(props: VerifiableProperties = null) extends Decoder[Message] {
def fromBytes(bytes: Array[Byte]) = new Message(bytes)
}
/**
* The string encoder translates strings into bytes. It uses UTF8 by default but takes
* an optional property serializer.encoding to control this.

View File

@ -18,8 +18,6 @@
package kafka.serializer
import kafka.utils.VerifiableProperties
import kafka.message._
import kafka.utils.Utils
/**
* An encoder is a method of turning objects into byte arrays.

View File

@ -19,7 +19,7 @@ package kafka.server
import kafka.cluster.Broker
import kafka.consumer.SimpleConsumer
import kafka.common.{TopicAndPartition, ErrorMapping}
import kafka.common.{ClientIdAndBroker, TopicAndPartition, ErrorMapping}
import collection.mutable
import kafka.message.ByteBufferMessageSet
import kafka.message.MessageAndOffset
@ -38,12 +38,15 @@ import java.util.concurrent.locks.ReentrantLock
abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int,
fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1)
extends ShutdownableThread(name) {
private val partitionMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset map
private val partitionMapLock = new ReentrantLock
private val partitionMapCond = partitionMapLock.newCondition()
val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize)
val fetcherMetrics = FetcherStat.getFetcherStat(name + "-" + sourceBroker.id)
val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, clientId)
private val brokerInfo = "host_%s-port_%s".format(sourceBroker.host, sourceBroker.port)
private val metricId = new ClientIdAndBroker(clientId, brokerInfo)
val fetcherStats = new FetcherStats(metricId)
val fetcherMetrics = fetcherStats.getFetcherStats(name + "-" + sourceBroker.id)
val fetcherLagStats = new FetcherLagStats(metricId)
/* callbacks to be defined in subclass */
@ -117,7 +120,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke
case None => currentOffset.get
}
partitionMap.put(topicAndPartition, newOffset)
FetcherLagMetrics.getFetcherLagMetrics(topic, partitionId).lag = partitionData.hw - newOffset
fetcherLagStats.getFetcherLagStats(topic, partitionId).lag = partitionData.hw - newOffset
fetcherMetrics.byteRate.mark(validBytes)
// Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
processPartitionData(topicAndPartition, currentOffset.get, partitionData)
@ -182,10 +185,10 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke
}
}
class FetcherLagMetrics(name: (String, Int)) extends KafkaMetricsGroup {
class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMetricsGroup {
private[this] var lagVal = new AtomicLong(-1L)
newGauge(
name._1 + "-" + name._2 + "-ConsumerLag",
metricId + "-ConsumerLag",
new Gauge[Long] {
def getValue = lagVal.get
}
@ -198,25 +201,34 @@ class FetcherLagMetrics(name: (String, Int)) extends KafkaMetricsGroup {
def lag = lagVal.get
}
object FetcherLagMetrics {
private val valueFactory = (k: (String, Int)) => new FetcherLagMetrics(k)
private val stats = new Pool[(String, Int), FetcherLagMetrics](Some(valueFactory))
class FetcherLagStats(metricId: ClientIdAndBroker) {
private val valueFactory = (k: ClientIdBrokerTopicPartition) => new FetcherLagMetrics(k)
private val stats = new Pool[ClientIdBrokerTopicPartition, FetcherLagMetrics](Some(valueFactory))
def getFetcherLagMetrics(topic: String, partitionId: Int): FetcherLagMetrics = {
stats.getAndMaybePut( (topic, partitionId) )
def getFetcherLagStats(topic: String, partitionId: Int): FetcherLagMetrics = {
stats.getAndMaybePut(new ClientIdBrokerTopicPartition(metricId.clientId, metricId.brokerInfo, topic, partitionId))
}
}
class FetcherStat(name: String) extends KafkaMetricsGroup {
val requestRate = newMeter(name + "RequestsPerSec", "requests", TimeUnit.SECONDS)
val byteRate = newMeter(name + "BytesPerSec", "bytes", TimeUnit.SECONDS)
class FetcherMetrics(metricId: ClientIdBrokerTopic) extends KafkaMetricsGroup {
val requestRate = newMeter(metricId + "-RequestsPerSec", "requests", TimeUnit.SECONDS)
val byteRate = newMeter(metricId + "-BytesPerSec", "bytes", TimeUnit.SECONDS)
}
object FetcherStat {
private val valueFactory = (k: String) => new FetcherStat(k)
private val stats = new Pool[String, FetcherStat](Some(valueFactory))
class FetcherStats(metricId: ClientIdAndBroker) {
private val valueFactory = (k: ClientIdBrokerTopic) => new FetcherMetrics(k)
private val stats = new Pool[ClientIdBrokerTopic, FetcherMetrics](Some(valueFactory))
def getFetcherStat(name: String): FetcherStat = {
stats.getAndMaybePut(name)
def getFetcherStats(name: String): FetcherMetrics = {
stats.getAndMaybePut(new ClientIdBrokerTopic(metricId.clientId, metricId.brokerInfo, name))
}
}
case class ClientIdBrokerTopic(clientId: String, brokerInfo: String, topic: String) {
override def toString = "%s-%s-%s".format(clientId, brokerInfo, topic)
}
case class ClientIdBrokerTopicPartition(clientId: String, brokerInfo: String, topic: String, partitionId: Int) {
override def toString = "%s-%s-%s-%d".format(clientId, brokerInfo, topic, partitionId)
}

View File

@ -30,7 +30,7 @@ import java.io._
*/
object HighwaterMarkCheckpoint {
val highWatermarkFileName = ".highwatermark"
val highWatermarkFileName = "replication-offset-checkpoint"
val currentHighwaterMarkFileVersion = 0
}

View File

@ -41,8 +41,10 @@ class KafkaApis(val requestChannel: RequestChannel,
val zkClient: ZkClient,
brokerId: Int) extends Logging {
private val producerRequestPurgatory = new ProducerRequestPurgatory
private val fetchRequestPurgatory = new FetchRequestPurgatory(requestChannel)
private val producerRequestPurgatory =
new ProducerRequestPurgatory(replicaManager.config.producerRequestPurgatoryPurgeInterval)
private val fetchRequestPurgatory =
new FetchRequestPurgatory(requestChannel, replicaManager.config.fetchRequestPurgatoryPurgeInterval)
private val delayedRequestMetrics = new DelayedRequestMetrics
private val requestLogger = Logger.getLogger("kafka.request.logger")
@ -71,16 +73,16 @@ class KafkaApis(val requestChannel: RequestChannel,
case (topicAndPartition, data) =>
(topicAndPartition, ProducerResponseStatus(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1l))
}
val errorResponse = ProducerResponse(apiRequest.versionId, apiRequest.correlationId, producerResponseStatus)
val errorResponse = ProducerResponse(apiRequest.correlationId, producerResponseStatus)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
error("error when handling request %s".format(apiRequest), e)
case RequestKeys.FetchKey =>
val apiRequest = request.requestObj.asInstanceOf[FetchRequest]
val fetchResponsePartitionData = apiRequest.requestInfo.map {
case (topicAndPartition, data) =>
(topicAndPartition, FetchResponsePartitionData(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), 0, -1, null))
(topicAndPartition, FetchResponsePartitionData(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1, null))
}
val errorResponse = FetchResponse(apiRequest.versionId, apiRequest.correlationId, fetchResponsePartitionData)
val errorResponse = FetchResponse(apiRequest.correlationId, fetchResponsePartitionData)
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(errorResponse)))
error("error when handling request %s".format(apiRequest), e)
case RequestKeys.OffsetsKey =>
@ -89,7 +91,7 @@ class KafkaApis(val requestChannel: RequestChannel,
case (topicAndPartition, partitionOffsetRequest) =>
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), null))
}
val errorResponse = OffsetResponse(apiRequest.versionId, partitionOffsetResponseMap)
val errorResponse = OffsetResponse(apiRequest.correlationId, partitionOffsetResponseMap)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
error("error when handling request %s".format(apiRequest), e)
case RequestKeys.MetadataKey =>
@ -97,7 +99,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val topicMeatadata = apiRequest.topics.map {
topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
}
val errorResponse = TopicMetadataResponse(apiRequest.versionId, topicMeatadata)
val errorResponse = TopicMetadataResponse(topicMeatadata, apiRequest.correlationId)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
error("error when handling request %s".format(apiRequest), e)
case RequestKeys.LeaderAndIsrKey =>
@ -105,7 +107,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val responseMap = apiRequest.partitionStateInfos.map {
case (topicAndPartition, partitionAndState) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
}
val errorResponse = LeaderAndIsrResponse(apiRequest.versionId, responseMap)
val errorResponse = LeaderAndIsrResponse(apiRequest.correlationId, responseMap)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
error("error when handling request %s".format(apiRequest), e)
case RequestKeys.StopReplicaKey =>
@ -114,7 +116,7 @@ class KafkaApis(val requestChannel: RequestChannel,
case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
}.toMap
error("error when handling request %s".format(apiRequest), e)
val errorResponse = StopReplicaResponse(apiRequest.versionId, responseMap)
val errorResponse = StopReplicaResponse(apiRequest.correlationId, responseMap)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
}
} finally
@ -128,7 +130,7 @@ class KafkaApis(val requestChannel: RequestChannel,
trace("Handling leader and ISR request " + leaderAndIsrRequest)
try {
val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest)
val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.versionId, response, error)
val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse)))
} catch {
case e: KafkaStorageException =>
@ -145,7 +147,7 @@ class KafkaApis(val requestChannel: RequestChannel,
trace("Handling stop replica request " + stopReplicaRequest)
val (response, error) = replicaManager.stopReplicas(stopReplicaRequest)
val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.versionId, response.toMap, error)
val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, response.toMap, error)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse)))
replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads()
@ -162,7 +164,7 @@ class KafkaApis(val requestChannel: RequestChannel,
// send any newly unblocked responses
for(fetchReq <- satisfied) {
val topicData = readMessageSets(fetchReq.fetch)
val response = FetchResponse(FetchRequest.CurrentVersion, fetchReq.fetch.correlationId, topicData)
val response = FetchResponse(fetchReq.fetch.correlationId, topicData)
requestChannel.sendResponse(new RequestChannel.Response(fetchReq.request, new FetchResponseSend(response)))
}
}
@ -193,7 +195,7 @@ class KafkaApis(val requestChannel: RequestChannel,
allPartitionHaveReplicationFactorOne ||
numPartitionsInError == produceRequest.numPartitions) {
val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap
val response = ProducerResponse(produceRequest.versionId, produceRequest.correlationId, statuses)
val response = ProducerResponse(produceRequest.correlationId, statuses)
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
} else {
// create a list of (topic, partition) pairs to use as keys for this delayed request
@ -238,8 +240,8 @@ class KafkaApis(val requestChannel: RequestChannel,
private def appendToLocalLog(partitionAndData: Map[TopicAndPartition, MessageSet]): Iterable[ProduceResult] = {
trace("Append [%s] to local log ".format(partitionAndData.toString))
partitionAndData.map {case (topicAndPartition, messages) =>
BrokerTopicStat.getBrokerTopicStat(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes)
BrokerTopicStat.getBrokerAllTopicStat.bytesInRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicStats.bytesInRate.mark(messages.sizeInBytes)
try {
val localReplica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition)
@ -247,8 +249,8 @@ class KafkaApis(val requestChannel: RequestChannel,
val info = log.append(messages.asInstanceOf[ByteBufferMessageSet], assignOffsets = true)
// update stats
BrokerTopicStat.getBrokerTopicStat(topicAndPartition.topic).messagesInRate.mark(info.count)
BrokerTopicStat.getBrokerAllTopicStat.messagesInRate.mark(info.count)
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(info.count)
BrokerTopicStats.getBrokerAllTopicStats.messagesInRate.mark(info.count)
// we may need to increment high watermark since ISR could be down to 1
localReplica.partition.maybeIncrementLeaderHW(localReplica)
@ -261,8 +263,8 @@ class KafkaApis(val requestChannel: RequestChannel,
Runtime.getRuntime.halt(1)
null
case e =>
BrokerTopicStat.getBrokerTopicStat(topicAndPartition.topic).failedProduceRequestRate.mark()
BrokerTopicStat.getBrokerAllTopicStat.failedProduceRequestRate.mark()
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicStats.failedProduceRequestRate.mark()
error("Error processing ProducerRequest on %s:%d".format(topicAndPartition.topic, topicAndPartition.partition), e)
new ProduceResult(topicAndPartition, e)
}
@ -298,7 +300,7 @@ class KafkaApis(val requestChannel: RequestChannel,
bytesReadable >= fetchRequest.minBytes ||
fetchRequest.numPartitions <= 0) {
debug("Returning fetch response %s for fetch request with correlation id %d".format(dataRead.values.map(_.error).mkString(","), fetchRequest.correlationId))
val response = new FetchResponse(FetchRequest.CurrentVersion, fetchRequest.correlationId, dataRead)
val response = new FetchResponse(fetchRequest.correlationId, dataRead)
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response)))
} else {
debug("Putting fetch request into purgatory")
@ -329,22 +331,21 @@ class KafkaApis(val requestChannel: RequestChannel,
val partitionData =
try {
val (messages, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId)
BrokerTopicStat.getBrokerTopicStat(topic).bytesOutRate.mark(messages.sizeInBytes)
BrokerTopicStat.getBrokerAllTopicStat.bytesOutRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicStats.bytesOutRate.mark(messages.sizeInBytes)
if (!isFetchFromFollower) {
new FetchResponsePartitionData(ErrorMapping.NoError, offset, highWatermark, messages)
new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages)
} else {
debug("Leader %d for topic %s partition %d received fetch request from follower %d"
.format(brokerId, topic, partition, fetchRequest.replicaId))
new FetchResponsePartitionData(ErrorMapping.NoError, offset, highWatermark, messages)
new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages)
}
} catch {
case t: Throwable =>
BrokerTopicStat.getBrokerTopicStat(topic).failedFetchRequestRate.mark()
BrokerTopicStat.getBrokerAllTopicStat.failedFetchRequestRate.mark()
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicStats.failedFetchRequestRate.mark()
error("error when processing request " + (topic, partition, offset, fetchSize), t)
new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]),
offset, -1L, MessageSet.Empty)
new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty)
}
(TopicAndPartition(topic, partition), partitionData)
}
@ -418,7 +419,7 @@ class KafkaApis(val requestChannel: RequestChannel,
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil) )
}
})
val response = OffsetResponse(OffsetRequest.CurrentVersion, responseMap)
val response = OffsetResponse(offsetRequest.correlationId, responseMap)
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
}
@ -496,9 +497,13 @@ class KafkaApis(val requestChannel: RequestChannel,
try {
/* check if auto creation of topics is turned on */
if (config.autoCreateTopics) {
CreateTopicCommand.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))
try {
CreateTopicCommand.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 {
@ -516,7 +521,7 @@ class KafkaApis(val requestChannel: RequestChannel,
}
})
topicsMetadata.foreach(metadata => trace("Sending topic metadata " + metadata.toString))
val response = new TopicMetadataResponse(metadataRequest.versionId, topicsMetadata.toSeq)
val response = new TopicMetadataResponse(topicsMetadata.toSeq, metadataRequest.correlationId)
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
}
@ -555,7 +560,8 @@ class KafkaApis(val requestChannel: RequestChannel,
/**
* A holding pen for fetch requests waiting to be satisfied
*/
class FetchRequestPurgatory(requestChannel: RequestChannel) extends RequestPurgatory[DelayedFetch, MessageSet](brokerId) {
class FetchRequestPurgatory(requestChannel: RequestChannel, purgeInterval: Int)
extends RequestPurgatory[DelayedFetch, MessageSet](brokerId, purgeInterval) {
this.logIdent = "[FetchRequestPurgatory-%d] ".format(brokerId)
/**
@ -573,7 +579,7 @@ class KafkaApis(val requestChannel: RequestChannel,
debug("Expiring fetch request %s.".format(delayed.fetch))
try {
val topicData = readMessageSets(delayed.fetch)
val response = FetchResponse(FetchRequest.CurrentVersion, delayed.fetch.correlationId, topicData)
val response = FetchResponse(delayed.fetch.correlationId, topicData)
val fromFollower = delayed.fetch.isFromFollower
delayedRequestMetrics.recordDelayedFetchExpired(fromFollower)
requestChannel.sendResponse(new RequestChannel.Response(delayed.request, new FetchResponseSend(response)))
@ -623,7 +629,7 @@ class KafkaApis(val requestChannel: RequestChannel,
(status._1, ProducerResponseStatus(pstat.error, pstat.requiredOffset))
})
val response = ProducerResponse(produce.versionId, produce.correlationId, finalErrorsAndOffsets)
val response = ProducerResponse(produce.correlationId, finalErrorsAndOffsets)
requestChannel.sendResponse(new RequestChannel.Response(
request, new BoundedByteBufferSend(response)))
@ -692,7 +698,8 @@ class KafkaApis(val requestChannel: RequestChannel,
/**
* A holding pen for produce requests waiting to be satisfied.
*/
private [kafka] class ProducerRequestPurgatory extends RequestPurgatory[DelayedProduce, RequestKey] {
private [kafka] class ProducerRequestPurgatory(purgeInterval: Int)
extends RequestPurgatory[DelayedProduce, RequestKey](brokerId, purgeInterval) {
this.logIdent = "[ProducerRequestPurgatory-%d] ".format(brokerId)
protected def checkSatisfied(followerFetchRequestKey: RequestKey,

View File

@ -167,4 +167,10 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
/* the frequency with which the highwater mark is saved out to disk */
val highWaterMarkCheckpointIntervalMs = props.getLong("replica.highwatermark.checkpoint.ms", 5000L)
/* the purge interval (in number of requests) of the fetch request purgatory */
val fetchRequestPurgatoryPurgeInterval = props.getInt("fetch.purgatory.purge.interval", 10000)
/* the purge interval (in number of requests) of the producer request purgatory */
val producerRequestPurgatoryPurgeInterval = props.getInt("producer.purgatory.purge.interval", 10000)
}

View File

@ -79,14 +79,14 @@ class BrokerTopicMetrics(name: String) extends KafkaMetricsGroup {
val failedFetchRequestRate = newMeter(name + "FailedFetchRequestsPerSec", "requests", TimeUnit.SECONDS)
}
object BrokerTopicStat extends Logging {
object BrokerTopicStats extends Logging {
private val valueFactory = (k: String) => new BrokerTopicMetrics(k)
private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory))
private val allTopicStat = new BrokerTopicMetrics("AllTopics")
private val allTopicStats = new BrokerTopicMetrics("AllTopics")
def getBrokerAllTopicStat(): BrokerTopicMetrics = allTopicStat
def getBrokerAllTopicStats(): BrokerTopicMetrics = allTopicStats
def getBrokerTopicStat(topic: String): BrokerTopicMetrics = {
def getBrokerTopicStats(topic: String): BrokerTopicMetrics = {
stats.getAndMaybePut(topic + "-")
}
}

View File

@ -23,7 +23,7 @@ import kafka.utils._
import java.util.concurrent._
import atomic.AtomicBoolean
import org.I0Itec.zkclient.ZkClient
import kafka.controller.{ControllerStat, KafkaController}
import kafka.controller.{ControllerStats, KafkaController}
/**
* Represents the lifecycle of a single Kafka broker. Handles all functionality required
@ -96,9 +96,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
* Forces some dynamic jmx beans to be registered on server startup.
*/
private def registerStats() {
BrokerTopicStat.getBrokerAllTopicStat()
ControllerStat.offlinePartitionRate
ControllerStat.uncleanLeaderElectionRate
BrokerTopicStats.getBrokerAllTopicStats()
ControllerStats.offlinePartitionRate
ControllerStats.uncleanLeaderElectionRate
}
/**

View File

@ -43,8 +43,7 @@ class KafkaZooKeeper(config: KafkaConfig) extends Logging {
private def registerBrokerInZk() {
info("Registering broker " + brokerIdPath)
val hostName = config.hostName
val creatorId = hostName + "-" + System.currentTimeMillis
ZkUtils.registerBrokerInZk(zkClient, config.brokerId, hostName, creatorId, config.port)
ZkUtils.registerBrokerInZk(zkClient, config.brokerId, hostName, config.port)
}
/**

View File

@ -28,7 +28,7 @@ class ReplicaFetcherThread(name:String,
brokerConfig: KafkaConfig,
replicaMgr: ReplicaManager)
extends AbstractFetcherThread(name = name,
clientId = FetchRequest.ReplicaFetcherClientId + "- %s:%d".format(sourceBroker.host, sourceBroker.port) ,
clientId = FetchRequest.ReplicaFetcherClientId,
sourceBroker = sourceBroker,
socketTimeout = brokerConfig.replicaSocketTimeoutMs,
socketBufferSize = brokerConfig.replicaSocketBufferSize,

View File

@ -18,13 +18,13 @@
package kafka.server
import scala.collection._
import java.util.LinkedList
import java.util.concurrent._
import java.util.concurrent.atomic._
import kafka.network._
import kafka.utils._
import com.yammer.metrics.core.Gauge
import kafka.metrics.KafkaMetricsGroup
import java.util
import com.yammer.metrics.core.Gauge
/**
@ -61,11 +61,21 @@ 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.
*
*/
abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) extends Logging with KafkaMetricsGroup {
abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0, purgeInterval: Int = 10000)
extends Logging with KafkaMetricsGroup {
/* a list of requests watching each key */
private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers))
private val requestCounter = new AtomicInteger(0)
newGauge(
"PurgatorySize",
new Gauge[Int] {
def getValue = watchersForKey.values.map(_.numRequests).sum + expiredRequestReaper.numRequests
}
)
newGauge(
"NumDelayedRequests",
new Gauge[Int] {
@ -78,10 +88,19 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
private val expirationThread = Utils.daemonThread("request-expiration-task", expiredRequestReaper)
expirationThread.start()
def purgeSatisfied() {
expiredRequestReaper.forcePurge()
}
/**
* Add a new delayed request watching the contained keys
*/
def watch(delayedRequest: T) {
if (requestCounter.getAndIncrement() >= purgeInterval) {
requestCounter.set(0)
purgeSatisfied()
}
for(key <- delayedRequest.keys) {
var lst = watchersFor(key)
lst.add(delayedRequest)
@ -125,37 +144,29 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
*/
private class Watchers {
/* a few magic parameters to help do cleanup to avoid accumulating old watchers */
private val CleanupThresholdSize = 100
private val CleanupThresholdPrct = 0.5
private val requests = new LinkedList[T]
private val requests = new util.ArrayList[T]
/* you can only change this if you have added something or marked something satisfied */
var liveCount = 0.0
def numRequests = requests.size
def add(t: T) {
synchronized {
requests.add(t)
liveCount += 1
maybePurge()
}
}
private def maybePurge() {
if(requests.size > CleanupThresholdSize && liveCount / requests.size < CleanupThresholdPrct) {
def purgeSatisfied(): Int = {
synchronized {
val iter = requests.iterator()
var purged = 0
while(iter.hasNext) {
val curr = iter.next
if(curr.satisfied.get())
if(curr.satisfied.get()) {
iter.remove()
purged += 1
}
}
}
}
def decLiveCount() {
synchronized {
liveCount -= 1
purged
}
}
@ -177,7 +188,6 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
val updated = curr.satisfied.compareAndSet(false, true)
if(updated == true) {
response += curr
liveCount -= 1
expiredRequestReaper.satisfyRequest()
}
}
@ -193,17 +203,16 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
*/
private class ExpiredRequestReaper extends Runnable with Logging {
this.logIdent = "ExpiredRequestReaper-%d ".format(brokerId)
/* a few magic parameters to help do cleanup to avoid accumulating old watchers */
private val CleanupThresholdSize = 100
private val CleanupThresholdPrct = 0.5
private val delayed = new DelayQueue[T]
private val running = new AtomicBoolean(true)
private val shutdownLatch = new CountDownLatch(1)
private val needsPurge = new AtomicBoolean(false)
/* The count of elements in the delay queue that are unsatisfied */
private [kafka] val unsatisfied = new AtomicInteger(0)
def numRequests = delayed.size()
/** Main loop for the expiry thread */
def run() {
while(running.get) {
@ -214,10 +223,10 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
}
} catch {
case ie: InterruptedException =>
if(needsPurge.getAndSet(false)) {
val purged = purgeSatisfied()
debug("Forced purge of " + purged + " requests from delay queue.")
}
val purged = purgeSatisfied()
debug("Purged %d requests from delay queue.".format(purged))
val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum
debug("Purged %d (watcher) requests.".format(numPurgedFromWatchers))
case e: Exception =>
error("Error in long poll expiry thread: ", e)
}
@ -229,12 +238,9 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
def enqueue(t: T) {
delayed.add(t)
unsatisfied.incrementAndGet()
if(unsatisfied.get > CleanupThresholdSize && unsatisfied.get / delayed.size.toDouble < CleanupThresholdPrct)
forcePurge()
}
private def forcePurge() {
needsPurge.set(true)
def forcePurge() {
expirationThread.interrupt()
}
@ -259,8 +265,6 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
val updated = curr.satisfied.compareAndSet(false, true)
if(updated) {
unsatisfied.getAndDecrement()
for(key <- curr.keys)
watchersFor(key).decLiveCount()
return curr
}
}

View File

@ -41,7 +41,7 @@ object ConsumerOffsetChecker extends Logging {
val brokerInfo = ZkUtils.readDataMaybeNull(zkClient, "/brokers/ids/%s".format(bid))._1
val consumer = brokerInfo match {
case BrokerIpPattern(ip, port) =>
Some(new SimpleConsumer(ip, port.toInt, 10000, 100000))
Some(new SimpleConsumer(ip, port.toInt, 10000, 100000, "ConsumerOffsetChecker"))
case _ =>
error("Could not parse broker info %s".format(brokerInfo))
None

View File

@ -57,8 +57,8 @@ object DumpLogSegments {
val verifyOnly = if(options.has(verifyOpt)) true else false
val files = options.valueOf(filesOpt).split(",")
val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Int, Int)]]
val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Int, Int)]]
val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Long, Long)]]
val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Long, Long)]]
for(arg <- files) {
val file = new File(arg)
@ -89,7 +89,7 @@ object DumpLogSegments {
}
/* print out the contents of the index */
private def dumpIndex(file: File, verifyOnly: Boolean, misMatchesForIndexFilesMap: mutable.HashMap[String, List[(Int, Int)]]) {
private def dumpIndex(file: File, verifyOnly: Boolean, misMatchesForIndexFilesMap: mutable.HashMap[String, List[(Long, Long)]]) {
val startOffset = file.getName().split("\\.")(0).toLong
val logFileName = file.getAbsolutePath.split("\\.")(0) + Log.LogFileSuffix
val logFile = new File(logFileName)
@ -100,8 +100,8 @@ object DumpLogSegments {
val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, messageSet.sizeInBytes())
val messageAndOffset = partialFileMessageSet.head
if(messageAndOffset.offset != entry.offset + index.baseOffset) {
var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getName, List[(Int, Int)]())
misMatchesSeq ::=((entry.offset + index.baseOffset, messageAndOffset.offset).asInstanceOf[(Int, Int)])
var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getName, List[(Long, Long)]())
misMatchesSeq ::=(entry.offset + index.baseOffset, messageAndOffset.offset)
misMatchesForIndexFilesMap.put(file.getName, misMatchesSeq)
}
// since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one
@ -113,7 +113,7 @@ object DumpLogSegments {
}
/* print out the contents of the log */
private def dumpLog(file: File, printContents: Boolean, nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Int, Int)]]) {
private def dumpLog(file: File, printContents: Boolean, nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Long, Long)]]) {
val startOffset = file.getName().split("\\.")(0).toLong
println("Starting offset: " + startOffset)
val messageSet = new FileMessageSet(file)
@ -126,8 +126,8 @@ object DumpLogSegments {
lastOffset = messageAndOffset.offset
// If it's uncompressed message, its offset must be lastOffset + 1 no matter last message is compressed or uncompressed
else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) {
var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getName, List[(Int, Int)]())
nonConsecutivePairsSeq ::=((lastOffset, messageAndOffset.offset).asInstanceOf[(Int, Int)])
var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getName, List[(Long, Long)]())
nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset)
nonConsecutivePairsForLogFilesMap.put(file.getName, nonConsecutivePairsSeq)
}
lastOffset = messageAndOffset.offset

View File

@ -67,7 +67,7 @@ object GetOffsetShell {
val partition = options.valueOf(partitionOpt).intValue
var time = options.valueOf(timeOpt).longValue
val nOffsets = options.valueOf(nOffsetsOpt).intValue
val consumer = new SimpleConsumer(url.getHost, url.getPort, 10000, 100000)
val consumer = new SimpleConsumer(url.getHost, url.getPort, 10000, 100000, "GetOffsetShell")
val topicAndPartition = TopicAndPartition(topic, partition)
val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets)))
val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets

View File

@ -17,7 +17,6 @@
package kafka.tools
import kafka.message.Message
import joptsimple.OptionParser
import kafka.utils.{Utils, CommandLineUtils, Logging}
import kafka.producer.{KeyedMessage, ProducerConfig, Producer}

View File

@ -1,71 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.tools
import java.io._
import joptsimple._
import kafka.producer._
import kafka.utils.Utils
/**
* Interactive shell for producing messages from the command line
*/
object ProducerShell {
def main(args: Array[String]) {
val parser = new OptionParser
val producerPropsOpt = parser.accepts("props", "REQUIRED: Properties file with the producer properties.")
.withRequiredArg
.describedAs("properties")
.ofType(classOf[String])
val topicOpt = parser.accepts("topic", "REQUIRED: The topic to produce to.")
.withRequiredArg
.describedAs("topic")
.ofType(classOf[String])
val options = parser.parse(args : _*)
for(arg <- List(producerPropsOpt, topicOpt)) {
if(!options.has(arg)) {
System.err.println("Missing required argument \"" + arg + "\"")
parser.printHelpOn(System.err)
System.exit(1)
}
}
val propsFile = options.valueOf(producerPropsOpt)
val producerConfig = new ProducerConfig(Utils.loadProps(propsFile))
val topic = options.valueOf(topicOpt)
val producer = new Producer[String, String](producerConfig)
val input = new BufferedReader(new InputStreamReader(System.in))
var done = false
while(!done) {
val line = input.readLine()
if(line == null) {
done = true
} else {
val message = line.trim
producer.send(new KeyedMessage[String, String](topic, message))
println("Sent: %s (%d bytes)".format(line, message.getBytes.length))
}
}
producer.close()
}
}

View File

@ -24,7 +24,7 @@ import kafka.producer.{KeyedMessage, ProducerConfig, Producer}
import kafka.consumer._
import kafka.utils.{Logging, ZkUtils}
import kafka.api.OffsetRequest
import kafka.message.{CompressionCodec, Message}
import kafka.message.CompressionCodec
object ReplayLogProducer extends Logging {

View File

@ -19,12 +19,10 @@ package kafka.tools
import joptsimple._
import kafka.utils._
import kafka.producer.ProducerConfig
import kafka.consumer._
import kafka.client.ClientUtils
import kafka.api.{OffsetRequest, FetchRequestBuilder, Request}
import kafka.cluster.Broker
import java.util.Properties
import scala.collection.JavaConversions._
/**
@ -74,7 +72,7 @@ object SimpleConsumerShell extends Logging {
.withRequiredArg
.describedAs("class")
.ofType(classOf[String])
.defaultsTo(classOf[NewlineMessageFormatter].getName)
.defaultsTo(classOf[DefaultMessageFormatter].getName)
val messageFormatterArgOpt = parser.accepts("property")
.withRequiredArg
.describedAs("prop")
@ -127,7 +125,7 @@ object SimpleConsumerShell extends Logging {
// getting topic metadata
info("Getting topic metatdata...")
val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt))
val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers).topicsMetadata
val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId).topicsMetadata
if(topicsMetadata.size != 1 || !topicsMetadata(0).topic.equals(topic)) {
System.err.println(("Error: no valid topic metadata for topic: %s, " + "what we get from server is only: %s").format(topic, topicsMetadata))
System.exit(1)
@ -167,7 +165,7 @@ object SimpleConsumerShell extends Logging {
System.exit(1)
}
if(startingOffset < 0)
startingOffset = SimpleConsumer.earliestOrLatestOffset(fetchTargetBroker, topic, partitionId, startingOffset, false)
startingOffset = SimpleConsumer.earliestOrLatestOffset(fetchTargetBroker, topic, partitionId, startingOffset, clientId, false)
// initializing formatter
val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter]
@ -175,7 +173,7 @@ object SimpleConsumerShell extends Logging {
info("Starting simple consumer shell to partition [%s, %d], replica [%d], host and port: [%s, %d], from offset [%d]"
.format(topic, partitionId, replicaId, fetchTargetBroker.host, fetchTargetBroker.port, startingOffset))
val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, 10000, 64*1024)
val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, 10000, 64*1024, clientId)
val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() {
def run() {
var offset = startingOffset

View File

@ -65,7 +65,7 @@ object UpdateOffsetsInZK {
ZkUtils.getBrokerInfo(zkClient, broker) match {
case Some(brokerInfo) =>
val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024)
val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024, "UpdateOffsetsInZk")
val topicAndPartition = TopicAndPartition(topic, partition)
val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1)))
val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head

View File

@ -1,41 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.utils
import kafka.common.InvalidTopicException
import util.matching.Regex
object Topic {
val legalChars = "[a-zA-Z0-9_-]"
val maxNameLength = 255
private val rgx = new Regex(legalChars + "+")
def validate(topic: String) {
if (topic.length <= 0)
throw new InvalidTopicException("topic name is illegal, can't be empty")
else if (topic.length > maxNameLength)
throw new InvalidTopicException("topic name is illegal, can't be longer than " + maxNameLength + " characters")
rgx.findFirstIn(topic) match {
case Some(t) =>
if (!t.equals(topic))
throw new InvalidTopicException("topic name " + topic + " is illegal, contains a character other than ASCII alphanumerics, _ and -")
case None => throw new InvalidTopicException("topic name " + topic + " is illegal, contains a character other than ASCII alphanumerics, _ and -")
}
}
}

View File

@ -405,7 +405,7 @@ object Utils extends Logging {
}
/**
* This method gets comma seperated values which contains key,value pairs and returns a map of
* This method gets comma separated values which contains key,value pairs and returns a map of
* key value pairs. the format of allCSVal is key1:val1, key2:val2 ....
*/
def parseCsvMap(str: String): Map[String, String] = {

View File

@ -180,11 +180,11 @@ object ZkUtils extends Logging {
replicas.contains(brokerId.toString)
}
def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, creator: String, port: Int) {
def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int) {
val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id
val broker = new Broker(id, creator, host, port)
val broker = new Broker(id, host, port)
try {
createEphemeralPathExpectConflict(zkClient, brokerIdPath, broker.getZKString)
createEphemeralPathExpectConflict(zkClient, brokerIdPath, broker.getZkString)
} catch {
case e: ZkNodeExistsException =>
throw new RuntimeException("A broker is already registered on the path " + brokerIdPath + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " + "else you have shutdown this broker and restarted it faster than the zookeeper " + "timeout so it appears to be re-registering.")

View File

@ -17,7 +17,6 @@
package kafka
import message.Message
import org.apache.log4j.PropertyConfigurator
import kafka.utils.Logging
import serializer.Encoder

View File

@ -18,7 +18,6 @@
package kafka
import consumer._
import message.Message
import utils.Utils
import java.util.concurrent.CountDownLatch

View File

@ -22,7 +22,7 @@ import org.scalatest.junit.JUnit3Suite
import kafka.zk.ZooKeeperTestHarness
import kafka.server.KafkaConfig
import kafka.utils.{ZkUtils, TestUtils}
import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.common.{TopicExistsException, ErrorMapping, TopicAndPartition}
class AdminTest extends JUnit3Suite with ZooKeeperTestHarness {
@ -170,7 +170,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness {
AdminUtils.createTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient)
fail("shouldn't be able to create a topic already exists")
} catch {
case e: AdministrationException => // this is good
case e: TopicExistsException => // this is good
case e2 => throw e2
}
}
@ -374,33 +374,34 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness {
var controllerId = ZkUtils.getController(zkClient)
var controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
var partitionsRemaining = controller.shutdownBroker(2)
assertEquals(0, partitionsRemaining)
var topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
var leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
assertTrue(leaderAfterShutdown != leaderBeforeShutdown)
assertEquals(2, topicMetadata.partitionsMetadata.head.isr.size)
try {
assertEquals(0, partitionsRemaining)
var topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
var leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
assertTrue(leaderAfterShutdown != leaderBeforeShutdown)
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, topicMetadata.partitionsMetadata.head.isr.size)
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.allLeaders(TopicAndPartition("test", 1)).leaderAndIsr.isr.size)
leaderBeforeShutdown = leaderAfterShutdown
controllerId = ZkUtils.getController(zkClient)
controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
partitionsRemaining = controller.shutdownBroker(0)
assertEquals(1, partitionsRemaining)
topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
assertTrue(leaderAfterShutdown == leaderBeforeShutdown)
assertEquals(1, topicMetadata.partitionsMetadata.head.isr.size)
servers.foreach(_.shutdown())
leaderBeforeShutdown = leaderAfterShutdown
controllerId = ZkUtils.getController(zkClient)
controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
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.allLeaders(TopicAndPartition("test", 1)).leaderAndIsr.isr.size)
} finally {
servers.foreach(_.shutdown())
}
}
private def checkIfReassignPartitionPathExists(): Boolean = {

View File

@ -75,10 +75,11 @@ object SerializationTestUtils{
TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100)
)
private val partitionMetaData0 = new PartitionMetadata(0, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty)
private val partitionMetaData1 = new PartitionMetadata(1, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty)
private val partitionMetaData2 = new PartitionMetadata(2, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty)
private val partitionMetaData3 = new PartitionMetadata(3, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty)
private val brokers = List(new Broker(0, "localhost", 1011), new Broker(1, "localhost", 1012), new Broker(2, "localhost", 1013))
private val partitionMetaData0 = new PartitionMetadata(0, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 0)
private val partitionMetaData1 = new PartitionMetadata(1, Some(brokers.head), replicas = brokers, isr = brokers.tail, errorCode = 1)
private val partitionMetaData2 = new PartitionMetadata(2, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 2)
private val partitionMetaData3 = new PartitionMetadata(3, Some(brokers.head), replicas = brokers, isr = brokers.tail.tail, errorCode = 3)
private val partitionMetaDataSeq = Seq(partitionMetaData0, partitionMetaData1, partitionMetaData2, partitionMetaData3)
private val topicmetaData1 = new TopicMetadata(topic1, partitionMetaDataSeq)
private val topicmetaData2 = new TopicMetadata(topic2, partitionMetaDataSeq)
@ -104,7 +105,7 @@ object SerializationTestUtils{
def createTestStopReplicaResponse() : StopReplicaResponse = {
val responseMap = Map(((topic1, 0), ErrorMapping.NoError),
((topic2, 0), ErrorMapping.NoError))
new StopReplicaResponse(1, responseMap.toMap)
new StopReplicaResponse(0, responseMap.toMap)
}
def createTestProducerRequest: ProducerRequest = {
@ -112,7 +113,7 @@ object SerializationTestUtils{
}
def createTestProducerResponse: ProducerResponse =
ProducerResponse(1, 1, Map(
ProducerResponse(1, Map(
TopicAndPartition(topic1, 0) -> ProducerResponseStatus(0.toShort, 10001),
TopicAndPartition(topic2, 0) -> ProducerResponseStatus(0.toShort, 20001)
))
@ -122,7 +123,7 @@ object SerializationTestUtils{
}
def createTestFetchResponse: FetchResponse = {
FetchResponse(1, 1, topicDataFetchResponse)
FetchResponse(1, topicDataFetchResponse)
}
def createTestOffsetRequest = new OffsetRequest(
@ -131,17 +132,17 @@ object SerializationTestUtils{
)
def createTestOffsetResponse: OffsetResponse = {
new OffsetResponse(OffsetRequest.CurrentVersion, collection.immutable.Map(
new OffsetResponse(0, collection.immutable.Map(
TopicAndPartition(topic1, 1) -> PartitionOffsetsResponse(ErrorMapping.NoError, Seq(1000l, 2000l, 3000l, 4000l)))
)
}
def createTestTopicMetadataRequest: TopicMetadataRequest = {
new TopicMetadataRequest(1, "client 1", Seq(topic1, topic2))
new TopicMetadataRequest(1, "client 1", Seq(topic1, topic2), 1)
}
def createTestTopicMetadataResponse: TopicMetadataResponse = {
new TopicMetadataResponse(1, Seq(topicmetaData1, topicmetaData2))
new TopicMetadataResponse(Seq(topicmetaData1, topicmetaData2), 1)
}
}

View File

@ -0,0 +1,89 @@
/**
* 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 unit.kafka.common
import junit.framework.Assert._
import collection.mutable.ArrayBuffer
import org.junit.Test
import kafka.common.InvalidConfigException
import kafka.producer.ProducerConfig
import kafka.consumer.ConsumerConfig
class ConfigTest {
@Test
def testInvalidClientIds() {
val invalidClientIds = new ArrayBuffer[String]()
val badChars = Array('/', '\\', ',', '\0', ':', "\"", '\'', ';', '*', '?', '.', ' ', '\t', '\r', '\n', '=')
for (weirdChar <- badChars) {
invalidClientIds += "Is" + weirdChar + "illegal"
}
for (i <- 0 until invalidClientIds.size) {
try {
ProducerConfig.validateClientId(invalidClientIds(i))
fail("Should throw InvalidClientIdException.")
}
catch {
case e: InvalidConfigException => "This is good."
}
}
val validClientIds = new ArrayBuffer[String]()
validClientIds += ("valid", "CLIENT", "iDs", "ar6", "VaL1d", "_0-9_", "")
for (i <- 0 until validClientIds.size) {
try {
ProducerConfig.validateClientId(validClientIds(i))
}
catch {
case e: Exception => fail("Should not throw exception.")
}
}
}
@Test
def testInvalidGroupIds() {
val invalidGroupIds = new ArrayBuffer[String]()
val badChars = Array('/', '\\', ',', '\0', ':', "\"", '\'', ';', '*', '?', '.', ' ', '\t', '\r', '\n', '=')
for (weirdChar <- badChars) {
invalidGroupIds += "Is" + weirdChar + "illegal"
}
for (i <- 0 until invalidGroupIds.size) {
try {
ConsumerConfig.validateGroupId(invalidGroupIds(i))
fail("Should throw InvalidGroupIdException.")
}
catch {
case e: InvalidConfigException => "This is good."
}
}
val validGroupIds = new ArrayBuffer[String]()
validGroupIds += ("valid", "GROUP", "iDs", "ar6", "VaL1d", "_0-9_", "")
for (i <- 0 until validGroupIds.size) {
try {
ConsumerConfig.validateGroupId(validGroupIds(i))
}
catch {
case e: Exception => fail("Should not throw exception.")
}
}
}
}

View File

@ -15,11 +15,11 @@
* limitations under the License.
*/
package kafka.utils
package unit.kafka.common
import junit.framework.Assert._
import collection.mutable.ArrayBuffer
import kafka.common.InvalidTopicException
import kafka.common.{Topic, InvalidTopicException}
import org.junit.Test
class TopicTest {
@ -34,7 +34,7 @@ class TopicTest {
invalidTopicNames += longName
val badChars = Array('/', '\\', ',', '\0', ':', "\"", '\'', ';', '*', '?', '.')
for (weirdChar <- badChars) {
invalidTopicNames += "Is" + weirdChar + "funny"
invalidTopicNames += "Is" + weirdChar + "illegal"
}
for (i <- 0 until invalidTopicNames.size) {

View File

@ -47,7 +47,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness {
val group = "group1"
val consumer0 = "consumer0"
val consumedOffset = 5
val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.brokerId.toString, "localhost", c.port)))
val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port)))
val queue = new LinkedBlockingQueue[FetchedDataChunk]
val topicInfos = configs.map(c => new PartitionTopicInfo(topic,
c.brokerId,
@ -55,7 +55,8 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness {
queue,
new AtomicLong(consumedOffset),
new AtomicLong(0),
new AtomicInteger(0)))
new AtomicInteger(0),
""))
val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0))
override def setUp() {
@ -78,8 +79,9 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness {
consumerConfig.consumerTimeoutMs,
new StringDecoder(),
new StringDecoder(),
enableShallowIterator = false)
var receivedMessages = (0 until 5).map(i => iter.next.message).toList
enableShallowIterator = false,
clientId = "")
val receivedMessages = (0 until 5).map(i => iter.next.message).toList
assertFalse(iter.hasNext)
assertEquals(1, queue.size) // This is only the shutdown command.

View File

@ -24,7 +24,6 @@ import kafka.server._
import org.apache.log4j.{Level, Logger}
import org.scalatest.junit.JUnit3Suite
import kafka.utils.TestUtils
import kafka.message.Message
import kafka.serializer._
import kafka.producer.{Producer, KeyedMessage}

View File

@ -23,7 +23,6 @@ import scala.collection._
import junit.framework.Assert._
import kafka.cluster._
import kafka.message._
import kafka.server._
import org.scalatest.junit.JUnit3Suite
import kafka.consumer._
@ -41,7 +40,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
yield new KafkaConfig(props)
val messages = new mutable.HashMap[Int, Seq[Array[Byte]]]
val topic = "topic"
val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.brokerId.toString, "localhost", c.port)))
val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port)))
val shutdown = ZookeeperConsumerConnector.shutdownCommand
val queue = new LinkedBlockingQueue[FetchedDataChunk]
val topicInfos = configs.map(c => new PartitionTopicInfo(topic,
@ -50,7 +49,8 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
queue,
new AtomicLong(0),
new AtomicLong(0),
new AtomicInteger(0)))
new AtomicInteger(0),
""))
var fetcher: ConsumerFetcherManager = null
@ -84,7 +84,9 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
def sendMessages(messagesPerNode: Int): Int = {
var count = 0
for(conf <- configs) {
val producer: Producer[String, Array[Byte]] = TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs), new DefaultEncoder(), new StringEncoder())
val producer: Producer[String, Array[Byte]] = TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs),
new DefaultEncoder(),
new StringEncoder())
val ms = 0.until(messagesPerNode).map(x => (conf.brokerId * 5 + x).toString.getBytes).toArray
messages += conf.brokerId -> ms
producer.send(ms.map(m => KeyedMessage[String, Array[Byte]](topic, topic, m)):_*)

View File

@ -18,7 +18,7 @@
package kafka.integration
import kafka.api.FetchRequestBuilder
import kafka.message.{Message, ByteBufferMessageSet}
import kafka.message.ByteBufferMessageSet
import kafka.server.{KafkaRequestHandler, KafkaConfig}
import org.apache.log4j.{Level, Logger}
import org.junit.Assert._

View File

@ -25,7 +25,6 @@ import java.util.Properties
import kafka.utils.Utils
import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
import kafka.serializer._
import kafka.message.Message
import kafka.utils.TestUtils
import org.apache.log4j.{Level, Logger}
import org.I0Itec.zkclient.ZkClient

View File

@ -21,7 +21,6 @@ import kafka.consumer.SimpleConsumer
import org.scalatest.junit.JUnit3Suite
import java.util.Properties
import kafka.producer.{ProducerConfig, Producer}
import kafka.message.Message
import kafka.utils.TestUtils
import kafka.serializer._
@ -44,10 +43,7 @@ trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarnes
props.put("producer.request.required.acks", "-1")
props.put("serializer.class", classOf[StringEncoder].getName.toString)
producer = new Producer(new ProducerConfig(props))
consumer = new SimpleConsumer(host,
port,
1000000,
64*1024)
consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "")
}
override def tearDown() {

View File

@ -97,7 +97,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
private def mockLogManagerAndTestTopic(topic: String): 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).times(2)
EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes()
EasyMock.replay(replicaManager)
// create a topic metadata request

View File

@ -29,7 +29,7 @@ import kafka.producer.KeyedMessage
import kafka.javaapi.producer.Producer
import kafka.utils.IntEncoder
import kafka.utils.TestUtils._
import kafka.utils.{Utils, Logging, TestUtils}
import kafka.utils.{Logging, TestUtils}
import kafka.consumer.{KafkaStream, ConsumerConfig}
import kafka.zk.ZooKeeperTestHarness

View File

@ -439,6 +439,33 @@ class LogTest extends JUnitSuite {
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
}
/**
* When we open a log any index segments without an associated log segment should be deleted.
*/
@Test
def testBogusIndexSegmentsAreRemoved() {
val bogusIndex1 = Log.indexFilename(logDir, 0)
val bogusIndex2 = Log.indexFilename(logDir, 5)
val set = TestUtils.singleMessageSet("test".getBytes())
val log = new Log(logDir,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.maxMessageSize,
maxIndexSize = 1000,
indexIntervalBytes = 1,
needsRecovery = false)
assertTrue("The first index file should have been replaced with a larger file", bogusIndex1.length > 0)
assertFalse("The second index file should have been deleted.", bogusIndex2.exists)
// check that we can append to the log
for(i <- 0 until 10)
log.append(set)
log.delete()
}
/**
* Verify that truncation works correctly after re-opening the log
*/

View File

@ -24,7 +24,6 @@ import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.{TestUtils, Utils, Logging}
import junit.framework.Assert._
import kafka.api.FetchRequestBuilder
import kafka.message.Message
import kafka.producer.async.MissingConfigException
import kafka.serializer.Encoder
import kafka.zk.ZooKeeperTestHarness
@ -57,7 +56,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with
logDirZk = new File(logDirZkPath)
config = new KafkaConfig(propsZk)
serverZk = TestUtils.createServer(config);
simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64*1024)
simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64*1024, "")
}
@After

Some files were not shown because too many files have changed in this diff Show More