mirror of https://github.com/apache/kafka.git
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:
commit
32dae955d5
|
@ -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 $@
|
|
|
@ -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
|
|
@ -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
|
|
@ -109,7 +109,7 @@ public class KafkaETLContext {
|
||||||
|
|
||||||
// read data from queue
|
// read data from queue
|
||||||
URI uri = _request.getURI();
|
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
|
// get available offset range
|
||||||
_offsetRange = getOffsetRange();
|
_offsetRange = getOffsetRange();
|
||||||
|
|
|
@ -25,7 +25,8 @@ import org.I0Itec.zkclient.ZkClient
|
||||||
import org.I0Itec.zkclient.exception.ZkNodeExistsException
|
import org.I0Itec.zkclient.exception.ZkNodeExistsException
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
import kafka.common.{BrokerNotAvailableException, LeaderNotAvailableException, ReplicaNotAvailableException, ErrorMapping}
|
import kafka.common._
|
||||||
|
import scala.Some
|
||||||
|
|
||||||
object AdminUtils extends Logging {
|
object AdminUtils extends Logging {
|
||||||
val rand = new Random
|
val rand = new Random
|
||||||
|
@ -82,7 +83,7 @@ object AdminUtils extends Logging {
|
||||||
ZkUtils.createPersistentPath(zkClient, zkPath, jsonPartitionMap)
|
ZkUtils.createPersistentPath(zkClient, zkPath, jsonPartitionMap)
|
||||||
debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionMap))
|
debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionMap))
|
||||||
} catch {
|
} 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)
|
case e2 => throw new AdministrationException(e2.toString)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ import joptsimple.OptionParser
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.I0Itec.zkclient.ZkClient
|
import org.I0Itec.zkclient.ZkClient
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
|
import kafka.common.Topic
|
||||||
|
|
||||||
object CreateTopicCommand extends Logging {
|
object CreateTopicCommand extends Logging {
|
||||||
|
|
||||||
|
|
|
@ -30,10 +30,10 @@ case class PartitionFetchInfo(offset: Long, fetchSize: Int)
|
||||||
|
|
||||||
|
|
||||||
object FetchRequest {
|
object FetchRequest {
|
||||||
val CurrentVersion = 1.shortValue()
|
val CurrentVersion = 0.shortValue
|
||||||
val DefaultMaxWait = 0
|
val DefaultMaxWait = 0
|
||||||
val DefaultMinBytes = 0
|
val DefaultMinBytes = 0
|
||||||
val ReplicaFetcherClientId = "replica fetcher"
|
val ReplicaFetcherClientId = "replica-fetcher"
|
||||||
val DefaultCorrelationId = 0
|
val DefaultCorrelationId = 0
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): FetchRequest = {
|
def readFrom(buffer: ByteBuffer): FetchRequest = {
|
||||||
|
|
|
@ -27,29 +27,25 @@ import kafka.api.ApiUtils._
|
||||||
object FetchResponsePartitionData {
|
object FetchResponsePartitionData {
|
||||||
def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = {
|
def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = {
|
||||||
val error = buffer.getShort
|
val error = buffer.getShort
|
||||||
val initialOffset = buffer.getLong
|
|
||||||
val hw = buffer.getLong
|
val hw = buffer.getLong
|
||||||
val messageSetSize = buffer.getInt
|
val messageSetSize = buffer.getInt
|
||||||
val messageSetBuffer = buffer.slice()
|
val messageSetBuffer = buffer.slice()
|
||||||
messageSetBuffer.limit(messageSetSize)
|
messageSetBuffer.limit(messageSetSize)
|
||||||
buffer.position(buffer.position + messageSetSize)
|
buffer.position(buffer.position + messageSetSize)
|
||||||
new FetchResponsePartitionData(error, initialOffset,
|
new FetchResponsePartitionData(error, hw, new ByteBufferMessageSet(messageSetBuffer))
|
||||||
hw, new ByteBufferMessageSet(messageSetBuffer))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
val headerSize =
|
val headerSize =
|
||||||
2 + /* error code */
|
2 + /* error code */
|
||||||
8 + /* initialOffset */
|
|
||||||
8 + /* high watermark */
|
8 + /* high watermark */
|
||||||
4 /* messageSetSize */
|
4 /* messageSetSize */
|
||||||
}
|
}
|
||||||
|
|
||||||
case class FetchResponsePartitionData(error: Short = ErrorMapping.NoError,
|
case class FetchResponsePartitionData(error: Short = ErrorMapping.NoError, hw: Long = -1L, messages: MessageSet) {
|
||||||
initialOffset:Long = 0L, hw: Long = -1L, messages: MessageSet) {
|
|
||||||
|
|
||||||
val sizeInBytes = FetchResponsePartitionData.headerSize + messages.sizeInBytes
|
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)
|
private val buffer = ByteBuffer.allocate( 4 /** partitionId **/ + FetchResponsePartitionData.headerSize)
|
||||||
buffer.putInt(partitionId)
|
buffer.putInt(partitionId)
|
||||||
buffer.putShort(partitionData.error)
|
buffer.putShort(partitionData.error)
|
||||||
buffer.putLong(partitionData.initialOffset)
|
|
||||||
buffer.putLong(partitionData.hw)
|
buffer.putLong(partitionData.hw)
|
||||||
buffer.putInt(partitionData.messages.sizeInBytes)
|
buffer.putInt(partitionData.messages.sizeInBytes)
|
||||||
buffer.rewind()
|
buffer.rewind()
|
||||||
|
@ -141,12 +136,10 @@ class TopicDataSend(val topicData: TopicData) extends Send {
|
||||||
object FetchResponse {
|
object FetchResponse {
|
||||||
|
|
||||||
val headerSize =
|
val headerSize =
|
||||||
2 + /* versionId */
|
|
||||||
4 + /* correlationId */
|
4 + /* correlationId */
|
||||||
4 /* topic count */
|
4 /* topic count */
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): FetchResponse = {
|
def readFrom(buffer: ByteBuffer): FetchResponse = {
|
||||||
val versionId = buffer.getShort
|
|
||||||
val correlationId = buffer.getInt
|
val correlationId = buffer.getInt
|
||||||
val topicCount = buffer.getInt
|
val topicCount = buffer.getInt
|
||||||
val pairs = (1 to topicCount).flatMap(_ => {
|
val pairs = (1 to topicCount).flatMap(_ => {
|
||||||
|
@ -156,13 +149,12 @@ object FetchResponse {
|
||||||
(TopicAndPartition(topicData.topic, partitionId), partitionData)
|
(TopicAndPartition(topicData.topic, partitionId), partitionData)
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
FetchResponse(versionId, correlationId, Map(pairs:_*))
|
FetchResponse(correlationId, Map(pairs:_*))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
case class FetchResponse(versionId: Short,
|
case class FetchResponse(correlationId: Int,
|
||||||
correlationId: Int,
|
|
||||||
data: Map[TopicAndPartition, FetchResponsePartitionData]) {
|
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)
|
private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize)
|
||||||
buffer.putInt(size)
|
buffer.putInt(size)
|
||||||
buffer.putShort(fetchResponse.versionId)
|
|
||||||
buffer.putInt(fetchResponse.correlationId)
|
buffer.putInt(fetchResponse.correlationId)
|
||||||
buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count
|
buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count
|
||||||
buffer.rewind()
|
buffer.rewind()
|
||||||
|
|
|
@ -79,7 +79,7 @@ case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndContr
|
||||||
}
|
}
|
||||||
|
|
||||||
object LeaderAndIsrRequest {
|
object LeaderAndIsrRequest {
|
||||||
val CurrentVersion = 1.shortValue()
|
val CurrentVersion = 0.shortValue
|
||||||
val DefaultClientId = ""
|
val DefaultClientId = ""
|
||||||
val IsInit: Boolean = true
|
val IsInit: Boolean = true
|
||||||
val NotInit: Boolean = false
|
val NotInit: Boolean = false
|
||||||
|
@ -87,6 +87,7 @@ object LeaderAndIsrRequest {
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): LeaderAndIsrRequest = {
|
def readFrom(buffer: ByteBuffer): LeaderAndIsrRequest = {
|
||||||
val versionId = buffer.getShort
|
val versionId = buffer.getShort
|
||||||
|
val correlationId = buffer.getInt
|
||||||
val clientId = readShortString(buffer)
|
val clientId = readShortString(buffer)
|
||||||
val ackTimeoutMs = buffer.getInt
|
val ackTimeoutMs = buffer.getInt
|
||||||
val controllerEpoch = buffer.getInt
|
val controllerEpoch = buffer.getInt
|
||||||
|
@ -106,11 +107,12 @@ object LeaderAndIsrRequest {
|
||||||
for (i <- 0 until leadersCount)
|
for (i <- 0 until leadersCount)
|
||||||
leaders += Broker.readFrom(buffer)
|
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,
|
case class LeaderAndIsrRequest (versionId: Short,
|
||||||
|
correlationId: Int,
|
||||||
clientId: String,
|
clientId: String,
|
||||||
ackTimeoutMs: Int,
|
ackTimeoutMs: Int,
|
||||||
partitionStateInfos: Map[(String, Int), PartitionStateInfo],
|
partitionStateInfos: Map[(String, Int), PartitionStateInfo],
|
||||||
|
@ -119,12 +121,13 @@ case class LeaderAndIsrRequest (versionId: Short,
|
||||||
extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) {
|
extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) {
|
||||||
|
|
||||||
def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], liveBrokers: Set[Broker], controllerEpoch: Int) = {
|
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)
|
partitionStateInfos, liveBrokers, controllerEpoch)
|
||||||
}
|
}
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putShort(versionId)
|
buffer.putShort(versionId)
|
||||||
|
buffer.putInt(correlationId)
|
||||||
writeShortString(buffer, clientId)
|
writeShortString(buffer, clientId)
|
||||||
buffer.putInt(ackTimeoutMs)
|
buffer.putInt(ackTimeoutMs)
|
||||||
buffer.putInt(controllerEpoch)
|
buffer.putInt(controllerEpoch)
|
||||||
|
@ -141,6 +144,7 @@ case class LeaderAndIsrRequest (versionId: Short,
|
||||||
def sizeInBytes(): Int = {
|
def sizeInBytes(): Int = {
|
||||||
var size =
|
var size =
|
||||||
2 /* version id */ +
|
2 /* version id */ +
|
||||||
|
4 /* correlation id */ +
|
||||||
(2 + clientId.length) /* client id */ +
|
(2 + clientId.length) /* client id */ +
|
||||||
4 /* ack timeout */ +
|
4 /* ack timeout */ +
|
||||||
4 /* controller epoch */ +
|
4 /* controller epoch */ +
|
||||||
|
|
|
@ -26,7 +26,7 @@ import collection.Map
|
||||||
|
|
||||||
object LeaderAndIsrResponse {
|
object LeaderAndIsrResponse {
|
||||||
def readFrom(buffer: ByteBuffer): LeaderAndIsrResponse = {
|
def readFrom(buffer: ByteBuffer): LeaderAndIsrResponse = {
|
||||||
val versionId = buffer.getShort
|
val correlationId = buffer.getInt
|
||||||
val errorCode = buffer.getShort
|
val errorCode = buffer.getShort
|
||||||
val numEntries = buffer.getInt
|
val numEntries = buffer.getInt
|
||||||
val responseMap = new HashMap[(String, Int), Short]()
|
val responseMap = new HashMap[(String, Int), Short]()
|
||||||
|
@ -36,18 +36,18 @@ object LeaderAndIsrResponse {
|
||||||
val partitionErrorCode = buffer.getShort
|
val partitionErrorCode = buffer.getShort
|
||||||
responseMap.put((topic, partition), partitionErrorCode)
|
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],
|
responseMap: Map[(String, Int), Short],
|
||||||
errorCode: Short = ErrorMapping.NoError)
|
errorCode: Short = ErrorMapping.NoError)
|
||||||
extends RequestOrResponse {
|
extends RequestOrResponse {
|
||||||
def sizeInBytes(): Int ={
|
def sizeInBytes(): Int ={
|
||||||
var size =
|
var size =
|
||||||
2 /* version id */ +
|
4 /* correlation id */ +
|
||||||
2 /* error code */ +
|
2 /* error code */ +
|
||||||
4 /* number of responses */
|
4 /* number of responses */
|
||||||
for ((key, value) <- responseMap) {
|
for ((key, value) <- responseMap) {
|
||||||
|
@ -60,7 +60,7 @@ case class LeaderAndIsrResponse(versionId: Short,
|
||||||
}
|
}
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putShort(versionId)
|
buffer.putInt(correlationId)
|
||||||
buffer.putShort(errorCode)
|
buffer.putShort(errorCode)
|
||||||
buffer.putInt(responseMap.size)
|
buffer.putInt(responseMap.size)
|
||||||
for ((key:(String, Int), value) <- responseMap){
|
for ((key:(String, Int), value) <- responseMap){
|
||||||
|
|
|
@ -23,7 +23,7 @@ import kafka.api.ApiUtils._
|
||||||
|
|
||||||
|
|
||||||
object OffsetRequest {
|
object OffsetRequest {
|
||||||
val CurrentVersion = 1.shortValue()
|
val CurrentVersion = 0.shortValue
|
||||||
val DefaultClientId = ""
|
val DefaultClientId = ""
|
||||||
|
|
||||||
val SmallestTimeString = "smallest"
|
val SmallestTimeString = "smallest"
|
||||||
|
@ -33,6 +33,7 @@ object OffsetRequest {
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): OffsetRequest = {
|
def readFrom(buffer: ByteBuffer): OffsetRequest = {
|
||||||
val versionId = buffer.getShort
|
val versionId = buffer.getShort
|
||||||
|
val correlationId = buffer.getInt
|
||||||
val clientId = readShortString(buffer)
|
val clientId = readShortString(buffer)
|
||||||
val replicaId = buffer.getInt
|
val replicaId = buffer.getInt
|
||||||
val topicCount = buffer.getInt
|
val topicCount = buffer.getInt
|
||||||
|
@ -54,16 +55,18 @@ case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int)
|
||||||
|
|
||||||
case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo],
|
case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo],
|
||||||
versionId: Short = OffsetRequest.CurrentVersion,
|
versionId: Short = OffsetRequest.CurrentVersion,
|
||||||
|
correlationId: Int = 0,
|
||||||
clientId: String = OffsetRequest.DefaultClientId,
|
clientId: String = OffsetRequest.DefaultClientId,
|
||||||
replicaId: Int = Request.OrdinaryConsumerId)
|
replicaId: Int = Request.OrdinaryConsumerId)
|
||||||
extends RequestOrResponse(Some(RequestKeys.OffsetsKey)) {
|
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)
|
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putShort(versionId)
|
buffer.putShort(versionId)
|
||||||
|
buffer.putInt(correlationId)
|
||||||
writeShortString(buffer, clientId)
|
writeShortString(buffer, clientId)
|
||||||
buffer.putInt(replicaId)
|
buffer.putInt(replicaId)
|
||||||
|
|
||||||
|
@ -83,6 +86,7 @@ case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequ
|
||||||
|
|
||||||
def sizeInBytes =
|
def sizeInBytes =
|
||||||
2 + /* versionId */
|
2 + /* versionId */
|
||||||
|
4 + /* correlationId */
|
||||||
shortStringLength(clientId) +
|
shortStringLength(clientId) +
|
||||||
4 + /* replicaId */
|
4 + /* replicaId */
|
||||||
4 + /* topic count */
|
4 + /* topic count */
|
||||||
|
|
|
@ -25,7 +25,7 @@ import kafka.api.ApiUtils._
|
||||||
object OffsetResponse {
|
object OffsetResponse {
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): OffsetResponse = {
|
def readFrom(buffer: ByteBuffer): OffsetResponse = {
|
||||||
val versionId = buffer.getShort
|
val correlationId = buffer.getInt
|
||||||
val numTopics = buffer.getInt
|
val numTopics = buffer.getInt
|
||||||
val pairs = (1 to numTopics).flatMap(_ => {
|
val pairs = (1 to numTopics).flatMap(_ => {
|
||||||
val topic = readShortString(buffer)
|
val topic = readShortString(buffer)
|
||||||
|
@ -38,7 +38,7 @@ object OffsetResponse {
|
||||||
(TopicAndPartition(topic, partition), PartitionOffsetsResponse(error, offsets))
|
(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 PartitionOffsetsResponse(error: Short, offsets: Seq[Long])
|
||||||
|
|
||||||
|
|
||||||
case class OffsetResponse(versionId: Short,
|
case class OffsetResponse(correlationId: Int,
|
||||||
partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse])
|
partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse])
|
||||||
extends RequestOrResponse {
|
extends RequestOrResponse {
|
||||||
|
|
||||||
|
@ -56,7 +56,7 @@ case class OffsetResponse(versionId: Short,
|
||||||
def hasError = partitionErrorAndOffsets.values.exists(_.error != ErrorMapping.NoError)
|
def hasError = partitionErrorAndOffsets.values.exists(_.error != ErrorMapping.NoError)
|
||||||
|
|
||||||
val sizeInBytes = {
|
val sizeInBytes = {
|
||||||
2 + /* versionId */
|
4 + /* correlation id */
|
||||||
4 + /* topic count */
|
4 + /* topic count */
|
||||||
offsetsGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => {
|
offsetsGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => {
|
||||||
val (topic, errorAndOffsetsMap) = currTopic
|
val (topic, errorAndOffsetsMap) = currTopic
|
||||||
|
@ -74,7 +74,7 @@ case class OffsetResponse(versionId: Short,
|
||||||
}
|
}
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putShort(versionId)
|
buffer.putInt(correlationId)
|
||||||
buffer.putInt(offsetsGroupedByTopic.size) // topic count
|
buffer.putInt(offsetsGroupedByTopic.size) // topic count
|
||||||
offsetsGroupedByTopic.foreach {
|
offsetsGroupedByTopic.foreach {
|
||||||
case((topic, errorAndOffsetsMap)) =>
|
case((topic, errorAndOffsetsMap)) =>
|
||||||
|
|
|
@ -25,7 +25,7 @@ import kafka.api.ApiUtils._
|
||||||
|
|
||||||
|
|
||||||
object ProducerRequest {
|
object ProducerRequest {
|
||||||
val CurrentVersion: Short = 0
|
val CurrentVersion = 0.shortValue
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): ProducerRequest = {
|
def readFrom(buffer: ByteBuffer): ProducerRequest = {
|
||||||
val versionId: Short = buffer.getShort
|
val versionId: Short = buffer.getShort
|
||||||
|
@ -57,7 +57,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
|
||||||
clientId: String,
|
clientId: String,
|
||||||
requiredAcks: Short,
|
requiredAcks: Short,
|
||||||
ackTimeoutMs: Int,
|
ackTimeoutMs: Int,
|
||||||
data: Map[TopicAndPartition, MessageSet])
|
data: Map[TopicAndPartition, ByteBufferMessageSet])
|
||||||
extends RequestOrResponse(Some(RequestKeys.ProduceKey)) {
|
extends RequestOrResponse(Some(RequestKeys.ProduceKey)) {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -69,7 +69,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
|
||||||
clientId: String,
|
clientId: String,
|
||||||
requiredAcks: Short,
|
requiredAcks: Short,
|
||||||
ackTimeoutMs: Int,
|
ackTimeoutMs: Int,
|
||||||
data: Map[TopicAndPartition, MessageSet]) =
|
data: Map[TopicAndPartition, ByteBufferMessageSet]) =
|
||||||
this(ProducerRequest.CurrentVersion, correlationId, clientId, requiredAcks, ackTimeoutMs, data)
|
this(ProducerRequest.CurrentVersion, correlationId, clientId, requiredAcks, ackTimeoutMs, data)
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
|
@ -88,7 +88,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
|
||||||
topicAndPartitionData.foreach(partitionAndData => {
|
topicAndPartitionData.foreach(partitionAndData => {
|
||||||
val partition = partitionAndData._1.partition
|
val partition = partitionAndData._1.partition
|
||||||
val partitionMessageData = partitionAndData._2
|
val partitionMessageData = partitionAndData._2
|
||||||
val bytes = partitionMessageData.asInstanceOf[ByteBufferMessageSet].buffer
|
val bytes = partitionMessageData.buffer
|
||||||
buffer.putInt(partition)
|
buffer.putInt(partition)
|
||||||
buffer.putInt(bytes.limit)
|
buffer.putInt(bytes.limit)
|
||||||
buffer.put(bytes)
|
buffer.put(bytes)
|
||||||
|
|
|
@ -25,7 +25,6 @@ import kafka.api.ApiUtils._
|
||||||
|
|
||||||
object ProducerResponse {
|
object ProducerResponse {
|
||||||
def readFrom(buffer: ByteBuffer): ProducerResponse = {
|
def readFrom(buffer: ByteBuffer): ProducerResponse = {
|
||||||
val versionId = buffer.getShort
|
|
||||||
val correlationId = buffer.getInt
|
val correlationId = buffer.getInt
|
||||||
val topicCount = buffer.getInt
|
val topicCount = buffer.getInt
|
||||||
val statusPairs = (1 to topicCount).flatMap(_ => {
|
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 ProducerResponseStatus(error: Short, offset: Long)
|
||||||
|
|
||||||
|
|
||||||
case class ProducerResponse(versionId: Short,
|
case class ProducerResponse(correlationId: Int,
|
||||||
correlationId: Int,
|
|
||||||
status: Map[TopicAndPartition, ProducerResponseStatus]) extends RequestOrResponse {
|
status: Map[TopicAndPartition, ProducerResponseStatus]) extends RequestOrResponse {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -59,7 +57,6 @@ case class ProducerResponse(versionId: Short,
|
||||||
|
|
||||||
val sizeInBytes = {
|
val sizeInBytes = {
|
||||||
val groupedStatus = statusGroupedByTopic
|
val groupedStatus = statusGroupedByTopic
|
||||||
2 + /* version id */
|
|
||||||
4 + /* correlation id */
|
4 + /* correlation id */
|
||||||
4 + /* topic count */
|
4 + /* topic count */
|
||||||
groupedStatus.foldLeft (0) ((foldedTopics, currTopic) => {
|
groupedStatus.foldLeft (0) ((foldedTopics, currTopic) => {
|
||||||
|
@ -76,8 +73,6 @@ case class ProducerResponse(versionId: Short,
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
val groupedStatus = statusGroupedByTopic
|
val groupedStatus = statusGroupedByTopic
|
||||||
|
|
||||||
buffer.putShort(versionId)
|
|
||||||
buffer.putInt(correlationId)
|
buffer.putInt(correlationId)
|
||||||
buffer.putInt(groupedStatus.size) // topic count
|
buffer.putInt(groupedStatus.size) // topic count
|
||||||
|
|
||||||
|
|
|
@ -25,12 +25,13 @@ import kafka.network.InvalidRequestException
|
||||||
|
|
||||||
|
|
||||||
object StopReplicaRequest extends Logging {
|
object StopReplicaRequest extends Logging {
|
||||||
val CurrentVersion = 1.shortValue()
|
val CurrentVersion = 0.shortValue
|
||||||
val DefaultClientId = ""
|
val DefaultClientId = ""
|
||||||
val DefaultAckTimeout = 100
|
val DefaultAckTimeout = 100
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): StopReplicaRequest = {
|
def readFrom(buffer: ByteBuffer): StopReplicaRequest = {
|
||||||
val versionId = buffer.getShort
|
val versionId = buffer.getShort
|
||||||
|
val correlationId = buffer.getInt
|
||||||
val clientId = readShortString(buffer)
|
val clientId = readShortString(buffer)
|
||||||
val ackTimeoutMs = buffer.getInt
|
val ackTimeoutMs = buffer.getInt
|
||||||
val controllerEpoch = buffer.getInt
|
val controllerEpoch = buffer.getInt
|
||||||
|
@ -45,11 +46,12 @@ object StopReplicaRequest extends Logging {
|
||||||
(1 to topicPartitionPairCount) foreach { _ =>
|
(1 to topicPartitionPairCount) foreach { _ =>
|
||||||
topicPartitionPairSet.add(readShortString(buffer), buffer.getInt)
|
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,
|
case class StopReplicaRequest(versionId: Short,
|
||||||
|
correlationId: Int,
|
||||||
clientId: String,
|
clientId: String,
|
||||||
ackTimeoutMs: Int,
|
ackTimeoutMs: Int,
|
||||||
deletePartitions: Boolean,
|
deletePartitions: Boolean,
|
||||||
|
@ -58,12 +60,13 @@ case class StopReplicaRequest(versionId: Short,
|
||||||
extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) {
|
extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) {
|
||||||
|
|
||||||
def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerEpoch: Int) = {
|
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)
|
deletePartitions, partitions, controllerEpoch)
|
||||||
}
|
}
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putShort(versionId)
|
buffer.putShort(versionId)
|
||||||
|
buffer.putInt(correlationId)
|
||||||
writeShortString(buffer, clientId)
|
writeShortString(buffer, clientId)
|
||||||
buffer.putInt(ackTimeoutMs)
|
buffer.putInt(ackTimeoutMs)
|
||||||
buffer.putInt(controllerEpoch)
|
buffer.putInt(controllerEpoch)
|
||||||
|
@ -78,6 +81,7 @@ case class StopReplicaRequest(versionId: Short,
|
||||||
def sizeInBytes(): Int = {
|
def sizeInBytes(): Int = {
|
||||||
var size =
|
var size =
|
||||||
2 + /* versionId */
|
2 + /* versionId */
|
||||||
|
4 + /* correlation id */
|
||||||
ApiUtils.shortStringLength(clientId) +
|
ApiUtils.shortStringLength(clientId) +
|
||||||
4 + /* ackTimeoutMs */
|
4 + /* ackTimeoutMs */
|
||||||
4 + /* controller epoch */
|
4 + /* controller epoch */
|
||||||
|
|
|
@ -26,7 +26,7 @@ import kafka.api.ApiUtils._
|
||||||
|
|
||||||
object StopReplicaResponse {
|
object StopReplicaResponse {
|
||||||
def readFrom(buffer: ByteBuffer): StopReplicaResponse = {
|
def readFrom(buffer: ByteBuffer): StopReplicaResponse = {
|
||||||
val versionId = buffer.getShort
|
val correlationId = buffer.getInt
|
||||||
val errorCode = buffer.getShort
|
val errorCode = buffer.getShort
|
||||||
val numEntries = buffer.getInt
|
val numEntries = buffer.getInt
|
||||||
|
|
||||||
|
@ -37,17 +37,17 @@ object StopReplicaResponse {
|
||||||
val partitionErrorCode = buffer.getShort()
|
val partitionErrorCode = buffer.getShort()
|
||||||
responseMap.put((topic, partition), partitionErrorCode)
|
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 responseMap: Map[(String, Int), Short],
|
||||||
val errorCode: Short = ErrorMapping.NoError) extends RequestOrResponse{
|
val errorCode: Short = ErrorMapping.NoError) extends RequestOrResponse{
|
||||||
def sizeInBytes(): Int ={
|
def sizeInBytes(): Int ={
|
||||||
var size =
|
var size =
|
||||||
2 /* version id */ +
|
4 /* correlation id */ +
|
||||||
2 /* error code */ +
|
2 /* error code */ +
|
||||||
4 /* number of responses */
|
4 /* number of responses */
|
||||||
for ((key, value) <- responseMap) {
|
for ((key, value) <- responseMap) {
|
||||||
|
@ -60,7 +60,7 @@ case class StopReplicaResponse(val versionId: Short,
|
||||||
}
|
}
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putShort(versionId)
|
buffer.putInt(correlationId)
|
||||||
buffer.putShort(errorCode)
|
buffer.putShort(errorCode)
|
||||||
buffer.putInt(responseMap.size)
|
buffer.putInt(responseMap.size)
|
||||||
for ((key:(String, Int), value) <- responseMap){
|
for ((key:(String, Int), value) <- responseMap){
|
||||||
|
|
|
@ -21,57 +21,29 @@ import kafka.cluster.Broker
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import kafka.api.ApiUtils._
|
import kafka.api.ApiUtils._
|
||||||
import kafka.utils.Logging
|
import kafka.utils.Logging
|
||||||
import collection.mutable.ListBuffer
|
import collection.mutable.ArrayBuffer
|
||||||
import kafka.common.{KafkaException, ErrorMapping}
|
import kafka.common._
|
||||||
|
|
||||||
/**
|
|
||||||
* 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 }
|
|
||||||
|
|
||||||
object TopicMetadata {
|
object TopicMetadata {
|
||||||
|
|
||||||
|
val NoLeaderNodeId = -1
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): TopicMetadata = {
|
def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): TopicMetadata = {
|
||||||
val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue))
|
val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue))
|
||||||
val topic = readShortString(buffer)
|
val topic = readShortString(buffer)
|
||||||
val numPartitions = readIntInRange(buffer, "number of partitions", (0, Int.MaxValue))
|
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)
|
for(i <- 0 until numPartitions)
|
||||||
partitionsMetadata += PartitionMetadata.readFrom(buffer)
|
partitionsMetadata += PartitionMetadata.readFrom(buffer, brokers)
|
||||||
new TopicMetadata(topic, partitionsMetadata, errorCode)
|
new TopicMetadata(topic, partitionsMetadata, errorCode)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError) extends Logging {
|
case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError) extends Logging {
|
||||||
def sizeInBytes: Int = {
|
def sizeInBytes: Int = {
|
||||||
var size: Int = 2 /* error code */
|
2 /* error code */ +
|
||||||
size += shortStringLength(topic)
|
shortStringLength(topic) +
|
||||||
size += partitionsMetadata.foldLeft(4 /* number of partitions */)(_ + _.sizeInBytes)
|
4 + partitionsMetadata.map(_.sizeInBytes).sum /* size and partition data array */
|
||||||
debug("Size of topic metadata = " + size)
|
|
||||||
size
|
|
||||||
}
|
}
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
|
@ -87,40 +59,24 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat
|
||||||
|
|
||||||
object PartitionMetadata {
|
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 errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue))
|
||||||
val partitionId = readIntInRange(buffer, "partition id", (0, Int.MaxValue)) /* partition id */
|
val partitionId = readIntInRange(buffer, "partition id", (0, Int.MaxValue)) /* partition id */
|
||||||
val doesLeaderExist = getLeaderRequest(buffer.get)
|
val leaderId = buffer.getInt
|
||||||
val leader = doesLeaderExist match {
|
val leader = brokers.get(leaderId)
|
||||||
case LeaderExists => /* leader exists */
|
|
||||||
Some(Broker.readFrom(buffer))
|
|
||||||
case LeaderDoesNotExist => None
|
|
||||||
}
|
|
||||||
|
|
||||||
/* list of all replicas */
|
/* list of all replicas */
|
||||||
val numReplicas = readShortInRange(buffer, "number of all replicas", (0, Short.MaxValue))
|
val numReplicas = readIntInRange(buffer, "number of all replicas", (0, Int.MaxValue))
|
||||||
val replicas = new Array[Broker](numReplicas)
|
val replicaIds = (0 until numReplicas).map(_ => buffer.getInt)
|
||||||
for(i <- 0 until numReplicas) {
|
val replicas = replicaIds.map(brokers)
|
||||||
replicas(i) = Broker.readFrom(buffer)
|
|
||||||
}
|
|
||||||
|
|
||||||
/* list of in-sync replicas */
|
/* list of in-sync replicas */
|
||||||
val numIsr = readShortInRange(buffer, "number of in-sync replicas", (0, Short.MaxValue))
|
val numIsr = readIntInRange(buffer, "number of in-sync replicas", (0, Int.MaxValue))
|
||||||
val isr = new Array[Broker](numIsr)
|
val isrIds = (0 until numIsr).map(_ => buffer.getInt)
|
||||||
for(i <- 0 until numIsr) {
|
val isr = isrIds.map(brokers)
|
||||||
isr(i) = Broker.readFrom(buffer)
|
|
||||||
}
|
|
||||||
|
|
||||||
new PartitionMetadata(partitionId, leader, replicas, isr, errorCode)
|
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,
|
case class PartitionMetadata(partitionId: Int,
|
||||||
|
@ -129,42 +85,28 @@ case class PartitionMetadata(partitionId: Int,
|
||||||
isr: Seq[Broker] = Seq.empty,
|
isr: Seq[Broker] = Seq.empty,
|
||||||
errorCode: Short = ErrorMapping.NoError) extends Logging {
|
errorCode: Short = ErrorMapping.NoError) extends Logging {
|
||||||
def sizeInBytes: Int = {
|
def sizeInBytes: Int = {
|
||||||
var size: Int = 2 /* error code */ + 4 /* partition id */ + 1 /* if leader exists*/
|
2 /* error code */ +
|
||||||
|
4 /* partition id */ +
|
||||||
leader match {
|
4 /* leader */ +
|
||||||
case Some(l) => size += l.sizeInBytes
|
4 + 4 * replicas.size /* replica array */ +
|
||||||
case None =>
|
4 + 4 * isr.size /* isr array */
|
||||||
}
|
|
||||||
|
|
||||||
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
|
|
||||||
}
|
}
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putShort(errorCode)
|
buffer.putShort(errorCode)
|
||||||
buffer.putInt(partitionId)
|
buffer.putInt(partitionId)
|
||||||
|
|
||||||
/* if leader exists*/
|
/* leader */
|
||||||
leader match {
|
val leaderId = if(leader.isDefined) leader.get.id else TopicMetadata.NoLeaderNodeId
|
||||||
case Some(l) =>
|
buffer.putInt(leaderId)
|
||||||
buffer.put(LeaderExists.requestId)
|
|
||||||
/* leader id host_name port */
|
|
||||||
l.writeTo(buffer)
|
|
||||||
case None => buffer.put(LeaderDoesNotExist.requestId)
|
|
||||||
}
|
|
||||||
|
|
||||||
/* number of replicas */
|
/* number of replicas */
|
||||||
buffer.putShort(replicas.size.toShort)
|
buffer.putInt(replicas.size)
|
||||||
replicas.foreach(r => r.writeTo(buffer))
|
replicas.foreach(r => buffer.putInt(r.id))
|
||||||
|
|
||||||
/* number of in-sync replicas */
|
/* number of in-sync replicas */
|
||||||
buffer.putShort(isr.size.toShort)
|
buffer.putInt(isr.size)
|
||||||
isr.foreach(r => r.writeTo(buffer))
|
isr.foreach(r => buffer.putInt(r.id))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -23,7 +23,7 @@ import collection.mutable.ListBuffer
|
||||||
import kafka.utils.Logging
|
import kafka.utils.Logging
|
||||||
|
|
||||||
object TopicMetadataRequest extends Logging {
|
object TopicMetadataRequest extends Logging {
|
||||||
val CurrentVersion = 1.shortValue()
|
val CurrentVersion = 0.shortValue
|
||||||
val DefaultClientId = ""
|
val DefaultClientId = ""
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -33,6 +33,7 @@ object TopicMetadataRequest extends Logging {
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): TopicMetadataRequest = {
|
def readFrom(buffer: ByteBuffer): TopicMetadataRequest = {
|
||||||
val versionId = buffer.getShort
|
val versionId = buffer.getShort
|
||||||
|
val correlationId = buffer.getInt
|
||||||
val clientId = readShortString(buffer)
|
val clientId = readShortString(buffer)
|
||||||
val numTopics = readIntInRange(buffer, "number of topics", (0, Int.MaxValue))
|
val numTopics = readIntInRange(buffer, "number of topics", (0, Int.MaxValue))
|
||||||
val topics = new ListBuffer[String]()
|
val topics = new ListBuffer[String]()
|
||||||
|
@ -40,26 +41,28 @@ object TopicMetadataRequest extends Logging {
|
||||||
topics += readShortString(buffer)
|
topics += readShortString(buffer)
|
||||||
val topicsList = topics.toList
|
val topicsList = topics.toList
|
||||||
debug("topic = %s".format(topicsList.head))
|
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,
|
case class TopicMetadataRequest(val versionId: Short,
|
||||||
val clientId: String,
|
val clientId: String,
|
||||||
val topics: Seq[String])
|
val topics: Seq[String],
|
||||||
|
val correlationId: Int)
|
||||||
extends RequestOrResponse(Some(RequestKeys.MetadataKey)){
|
extends RequestOrResponse(Some(RequestKeys.MetadataKey)){
|
||||||
|
|
||||||
def this(topics: Seq[String]) =
|
def this(topics: Seq[String]) =
|
||||||
this(TopicMetadataRequest.CurrentVersion, TopicMetadataRequest.DefaultClientId, topics)
|
this(TopicMetadataRequest.CurrentVersion, TopicMetadataRequest.DefaultClientId, topics, 0)
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putShort(versionId)
|
buffer.putShort(versionId)
|
||||||
|
buffer.putInt(correlationId) // correlation id not set yet
|
||||||
writeShortString(buffer, clientId)
|
writeShortString(buffer, clientId)
|
||||||
buffer.putInt(topics.size)
|
buffer.putInt(topics.size)
|
||||||
topics.foreach(topic => writeShortString(buffer, topic))
|
topics.foreach(topic => writeShortString(buffer, topic))
|
||||||
}
|
}
|
||||||
|
|
||||||
def sizeInBytes(): Int = {
|
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 */
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,30 +17,43 @@
|
||||||
|
|
||||||
package kafka.api
|
package kafka.api
|
||||||
|
|
||||||
|
import kafka.cluster.Broker
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
|
|
||||||
object TopicMetadataResponse {
|
object TopicMetadataResponse {
|
||||||
|
|
||||||
def readFrom(buffer: ByteBuffer): 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 topicCount = buffer.getInt
|
||||||
val topicsMetadata = new Array[TopicMetadata](topicCount)
|
val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap))
|
||||||
for( i <- 0 until topicCount) {
|
new TopicMetadataResponse(topicsMetadata, correlationId)
|
||||||
topicsMetadata(i) = TopicMetadata.readFrom(buffer)
|
|
||||||
}
|
|
||||||
new TopicMetadataResponse(versionId, topicsMetadata.toSeq)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
case class TopicMetadataResponse(versionId: Short,
|
case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata],
|
||||||
topicsMetadata: Seq[TopicMetadata]) extends RequestOrResponse
|
correlationId: Int) extends RequestOrResponse {
|
||||||
{
|
val sizeInBytes: Int = {
|
||||||
val sizeInBytes = 2 + topicsMetadata.foldLeft(4)(_ + _.sizeInBytes)
|
val brokers = extractBrokers(topicsMetadata).values
|
||||||
|
4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum
|
||||||
|
}
|
||||||
|
|
||||||
def writeTo(buffer: ByteBuffer) {
|
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 */
|
/* topic metadata */
|
||||||
buffer.putInt(topicsMetadata.length)
|
buffer.putInt(topicsMetadata.length)
|
||||||
topicsMetadata.foreach(_.writeTo(buffer))
|
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
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -6,20 +6,28 @@ import kafka.api._
|
||||||
import kafka.producer._
|
import kafka.producer._
|
||||||
import kafka.common.KafkaException
|
import kafka.common.KafkaException
|
||||||
import kafka.utils.{Utils, Logging}
|
import kafka.utils.{Utils, Logging}
|
||||||
|
import java.util.Properties
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helper functions common to clients (producer, consumer, or admin)
|
* Helper functions common to clients (producer, consumer, or admin)
|
||||||
*/
|
*/
|
||||||
object ClientUtils extends Logging{
|
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 fetchMetaDataSucceeded: Boolean = false
|
||||||
var i: Int = 0
|
var i: Int = 0
|
||||||
val topicMetadataRequest = new TopicMetadataRequest(topics.toSeq)
|
val topicMetadataRequest = new TopicMetadataRequest(topics.toSeq)
|
||||||
var topicMetadataResponse: TopicMetadataResponse = null
|
var topicMetadataResponse: TopicMetadataResponse = null
|
||||||
var t: Throwable = null
|
var t: Throwable = null
|
||||||
while(i < brokers.size && !fetchMetaDataSucceeded) {
|
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))
|
info("Fetching metadata for topic %s".format(topics))
|
||||||
try {
|
try {
|
||||||
topicMetadataResponse = producer.send(topicMetadataRequest)
|
topicMetadataResponse = producer.send(topicMetadataRequest)
|
||||||
|
@ -39,7 +47,22 @@ object ClientUtils extends Logging{
|
||||||
}
|
}
|
||||||
return topicMetadataResponse
|
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, ...
|
* 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 brokerInfos = brokerStr.split(":")
|
||||||
val hostName = brokerInfos(0)
|
val hostName = brokerInfos(0)
|
||||||
val port = brokerInfos(1).toInt
|
val port = brokerInfos(1).toInt
|
||||||
val creatorId = hostName + "-" + System.currentTimeMillis()
|
new Broker(brokerId, hostName, port)
|
||||||
new Broker(brokerId, creatorId, hostName, port)
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -31,36 +31,32 @@ private[kafka] object Broker {
|
||||||
if(brokerInfoString == null)
|
if(brokerInfoString == null)
|
||||||
throw new BrokerNotAvailableException("Broker id %s does not exist".format(id))
|
throw new BrokerNotAvailableException("Broker id %s does not exist".format(id))
|
||||||
val brokerInfo = brokerInfoString.split(":")
|
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 = {
|
def readFrom(buffer: ByteBuffer): Broker = {
|
||||||
val id = buffer.getInt
|
val id = buffer.getInt
|
||||||
val creatorId = readShortString(buffer)
|
|
||||||
val host = readShortString(buffer)
|
val host = readShortString(buffer)
|
||||||
val port = buffer.getInt
|
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) {
|
def writeTo(buffer: ByteBuffer) {
|
||||||
buffer.putInt(id)
|
buffer.putInt(id)
|
||||||
writeShortString(buffer, creatorId)
|
|
||||||
writeShortString(buffer, host)
|
writeShortString(buffer, host)
|
||||||
buffer.putInt(port)
|
buffer.putInt(port)
|
||||||
}
|
}
|
||||||
|
|
||||||
def sizeInBytes: Int = {
|
def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/
|
||||||
val size = shortStringLength(creatorId) + shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/
|
|
||||||
debug("Size of broker info = " + size)
|
|
||||||
size
|
|
||||||
}
|
|
||||||
|
|
||||||
override def equals(obj: Any): Boolean = {
|
override def equals(obj: Any): Boolean = {
|
||||||
obj match {
|
obj match {
|
||||||
|
|
|
@ -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)
|
||||||
|
}
|
|
@ -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)
|
||||||
|
}
|
||||||
|
|
|
@ -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 -")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -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 -")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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)
|
||||||
|
}
|
|
@ -89,7 +89,7 @@ object ConsoleConsumer extends Logging {
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("class")
|
.describedAs("class")
|
||||||
.ofType(classOf[String])
|
.ofType(classOf[String])
|
||||||
.defaultsTo(classOf[NewlineMessageFormatter].getName)
|
.defaultsTo(classOf[DefaultMessageFormatter].getName)
|
||||||
val messageFormatterArgOpt = parser.accepts("property")
|
val messageFormatterArgOpt = parser.accepts("property")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("prop")
|
.describedAs("prop")
|
||||||
|
@ -176,6 +176,7 @@ object ConsoleConsumer extends Logging {
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
|
var numMessages = 0L
|
||||||
val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter]
|
val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter]
|
||||||
formatter.init(formatterArgs)
|
formatter.init(formatterArgs)
|
||||||
try {
|
try {
|
||||||
|
@ -188,6 +189,7 @@ object ConsoleConsumer extends Logging {
|
||||||
for(messageAndTopic <- iter) {
|
for(messageAndTopic <- iter) {
|
||||||
try {
|
try {
|
||||||
formatter.writeTo(messageAndTopic.key, messageAndTopic.message, System.out)
|
formatter.writeTo(messageAndTopic.key, messageAndTopic.message, System.out)
|
||||||
|
numMessages += 1
|
||||||
} catch {
|
} catch {
|
||||||
case e =>
|
case e =>
|
||||||
if (skipMessageOnError)
|
if (skipMessageOnError)
|
||||||
|
@ -198,6 +200,7 @@ object ConsoleConsumer extends Logging {
|
||||||
if(System.out.checkError()) {
|
if(System.out.checkError()) {
|
||||||
// This means no one is listening to our output stream any more, time to shutdown
|
// 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("Unable to write to standard out, closing consumer.")
|
||||||
|
System.err.println("Consumed %d messages".format(numMessages))
|
||||||
formatter.close()
|
formatter.close()
|
||||||
connector.shutdown()
|
connector.shutdown()
|
||||||
System.exit(1)
|
System.exit(1)
|
||||||
|
@ -206,6 +209,7 @@ object ConsoleConsumer extends Logging {
|
||||||
} catch {
|
} catch {
|
||||||
case e => error("Error processing message, stopping consumer: ", e)
|
case e => error("Error processing message, stopping consumer: ", e)
|
||||||
}
|
}
|
||||||
|
System.out.println("Consumed %d messages".format(numMessages))
|
||||||
System.out.flush()
|
System.out.flush()
|
||||||
formatter.close()
|
formatter.close()
|
||||||
connector.shutdown()
|
connector.shutdown()
|
||||||
|
@ -256,10 +260,27 @@ trait MessageFormatter {
|
||||||
def close() {}
|
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) {
|
def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {
|
||||||
|
if(printKey) {
|
||||||
|
output.write(key)
|
||||||
|
output.write(keySeparator)
|
||||||
|
}
|
||||||
output.write(value)
|
output.write(value)
|
||||||
output.write('\n')
|
output.write(lineSeparator)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,16 +19,17 @@ package kafka.consumer
|
||||||
|
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import kafka.api.OffsetRequest
|
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 SocketTimeout = 30 * 1000
|
||||||
val SocketBufferSize = 64*1024
|
val SocketBufferSize = 64*1024
|
||||||
val FetchSize = 1024 * 1024
|
val FetchSize = 1024 * 1024
|
||||||
val MaxFetchSize = 10*FetchSize
|
val MaxFetchSize = 10*FetchSize
|
||||||
val DefaultFetcherBackoffMs = 1000
|
val DefaultFetcherBackoffMs = 1000
|
||||||
val AutoCommit = true
|
val AutoCommit = true
|
||||||
val AutoCommitInterval = 10 * 1000
|
val AutoCommitInterval = 60 * 1000
|
||||||
val MaxQueuedChunks = 10
|
val MaxQueuedChunks = 10
|
||||||
val MaxRebalanceRetries = 4
|
val MaxRebalanceRetries = 4
|
||||||
val AutoOffsetReset = OffsetRequest.SmallestTimeString
|
val AutoOffsetReset = OffsetRequest.SmallestTimeString
|
||||||
|
@ -43,6 +44,28 @@ object ConsumerConfig {
|
||||||
val MirrorTopicsBlacklistProp = "mirror.topics.blacklist"
|
val MirrorTopicsBlacklistProp = "mirror.topics.blacklist"
|
||||||
val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads"
|
val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads"
|
||||||
val DefaultClientId = ""
|
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) {
|
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)
|
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)
|
val clientId = props.getString("clientid", groupId)
|
||||||
|
|
||||||
|
validate(this)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -54,7 +54,7 @@ class ConsumerFetcherManager(private val consumerIdString: String,
|
||||||
try {
|
try {
|
||||||
trace("Partitions without leader %s".format(noLeaderPartitionSet))
|
trace("Partitions without leader %s".format(noLeaderPartitionSet))
|
||||||
val brokers = getAllBrokersInCluster(zkClient)
|
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]
|
val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker]
|
||||||
topicsMetadata.foreach(
|
topicsMetadata.foreach(
|
||||||
tmd => {
|
tmd => {
|
||||||
|
|
|
@ -34,12 +34,14 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk
|
||||||
consumerTimeoutMs: Int,
|
consumerTimeoutMs: Int,
|
||||||
private val keyDecoder: Decoder[K],
|
private val keyDecoder: Decoder[K],
|
||||||
private val valueDecoder: Decoder[V],
|
private val valueDecoder: Decoder[V],
|
||||||
val enableShallowIterator: Boolean)
|
val enableShallowIterator: Boolean,
|
||||||
|
val clientId: String)
|
||||||
extends IteratorTemplate[MessageAndMetadata[K, V]] with Logging {
|
extends IteratorTemplate[MessageAndMetadata[K, V]] with Logging {
|
||||||
|
|
||||||
private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null)
|
private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null)
|
||||||
private var currentTopicInfo: PartitionTopicInfo = null
|
private var currentTopicInfo: PartitionTopicInfo = null
|
||||||
private var consumedOffset: Long = -1L
|
private var consumedOffset: Long = -1L
|
||||||
|
private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId)
|
||||||
|
|
||||||
override def next(): MessageAndMetadata[K, V] = {
|
override def next(): MessageAndMetadata[K, V] = {
|
||||||
val item = super.next()
|
val item = super.next()
|
||||||
|
@ -48,8 +50,8 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk
|
||||||
currentTopicInfo.resetConsumeOffset(consumedOffset)
|
currentTopicInfo.resetConsumeOffset(consumedOffset)
|
||||||
val topic = currentTopicInfo.topic
|
val topic = currentTopicInfo.topic
|
||||||
trace("Setting %s consumed offset to %d".format(topic, consumedOffset))
|
trace("Setting %s consumed offset to %d".format(topic, consumedOffset))
|
||||||
ConsumerTopicStat.getConsumerTopicStat(topic).messageRate.mark()
|
consumerTopicStats.getConsumerTopicStats(topic).messageRate.mark()
|
||||||
ConsumerTopicStat.getConsumerAllTopicStat().messageRate.mark()
|
consumerTopicStats.getConsumerAllTopicStats().messageRate.mark()
|
||||||
item
|
item
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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 + "-")
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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)
|
||||||
|
}
|
||||||
|
}
|
|
@ -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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
|
@ -26,11 +26,12 @@ class KafkaStream[K,V](private val queue: BlockingQueue[FetchedDataChunk],
|
||||||
consumerTimeoutMs: Int,
|
consumerTimeoutMs: Int,
|
||||||
private val keyDecoder: Decoder[K],
|
private val keyDecoder: Decoder[K],
|
||||||
private val valueDecoder: Decoder[V],
|
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]] {
|
extends Iterable[MessageAndMetadata[K,V]] with java.lang.Iterable[MessageAndMetadata[K,V]] {
|
||||||
|
|
||||||
private val iter: ConsumerIterator[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.
|
* Create an iterator over messages in the stream.
|
||||||
|
|
|
@ -28,11 +28,14 @@ class PartitionTopicInfo(val topic: String,
|
||||||
private val chunkQueue: BlockingQueue[FetchedDataChunk],
|
private val chunkQueue: BlockingQueue[FetchedDataChunk],
|
||||||
private val consumedOffset: AtomicLong,
|
private val consumedOffset: AtomicLong,
|
||||||
private val fetchedOffset: 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 consumer offset of " + this + " is " + consumedOffset.get)
|
||||||
debug("initial fetch offset of " + this + " is " + fetchedOffset.get)
|
debug("initial fetch offset of " + this + " is " + fetchedOffset.get)
|
||||||
|
|
||||||
|
private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId)
|
||||||
|
|
||||||
def getConsumeOffset() = consumedOffset.get
|
def getConsumeOffset() = consumedOffset.get
|
||||||
|
|
||||||
def getFetchOffset() = fetchedOffset.get
|
def getFetchOffset() = fetchedOffset.get
|
||||||
|
@ -58,8 +61,8 @@ class PartitionTopicInfo(val topic: String,
|
||||||
chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get))
|
chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get))
|
||||||
fetchedOffset.set(next)
|
fetchedOffset.set(next)
|
||||||
debug("updated fetch offset of (%s) to %d".format(this, next))
|
debug("updated fetch offset of (%s) to %d".format(this, next))
|
||||||
ConsumerTopicStat.getConsumerTopicStat(topic).byteRate.mark(size)
|
consumerTopicStats.getConsumerTopicStats(topic).byteRate.mark(size)
|
||||||
ConsumerTopicStat.getConsumerAllTopicStat().byteRate.mark(size)
|
consumerTopicStats.getConsumerAllTopicStats().byteRate.mark(size)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,8 +20,6 @@ package kafka.consumer
|
||||||
import kafka.api._
|
import kafka.api._
|
||||||
import kafka.network._
|
import kafka.network._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import java.util.concurrent.TimeUnit
|
|
||||||
import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
|
|
||||||
import kafka.utils.ZkUtils._
|
import kafka.utils.ZkUtils._
|
||||||
import collection.immutable
|
import collection.immutable
|
||||||
import kafka.common.{TopicAndPartition, KafkaException}
|
import kafka.common.{TopicAndPartition, KafkaException}
|
||||||
|
@ -30,19 +28,23 @@ import kafka.cluster.Broker
|
||||||
|
|
||||||
|
|
||||||
object SimpleConsumer extends Logging {
|
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 = {
|
isFromOrdinaryConsumer: Boolean): Long = {
|
||||||
var simpleConsumer: SimpleConsumer = null
|
var simpleConsumer: SimpleConsumer = null
|
||||||
var producedOffset: Long = -1L
|
var producedOffset: Long = -1L
|
||||||
try {
|
try {
|
||||||
simpleConsumer = new SimpleConsumer(broker.host, broker.port, ConsumerConfig.SocketTimeout,
|
simpleConsumer = new SimpleConsumer(broker.host, broker.port, ConsumerConfig.SocketTimeout,
|
||||||
ConsumerConfig.SocketBufferSize)
|
ConsumerConfig.SocketBufferSize, clientId)
|
||||||
val topicAndPartition = TopicAndPartition(topic, partitionId)
|
val topicAndPartition = TopicAndPartition(topic, partitionId)
|
||||||
val request = if(isFromOrdinaryConsumer)
|
val request = if(isFromOrdinaryConsumer)
|
||||||
new OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)))
|
new OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)))
|
||||||
else
|
else
|
||||||
new OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)),
|
new OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)),
|
||||||
Request.DebuggingConsumerId)
|
0, Request.DebuggingConsumerId)
|
||||||
producedOffset = simpleConsumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
|
producedOffset = simpleConsumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
|
||||||
} catch {
|
} catch {
|
||||||
case e =>
|
case e =>
|
||||||
|
@ -55,15 +57,20 @@ object SimpleConsumer extends Logging {
|
||||||
producedOffset
|
producedOffset
|
||||||
}
|
}
|
||||||
|
|
||||||
def earliestOrLatestOffset(zkClient: ZkClient, topic: String, brokerId: Int, partitionId: Int,
|
def earliestOrLatestOffset(zkClient: ZkClient,
|
||||||
earliestOrLatest: Long, isFromOrdinaryConsumer: Boolean = true): Long = {
|
topic: String,
|
||||||
|
brokerId: Int,
|
||||||
|
partitionId: Int,
|
||||||
|
earliestOrLatest: Long,
|
||||||
|
clientId: String,
|
||||||
|
isFromOrdinaryConsumer: Boolean = true): Long = {
|
||||||
val cluster = getCluster(zkClient)
|
val cluster = getCluster(zkClient)
|
||||||
val broker = cluster.getBroker(brokerId) match {
|
val broker = cluster.getBroker(brokerId) match {
|
||||||
case Some(b) => b
|
case Some(b) => b
|
||||||
case None => throw new KafkaException("Broker " + brokerId + " is unavailable. Cannot issue " +
|
case None => throw new KafkaException("Broker " + brokerId + " is unavailable. Cannot issue " +
|
||||||
"getOffsetsBefore request")
|
"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,
|
class SimpleConsumer(val host: String,
|
||||||
val port: Int,
|
val port: Int,
|
||||||
val soTimeout: 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 lock = new Object()
|
||||||
private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout)
|
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 = {
|
private def connect(): BlockingChannel = {
|
||||||
close
|
close
|
||||||
|
@ -143,12 +154,17 @@ class SimpleConsumer(val host: String,
|
||||||
*/
|
*/
|
||||||
def fetch(request: FetchRequest): FetchResponse = {
|
def fetch(request: FetchRequest): FetchResponse = {
|
||||||
var response: Receive = null
|
var response: Receive = null
|
||||||
FetchRequestAndResponseStat.requestTimer.time {
|
val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestTimer
|
||||||
response = sendRequest(request)
|
val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer
|
||||||
|
aggregateTimer.time {
|
||||||
|
specificTimer.time {
|
||||||
|
response = sendRequest(request)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
val fetchResponse = FetchResponse.readFrom(response.buffer)
|
val fetchResponse = FetchResponse.readFrom(response.buffer)
|
||||||
val fetchedSize = fetchResponse.sizeInBytes
|
val fetchedSize = fetchResponse.sizeInBytes
|
||||||
FetchRequestAndResponseStat.respondSizeHist.update(fetchedSize)
|
fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestSizeHist.update(fetchedSize)
|
||||||
|
fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize)
|
||||||
fetchResponse
|
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")
|
|
||||||
}
|
|
||||||
|
|
|
@ -20,7 +20,7 @@ package kafka.consumer
|
||||||
import java.util.concurrent._
|
import java.util.concurrent._
|
||||||
import java.util.concurrent.atomic._
|
import java.util.concurrent.atomic._
|
||||||
import locks.ReentrantLock
|
import locks.ReentrantLock
|
||||||
import scala.collection._
|
import collection._
|
||||||
import kafka.cluster._
|
import kafka.cluster._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.I0Itec.zkclient.exception.ZkNodeExistsException
|
import org.I0Itec.zkclient.exception.ZkNodeExistsException
|
||||||
|
@ -35,7 +35,7 @@ import kafka.client.ClientUtils
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
import kafka.api.OffsetRequest
|
import kafka.api.OffsetRequest
|
||||||
import kafka.metrics._
|
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,
|
private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
val enableFetcher: Boolean) // for testing only
|
val enableFetcher: Boolean) // for testing only
|
||||||
extends ConsumerConnector with Logging with KafkaMetricsGroup {
|
extends ConsumerConnector with Logging with KafkaMetricsGroup {
|
||||||
|
|
||||||
private val isShuttingDown = new AtomicBoolean(false)
|
private val isShuttingDown = new AtomicBoolean(false)
|
||||||
private val rebalanceLock = new Object
|
private val rebalanceLock = new Object
|
||||||
private var fetcher: Option[ConsumerFetcherManager] = None
|
private var fetcher: Option[ConsumerFetcherManager] = None
|
||||||
|
@ -198,7 +199,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
threadIdSet.map(_ => {
|
threadIdSet.map(_ => {
|
||||||
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks)
|
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks)
|
||||||
val stream = new KafkaStream[K,V](
|
val stream = new KafkaStream[K,V](
|
||||||
queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.enableShallowIterator)
|
queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.enableShallowIterator, config.clientId)
|
||||||
(queue, stream)
|
(queue, stream)
|
||||||
})
|
})
|
||||||
).flatten.toList
|
).flatten.toList
|
||||||
|
@ -402,7 +403,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic
|
val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic
|
||||||
val consumersPerTopicMap = getConsumersPerTopic(zkClient, group)
|
val consumersPerTopicMap = getConsumersPerTopic(zkClient, group)
|
||||||
val brokers = getAllBrokersInCluster(zkClient)
|
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 partitionsPerTopicMap = new mutable.HashMap[String, Seq[Int]]
|
||||||
val leaderIdForPartitionsMap = new mutable.HashMap[(String, Int), Int]
|
val leaderIdForPartitionsMap = new mutable.HashMap[(String, Int), Int]
|
||||||
topicsMetadata.foreach(m =>{
|
topicsMetadata.foreach(m =>{
|
||||||
|
@ -598,11 +599,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
case None =>
|
case None =>
|
||||||
config.autoOffsetReset match {
|
config.autoOffsetReset match {
|
||||||
case OffsetRequest.SmallestTimeString =>
|
case OffsetRequest.SmallestTimeString =>
|
||||||
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.EarliestTime)
|
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.EarliestTime, config.clientId)
|
||||||
case OffsetRequest.LargestTimeString =>
|
case OffsetRequest.LargestTimeString =>
|
||||||
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.LatestTime)
|
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.LatestTime, config.clientId)
|
||||||
case _ =>
|
|
||||||
throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig")
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
val queue = topicThreadIdAndQueues.get((topic, consumerThreadId))
|
val queue = topicThreadIdAndQueues.get((topic, consumerThreadId))
|
||||||
|
@ -614,7 +613,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
queue,
|
queue,
|
||||||
consumedOffset,
|
consumedOffset,
|
||||||
fetchedOffset,
|
fetchedOffset,
|
||||||
new AtomicInteger(config.fetchSize))
|
new AtomicInteger(config.fetchSize),
|
||||||
|
config.clientId)
|
||||||
partTopicInfoMap.put(partition, partTopicInfo)
|
partTopicInfoMap.put(partition, partTopicInfo)
|
||||||
debug(partTopicInfo + " selected new offset " + offset)
|
debug(partTopicInfo + " selected new offset " + offset)
|
||||||
}
|
}
|
||||||
|
@ -670,7 +670,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
val q = e._2._1
|
val q = e._2._1
|
||||||
topicThreadIdAndQueues.put(topicThreadId, q)
|
topicThreadIdAndQueues.put(topicThreadId, q)
|
||||||
newGauge(
|
newGauge(
|
||||||
config.groupId + "-" + topicThreadId._1 + "-" + topicThreadId._2 + "-FetchQueueSize",
|
config.clientId + "-" + config.groupId + "-" + topicThreadId._1 + "-" + topicThreadId._2 + "-FetchQueueSize",
|
||||||
new Gauge[Int] {
|
new Gauge[Int] {
|
||||||
def getValue = q.size
|
def getValue = q.size
|
||||||
}
|
}
|
||||||
|
@ -717,7 +717,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
config.consumerTimeoutMs,
|
config.consumerTimeoutMs,
|
||||||
keyDecoder,
|
keyDecoder,
|
||||||
valueDecoder,
|
valueDecoder,
|
||||||
config.enableShallowIterator)
|
config.enableShallowIterator,
|
||||||
|
config.clientId)
|
||||||
(queue, stream)
|
(queue, stream)
|
||||||
}).toList
|
}).toList
|
||||||
|
|
||||||
|
|
|
@ -961,7 +961,7 @@ case class PartitionAndReplica(topic: String, partition: Int, replica: Int)
|
||||||
|
|
||||||
case class LeaderIsrAndControllerEpoch(val leaderAndIsr: LeaderAndIsr, controllerEpoch: 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 offlinePartitionRate = newMeter("OfflinePartitionsPerSec", "partitions", TimeUnit.SECONDS)
|
||||||
val uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS)
|
val uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS)
|
||||||
val leaderElectionTimer = new KafkaTimer(newTimer("LeaderElectionRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
|
val leaderElectionTimer = new KafkaTimer(newTimer("LeaderElectionRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
|
||||||
|
|
|
@ -58,12 +58,12 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten
|
||||||
.format(liveAssignedReplicasToThisPartition.mkString(",")))
|
.format(liveAssignedReplicasToThisPartition.mkString(",")))
|
||||||
liveAssignedReplicasToThisPartition.isEmpty match {
|
liveAssignedReplicasToThisPartition.isEmpty match {
|
||||||
case true =>
|
case true =>
|
||||||
ControllerStat.offlinePartitionRate.mark()
|
ControllerStats.offlinePartitionRate.mark()
|
||||||
throw new PartitionOfflineException(("No replica for partition " +
|
throw new PartitionOfflineException(("No replica for partition " +
|
||||||
"([%s, %d]) is alive. Live brokers are: [%s],".format(topic, partition, controllerContext.liveBrokerIds)) +
|
"([%s, %d]) is alive. Live brokers are: [%s],".format(topic, partition, controllerContext.liveBrokerIds)) +
|
||||||
" Assigned replicas are: [%s]".format(assignedReplicas))
|
" Assigned replicas are: [%s]".format(assignedReplicas))
|
||||||
case false =>
|
case false =>
|
||||||
ControllerStat.uncleanLeaderElectionRate.mark()
|
ControllerStats.uncleanLeaderElectionRate.mark()
|
||||||
val newLeader = liveAssignedReplicasToThisPartition.head
|
val newLeader = liveAssignedReplicasToThisPartition.head
|
||||||
warn("No broker in ISR is alive, elected leader from the alive replicas is [%s], ".format(newLeader) +
|
warn("No broker in ISR is alive, elected leader from the alive replicas is [%s], ".format(newLeader) +
|
||||||
"There's potential data loss")
|
"There's potential data loss")
|
||||||
|
@ -78,7 +78,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten
|
||||||
partition))
|
partition))
|
||||||
(newLeaderAndIsr, liveAssignedReplicasToThisPartition)
|
(newLeaderAndIsr, liveAssignedReplicasToThisPartition)
|
||||||
case None =>
|
case None =>
|
||||||
ControllerStat.offlinePartitionRate.mark()
|
ControllerStats.offlinePartitionRate.mark()
|
||||||
throw new PartitionOfflineException("Partition [%s, %d] doesn't have".format(topic, partition) +
|
throw new PartitionOfflineException("Partition [%s, %d] doesn't have".format(topic, partition) +
|
||||||
"replicas assigned to it")
|
"replicas assigned to it")
|
||||||
}
|
}
|
||||||
|
|
|
@ -223,7 +223,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
|
||||||
val liveAssignedReplicas = replicaAssignment.filter(r => controllerContext.liveBrokerIds.contains(r))
|
val liveAssignedReplicas = replicaAssignment.filter(r => controllerContext.liveBrokerIds.contains(r))
|
||||||
liveAssignedReplicas.size match {
|
liveAssignedReplicas.size match {
|
||||||
case 0 =>
|
case 0 =>
|
||||||
ControllerStat.offlinePartitionRate.mark()
|
ControllerStats.offlinePartitionRate.mark()
|
||||||
throw new StateChangeFailedException(("During state change of partition %s from NEW to ONLINE, assigned replicas are " +
|
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,
|
"[%s], live brokers are [%s]. No assigned replica is alive").format(topicAndPartition,
|
||||||
replicaAssignment.mkString(","), controllerContext.liveBrokerIds))
|
replicaAssignment.mkString(","), controllerContext.liveBrokerIds))
|
||||||
|
@ -249,7 +249,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
|
||||||
// read the controller epoch
|
// read the controller epoch
|
||||||
val leaderIsrAndEpoch = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic,
|
val leaderIsrAndEpoch = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic,
|
||||||
topicAndPartition.partition).get
|
topicAndPartition.partition).get
|
||||||
ControllerStat.offlinePartitionRate.mark()
|
ControllerStats.offlinePartitionRate.mark()
|
||||||
throw new StateChangeFailedException("Error while changing partition %s's state from New to Online"
|
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 " +
|
.format(topicAndPartition) + " since Leader and isr path already exists with value " +
|
||||||
"%s and controller epoch %d".format(leaderIsrAndEpoch.leaderAndIsr.toString(), leaderIsrAndEpoch.controllerEpoch))
|
"%s and controller epoch %d".format(leaderIsrAndEpoch.leaderAndIsr.toString(), leaderIsrAndEpoch.controllerEpoch))
|
||||||
|
|
|
@ -227,7 +227,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
|
||||||
class BrokerChangeListener() extends IZkChildListener with Logging {
|
class BrokerChangeListener() extends IZkChildListener with Logging {
|
||||||
this.logIdent = "[BrokerChangeListener on Controller " + controller.config.brokerId + "]: "
|
this.logIdent = "[BrokerChangeListener on Controller " + controller.config.brokerId + "]: "
|
||||||
def handleChildChange(parentPath : String, currentBrokerList : java.util.List[String]) {
|
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(",")))
|
info("Broker change listener fired for path %s with children %s".format(parentPath, currentBrokerList.mkString(",")))
|
||||||
if(!isShuttingDown.get()) {
|
if(!isShuttingDown.get()) {
|
||||||
controllerContext.controllerLock synchronized {
|
controllerContext.controllerLock synchronized {
|
||||||
|
|
|
@ -20,14 +20,15 @@ import kafka.api._
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import scala.collection.JavaConversions
|
import scala.collection.JavaConversions
|
||||||
|
|
||||||
class TopicMetadataRequest(val versionId: Short,
|
class TopicMetadataRequest(val correlationId: Int,
|
||||||
|
val versionId: Short,
|
||||||
val clientId: String,
|
val clientId: String,
|
||||||
val topics: java.util.List[String]) extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) {
|
val topics: java.util.List[String]) extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) {
|
||||||
val underlying: kafka.api.TopicMetadataRequest =
|
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]) =
|
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)
|
def writeTo(buffer: ByteBuffer) = underlying.writeTo(buffer)
|
||||||
|
|
||||||
|
|
|
@ -29,8 +29,10 @@ import kafka.javaapi.OffsetRequest
|
||||||
class SimpleConsumer(val host: String,
|
class SimpleConsumer(val host: String,
|
||||||
val port: Int,
|
val port: Int,
|
||||||
val soTimeout: Int,
|
val soTimeout: Int,
|
||||||
val bufferSize: Int) {
|
val bufferSize: Int,
|
||||||
private val underlying = new kafka.consumer.SimpleConsumer(host, port, soTimeout, bufferSize)
|
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
|
* Fetch a set of messages from a topic. This version of the fetch method
|
||||||
|
|
|
@ -16,7 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package kafka.javaapi.consumer
|
package kafka.javaapi.consumer
|
||||||
|
|
||||||
import kafka.message.Message
|
|
||||||
import kafka.serializer._
|
import kafka.serializer._
|
||||||
import kafka.consumer._
|
import kafka.consumer._
|
||||||
import scala.collection.JavaConversions.asList
|
import scala.collection.JavaConversions.asList
|
||||||
|
|
|
@ -93,14 +93,23 @@ class Log(val dir: File,
|
||||||
val ls = dir.listFiles()
|
val ls = dir.listFiles()
|
||||||
if(ls != null) {
|
if(ls != null) {
|
||||||
for(file <- ls if file.isFile) {
|
for(file <- ls if file.isFile) {
|
||||||
|
if(!file.canRead)
|
||||||
|
throw new IOException("Could not read file " + file)
|
||||||
val filename = file.getName
|
val filename = file.getName
|
||||||
if(filename.endsWith(DeletedFileSuffix)) {
|
if(filename.endsWith(DeletedFileSuffix)) {
|
||||||
|
// if the file ends in .deleted, delete it
|
||||||
val deleted = file.delete()
|
val deleted = file.delete()
|
||||||
if(!deleted)
|
if(!deleted)
|
||||||
warn("Attempt to delete defunct segment file %s failed.".format(filename))
|
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)) {
|
} else if(filename.endsWith(LogFileSuffix)) {
|
||||||
if(!file.canRead)
|
// if its a log file, load the corresponding log segment
|
||||||
throw new IOException("Could not read file " + file)
|
|
||||||
val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong
|
val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong
|
||||||
val hasIndex = Log.indexFilename(dir, start).exists
|
val hasIndex = Log.indexFilename(dir, start).exists
|
||||||
val segment = new LogSegment(dir = dir,
|
val segment = new LogSegment(dir = dir,
|
||||||
|
|
|
@ -105,7 +105,7 @@ class LogManager(val config: KafkaConfig,
|
||||||
for(dir <- dirs) {
|
for(dir <- dirs) {
|
||||||
/* check if this set of logs was shut down cleanly */
|
/* check if this set of logs was shut down cleanly */
|
||||||
val cleanShutDownFile = new File(dir, CleanShutdownFile)
|
val cleanShutDownFile = new File(dir, CleanShutdownFile)
|
||||||
val needsRecovery = cleanShutDownFile.exists
|
val needsRecovery = !cleanShutDownFile.exists
|
||||||
cleanShutDownFile.delete
|
cleanShutDownFile.delete
|
||||||
/* load the logs */
|
/* load the logs */
|
||||||
val subDirs = dir.listFiles()
|
val subDirs = dir.listFiles()
|
||||||
|
|
|
@ -38,17 +38,20 @@ object Message {
|
||||||
val KeySizeOffset = AttributesOffset + AttributesLength
|
val KeySizeOffset = AttributesOffset + AttributesLength
|
||||||
val KeySizeLength = 4
|
val KeySizeLength = 4
|
||||||
val KeyOffset = KeySizeOffset + KeySizeLength
|
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
|
* 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
|
* 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.
|
* 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.AttributesLength +
|
||||||
Message.KeySizeLength +
|
Message.KeySizeLength +
|
||||||
(if(key == null) 0 else key.length) +
|
(if(key == null) 0 else key.length) +
|
||||||
|
Message.ValueSizeLength +
|
||||||
(if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset)))
|
(if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset)))
|
||||||
// skip crc, we will fill that in at the end
|
// skip crc, we will fill that in at the end
|
||||||
buffer.put(MagicOffset, CurrentMagicValue)
|
buffer.position(MagicOffset)
|
||||||
var attributes:Byte = 0
|
buffer.put(CurrentMagicValue)
|
||||||
|
var attributes: Byte = 0
|
||||||
if (codec.codec > 0)
|
if (codec.codec > 0)
|
||||||
attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte
|
attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte
|
||||||
buffer.put(AttributesOffset, attributes)
|
buffer.put(attributes)
|
||||||
if(key == null) {
|
if(key == null) {
|
||||||
buffer.putInt(KeySizeOffset, -1)
|
buffer.putInt(-1)
|
||||||
buffer.position(KeyOffset)
|
|
||||||
} else {
|
} else {
|
||||||
buffer.putInt(KeySizeOffset, key.length)
|
buffer.putInt(key.length)
|
||||||
buffer.position(KeyOffset)
|
|
||||||
buffer.put(key, 0, 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()
|
buffer.rewind()
|
||||||
|
|
||||||
// now compute the checksum and fill it in
|
// now compute the checksum and fill it in
|
||||||
|
@ -170,10 +175,15 @@ class Message(val buffer: ByteBuffer) {
|
||||||
*/
|
*/
|
||||||
def hasKey: Boolean = keySize >= 0
|
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
|
* 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
|
* The magic version of this message
|
||||||
|
@ -194,29 +204,27 @@ class Message(val buffer: ByteBuffer) {
|
||||||
/**
|
/**
|
||||||
* A ByteBuffer containing the content of the message
|
* A ByteBuffer containing the content of the message
|
||||||
*/
|
*/
|
||||||
def payload: ByteBuffer = {
|
def payload: ByteBuffer = sliceDelimited(payloadSizeOffset)
|
||||||
var payload = buffer.duplicate
|
|
||||||
payload.position(KeyOffset + max(keySize, 0))
|
|
||||||
payload = payload.slice()
|
|
||||||
payload.limit(payloadSize)
|
|
||||||
payload.rewind()
|
|
||||||
payload
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A ByteBuffer containing the message key
|
* A ByteBuffer containing the message key
|
||||||
*/
|
*/
|
||||||
def key: ByteBuffer = {
|
def key: ByteBuffer = sliceDelimited(KeySizeOffset)
|
||||||
val s = keySize
|
|
||||||
if(s < 0) {
|
/**
|
||||||
|
* 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
|
null
|
||||||
} else {
|
} else {
|
||||||
var key = buffer.duplicate
|
var b = buffer.duplicate
|
||||||
key.position(KeyOffset)
|
b.position(start + 4)
|
||||||
key = key.slice()
|
b = b.slice()
|
||||||
key.limit(s)
|
b.limit(size)
|
||||||
key.rewind()
|
b.rewind
|
||||||
key
|
b
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.yammer.metrics.Metrics
|
||||||
import java.io.File
|
import java.io.File
|
||||||
import com.yammer.metrics.reporting.CsvReporter
|
import com.yammer.metrics.reporting.CsvReporter
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
|
||||||
import kafka.utils.{Utils, VerifiableProperties, Logging}
|
import kafka.utils.{Utils, VerifiableProperties, Logging}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -13,7 +13,7 @@
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package kafka.producer
|
package kafka.producer
|
||||||
|
|
||||||
import collection.mutable.HashMap
|
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.
|
* 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
|
var topicsMetadata: Seq[TopicMetadata] = Nil
|
||||||
val topicMetadataResponse = ClientUtils.fetchTopicMetadata(topics, brokers)
|
val topicMetadataResponse = ClientUtils.fetchTopicMetadata(topics, brokers, producerConfig)
|
||||||
topicsMetadata = topicMetadataResponse.topicsMetadata
|
topicsMetadata = topicMetadataResponse.topicsMetadata
|
||||||
// throw partition specific exception
|
// throw partition specific exception
|
||||||
topicsMetadata.foreach(tmd =>{
|
topicsMetadata.foreach(tmd =>{
|
||||||
|
|
|
@ -21,6 +21,7 @@ import scala.collection.JavaConversions._
|
||||||
import joptsimple._
|
import joptsimple._
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import java.io._
|
import java.io._
|
||||||
|
import kafka.common._
|
||||||
import kafka.message._
|
import kafka.message._
|
||||||
import kafka.serializer._
|
import kafka.serializer._
|
||||||
|
|
||||||
|
@ -49,13 +50,18 @@ object ConsoleProducer {
|
||||||
.describedAs("timeout_ms")
|
.describedAs("timeout_ms")
|
||||||
.ofType(classOf[java.lang.Long])
|
.ofType(classOf[java.lang.Long])
|
||||||
.defaultsTo(1000)
|
.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
|
.withRequiredArg
|
||||||
.describedAs("encoder_class")
|
.describedAs("encoder_class")
|
||||||
.ofType(classOf[java.lang.String])
|
.ofType(classOf[java.lang.String])
|
||||||
.defaultsTo(classOf[StringEncoder].getName)
|
.defaultsTo(classOf[StringEncoder].getName)
|
||||||
val messageReaderOpt = parser.accepts("line-reader", "The class name of the class to use for reading lines from standard in. " +
|
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
|
.withRequiredArg
|
||||||
.describedAs("reader_class")
|
.describedAs("reader_class")
|
||||||
.ofType(classOf[java.lang.String])
|
.ofType(classOf[java.lang.String])
|
||||||
|
@ -82,9 +88,11 @@ object ConsoleProducer {
|
||||||
val compress = options.has(compressOpt)
|
val compress = options.has(compressOpt)
|
||||||
val batchSize = options.valueOf(batchSizeOpt)
|
val batchSize = options.valueOf(batchSizeOpt)
|
||||||
val sendTimeout = options.valueOf(sendTimeoutOpt)
|
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 readerClass = options.valueOf(messageReaderOpt)
|
||||||
val cmdLineProps = parseLineReaderArgs(options.valuesOf(propertyOpt))
|
val cmdLineProps = parseLineReaderArgs(options.valuesOf(propertyOpt))
|
||||||
|
cmdLineProps.put("topic", topic)
|
||||||
|
|
||||||
val props = new Properties()
|
val props = new Properties()
|
||||||
props.put("broker.list", brokerList)
|
props.put("broker.list", brokerList)
|
||||||
|
@ -94,12 +102,13 @@ object ConsoleProducer {
|
||||||
if(options.has(batchSizeOpt))
|
if(options.has(batchSizeOpt))
|
||||||
props.put("batch.size", batchSize.toString)
|
props.put("batch.size", batchSize.toString)
|
||||||
props.put("queue.time", sendTimeout.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)
|
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() {
|
Runtime.getRuntime.addShutdownHook(new Thread() {
|
||||||
override def run() {
|
override def run() {
|
||||||
|
@ -107,11 +116,11 @@ object ConsoleProducer {
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
var message: AnyRef = null
|
var message: KeyedMessage[AnyRef, AnyRef] = null
|
||||||
do {
|
do {
|
||||||
message = reader.readMessage()
|
message = reader.readMessage()
|
||||||
if(message != null)
|
if(message != null)
|
||||||
producer.send(new KeyedMessage(topic, message))
|
producer.send(message)
|
||||||
} while(message != null)
|
} while(message != null)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -127,19 +136,49 @@ object ConsoleProducer {
|
||||||
props
|
props
|
||||||
}
|
}
|
||||||
|
|
||||||
trait MessageReader {
|
trait MessageReader[K,V] {
|
||||||
def init(inputStream: InputStream, props: Properties) {}
|
def init(inputStream: InputStream, props: Properties) {}
|
||||||
def readMessage(): AnyRef
|
def readMessage(): KeyedMessage[K,V]
|
||||||
def close() {}
|
def close() {}
|
||||||
}
|
}
|
||||||
|
|
||||||
class LineMessageReader extends MessageReader {
|
class LineMessageReader extends MessageReader[String, String] {
|
||||||
|
var topic: String = null
|
||||||
var reader: BufferedReader = null
|
var reader: BufferedReader = null
|
||||||
|
var parseKey = false
|
||||||
|
var keySeparator = "\t"
|
||||||
|
var ignoreError = false
|
||||||
|
var lineNumber = 0
|
||||||
|
|
||||||
override def init(inputStream: InputStream, props: Properties) {
|
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))
|
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)
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,7 +17,6 @@
|
||||||
|
|
||||||
package kafka.producer
|
package kafka.producer
|
||||||
|
|
||||||
import kafka.utils.Utils
|
|
||||||
|
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
|
|
||||||
|
|
|
@ -16,23 +16,21 @@
|
||||||
*/
|
*/
|
||||||
package kafka.producer
|
package kafka.producer
|
||||||
|
|
||||||
import async.{AsyncProducerStats, DefaultEventHandler, ProducerSendThread, EventHandler}
|
import async.{DefaultEventHandler, ProducerSendThread, EventHandler}
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import java.util.Random
|
import java.util.Random
|
||||||
import java.util.concurrent.{TimeUnit, LinkedBlockingQueue}
|
import java.util.concurrent.{TimeUnit, LinkedBlockingQueue}
|
||||||
import kafka.serializer.Encoder
|
import kafka.serializer.Encoder
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
import kafka.common.{QueueFullException, InvalidConfigException}
|
import kafka.common.QueueFullException
|
||||||
import kafka.metrics._
|
import kafka.metrics._
|
||||||
|
|
||||||
|
|
||||||
class Producer[K,V](config: ProducerConfig,
|
class Producer[K,V](config: ProducerConfig,
|
||||||
private val eventHandler: EventHandler[K,V]) // for testing only
|
private val eventHandler: EventHandler[K,V]) // only for unit testing
|
||||||
extends Logging {
|
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 hasShutdown = new AtomicBoolean(false)
|
||||||
private val queue = new LinkedBlockingQueue[KeyedMessage[K,V]](config.queueSize)
|
private val queue = new LinkedBlockingQueue[KeyedMessage[K,V]](config.queueSize)
|
||||||
|
|
||||||
private val random = new Random
|
private val random = new Random
|
||||||
|
@ -47,11 +45,14 @@ extends Logging {
|
||||||
queue,
|
queue,
|
||||||
eventHandler,
|
eventHandler,
|
||||||
config.queueTime,
|
config.queueTime,
|
||||||
config.batchSize)
|
config.batchSize,
|
||||||
|
config.clientId)
|
||||||
producerSendThread.start()
|
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)
|
KafkaMetricsReporter.startReporters(config.props)
|
||||||
|
|
||||||
def this(config: ProducerConfig) =
|
def this(config: ProducerConfig) =
|
||||||
|
@ -65,7 +66,7 @@ extends Logging {
|
||||||
/**
|
/**
|
||||||
* Sends the data, partitioned by key to the topic using either the
|
* Sends the data, partitioned by key to the topic using either the
|
||||||
* synchronous or the asynchronous producer
|
* 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]*) {
|
def send(messages: KeyedMessage[K,V]*) {
|
||||||
if (hasShutdown.get)
|
if (hasShutdown.get)
|
||||||
|
@ -79,8 +80,8 @@ extends Logging {
|
||||||
|
|
||||||
private def recordStats(messages: Seq[KeyedMessage[K,V]]) {
|
private def recordStats(messages: Seq[KeyedMessage[K,V]]) {
|
||||||
for (message <- messages) {
|
for (message <- messages) {
|
||||||
ProducerTopicStat.getProducerTopicStat(message.topic).messageRate.mark()
|
producerTopicStats.getProducerTopicStats(message.topic).messageRate.mark()
|
||||||
ProducerTopicStat.getProducerAllTopicStat.messageRate.mark()
|
producerTopicStats.getProducerAllTopicStats.messageRate.mark()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -105,7 +106,7 @@ extends Logging {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if(!added) {
|
if(!added) {
|
||||||
AsyncProducerStats.droppedMessageRate.mark()
|
producerStats.droppedMessageRate.mark()
|
||||||
error("Event queue is full of unsent messages, could not send event: " + message.toString)
|
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)
|
throw new QueueFullException("Event queue is full of unsent messages, could not send event: " + message.toString)
|
||||||
}else {
|
}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)
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
|
@ -21,9 +21,36 @@ import async.AsyncProducerConfig
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import kafka.utils.{Utils, VerifiableProperties}
|
import kafka.utils.{Utils, VerifiableProperties}
|
||||||
import kafka.message.{CompressionCodec, NoCompressionCodec}
|
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)
|
class ProducerConfig private (val props: VerifiableProperties)
|
||||||
extends AsyncProducerConfig with SyncProducerConfigShared {
|
extends AsyncProducerConfig with SyncProducerConfigShared {
|
||||||
|
import ProducerConfig._
|
||||||
|
|
||||||
def this(originalProps: Properties) {
|
def this(originalProps: Properties) {
|
||||||
this(new VerifiableProperties(originalProps))
|
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
|
/** 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
|
* (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
|
* 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.
|
* a VIP pointing to a subset of brokers.
|
||||||
*/
|
*/
|
||||||
val brokerList = props.getString("broker.list")
|
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
|
* The amount of time to wait in between retries
|
||||||
*/
|
*/
|
||||||
val producerRetryBackoffMs = props.getInt("producer.retry.backoff.ms", 100)
|
val producerRetryBackoffMs = props.getInt("producer.retry.backoff.ms", 100)
|
||||||
|
|
||||||
|
validate(this)
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,13 +26,15 @@ import kafka.api.TopicMetadata
|
||||||
import kafka.common.UnavailableProducerException
|
import kafka.common.UnavailableProducerException
|
||||||
|
|
||||||
|
|
||||||
object ProducerPool{
|
object ProducerPool {
|
||||||
def createSyncProducer(configOpt: Option[ProducerConfig], broker: Broker): SyncProducer = {
|
/**
|
||||||
|
* Used in ProducerPool to initiate a SyncProducer connection with a broker.
|
||||||
|
*/
|
||||||
|
def createSyncProducer(config: ProducerConfig, broker: Broker): SyncProducer = {
|
||||||
val props = new Properties()
|
val props = new Properties()
|
||||||
props.put("host", broker.host)
|
props.put("host", broker.host)
|
||||||
props.put("port", broker.port.toString)
|
props.put("port", broker.port.toString)
|
||||||
if(configOpt.isDefined)
|
props.putAll(config.props.props)
|
||||||
props.putAll(configOpt.get.props.props)
|
|
||||||
new SyncProducer(new SyncProducerConfig(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 syncProducers = new HashMap[Int, SyncProducer]
|
||||||
private val lock = new Object()
|
private val lock = new Object()
|
||||||
|
|
||||||
def updateProducer(topicMetaDatas: Seq[TopicMetadata]) {
|
def updateProducer(topicMetadatas: Seq[TopicMetadata]) {
|
||||||
val newBrokers = new collection.mutable.HashSet[Broker]
|
val newBrokers = new collection.mutable.HashSet[Broker]
|
||||||
topicMetaDatas.foreach(tmd => {
|
topicMetadatas.foreach(tmd => {
|
||||||
tmd.partitionsMetadata.foreach(pmd => {
|
tmd.partitionsMetadata.foreach(pmd => {
|
||||||
if(pmd.leader.isDefined)
|
if(pmd.leader.isDefined)
|
||||||
newBrokers+=(pmd.leader.get)
|
newBrokers+=(pmd.leader.get)
|
||||||
|
@ -53,9 +55,9 @@ class ProducerPool(val config: ProducerConfig) extends Logging {
|
||||||
newBrokers.foreach(b => {
|
newBrokers.foreach(b => {
|
||||||
if(syncProducers.contains(b.id)){
|
if(syncProducers.contains(b.id)){
|
||||||
syncProducers(b.id).close()
|
syncProducers(b.id).close()
|
||||||
syncProducers.put(b.id, ProducerPool.createSyncProducer(Some(config), b))
|
syncProducers.put(b.id, ProducerPool.createSyncProducer(config, b))
|
||||||
} else
|
} else
|
||||||
syncProducers.put(b.id, ProducerPool.createSyncProducer(Some(config), b))
|
syncProducers.put(b.id, ProducerPool.createSyncProducer(config, b))
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -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)
|
||||||
|
}
|
||||||
|
}
|
|
@ -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)
|
||||||
|
}
|
||||||
|
}
|
|
@ -21,8 +21,6 @@ import kafka.api._
|
||||||
import kafka.network.{BlockingChannel, BoundedByteBufferSend, Receive}
|
import kafka.network.{BlockingChannel, BoundedByteBufferSend, Receive}
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import java.util.Random
|
import java.util.Random
|
||||||
import java.util.concurrent.TimeUnit
|
|
||||||
import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
|
|
||||||
|
|
||||||
object SyncProducer {
|
object SyncProducer {
|
||||||
val RequestKey: Short = 0
|
val RequestKey: Short = 0
|
||||||
|
@ -34,14 +32,13 @@ object SyncProducer {
|
||||||
*/
|
*/
|
||||||
@threadsafe
|
@threadsafe
|
||||||
class SyncProducer(val config: SyncProducerConfig) extends Logging {
|
class SyncProducer(val config: SyncProducerConfig) extends Logging {
|
||||||
|
|
||||||
private val MaxConnectBackoffMs = 60000
|
|
||||||
private var sentOnConnection = 0
|
|
||||||
|
|
||||||
private val lock = new Object()
|
private val lock = new Object()
|
||||||
@volatile private var shutdown: Boolean = false
|
@volatile private var shutdown: Boolean = false
|
||||||
private val blockingChannel = new BlockingChannel(config.host, config.port, BlockingChannel.UseDefaultBufferSize,
|
private val blockingChannel = new BlockingChannel(config.host, config.port, BlockingChannel.UseDefaultBufferSize,
|
||||||
config.bufferSize, config.requestTimeoutMs)
|
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")
|
trace("Instantiating Scala Sync Producer")
|
||||||
|
|
||||||
|
@ -89,10 +86,17 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging {
|
||||||
* Send a message
|
* Send a message
|
||||||
*/
|
*/
|
||||||
def send(producerRequest: ProducerRequest): ProducerResponse = {
|
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
|
var response: Receive = null
|
||||||
ProducerRequestStat.requestTimer.time {
|
val specificTimer = producerRequestStats.getProducerRequestStats(brokerInfo).requestTimer
|
||||||
response = doSend(producerRequest)
|
val aggregateTimer = producerRequestStats.getProducerRequestAllBrokersStats.requestTimer
|
||||||
|
aggregateTimer.time {
|
||||||
|
specificTimer.time {
|
||||||
|
response = doSend(producerRequest)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
ProducerResponse.readFrom(response.buffer)
|
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")
|
|
||||||
}
|
|
||||||
|
|
|
@ -41,10 +41,7 @@ trait SyncProducerConfigShared {
|
||||||
val maxMessageSize = props.getInt("max.message.size", 1000000)
|
val maxMessageSize = props.getInt("max.message.size", 1000000)
|
||||||
|
|
||||||
/* the client application sending the producer requests */
|
/* the client application sending the producer requests */
|
||||||
val correlationId = props.getInt("producer.request.correlation_id", SyncProducerConfig.DefaultCorrelationId)
|
val clientId = props.getString("clientid", SyncProducerConfig.DefaultClientId)
|
||||||
|
|
||||||
/* the client application sending the producer requests */
|
|
||||||
val clientId = props.getString("producer.request.client_id",SyncProducerConfig.DefaultClientId)
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* The required acks of the producer requests - negative value means ack
|
* The required acks of the producer requests - negative value means ack
|
||||||
|
@ -61,8 +58,7 @@ trait SyncProducerConfigShared {
|
||||||
}
|
}
|
||||||
|
|
||||||
object SyncProducerConfig {
|
object SyncProducerConfig {
|
||||||
val DefaultCorrelationId = -1
|
|
||||||
val DefaultClientId = ""
|
val DefaultClientId = ""
|
||||||
val DefaultRequiredAcks : Short = 0
|
val DefaultRequiredAcks : Short = 0
|
||||||
val DefaultAckTimeoutMs = 500
|
val DefaultAckTimeoutMs = 1500
|
||||||
}
|
}
|
|
@ -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)
|
|
||||||
}
|
|
|
@ -37,19 +37,23 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
|
||||||
extends EventHandler[K,V] with Logging {
|
extends EventHandler[K,V] with Logging {
|
||||||
val isSync = ("sync" == config.producerType)
|
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)
|
val brokerPartitionInfo = new BrokerPartitionInfo(config, producerPool, topicPartitionInfos)
|
||||||
|
|
||||||
private val lock = new Object()
|
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]]) {
|
def handle(events: Seq[KeyedMessage[K,V]]) {
|
||||||
lock synchronized {
|
lock synchronized {
|
||||||
val serializedData = serialize(events)
|
val serializedData = serialize(events)
|
||||||
serializedData.foreach{
|
serializedData.foreach{
|
||||||
keyed =>
|
keyed =>
|
||||||
val dataSize = keyed.message.payloadSize
|
val dataSize = keyed.message.payloadSize
|
||||||
ProducerTopicStat.getProducerTopicStat(keyed.topic).byteRate.mark(dataSize)
|
producerTopicStats.getProducerTopicStats(keyed.topic).byteRate.mark(dataSize)
|
||||||
ProducerTopicStat.getProducerAllTopicStat.byteRate.mark(dataSize)
|
producerTopicStats.getProducerAllTopicStats.byteRate.mark(dataSize)
|
||||||
}
|
}
|
||||||
var outstandingProduceRequests = serializedData
|
var outstandingProduceRequests = serializedData
|
||||||
var remainingRetries = config.producerRetries + 1
|
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
|
// get topics of the outstanding produce requests and refresh metadata for those
|
||||||
Utils.swallowError(brokerPartitionInfo.updateInfo(outstandingProduceRequests.map(_.topic).toSet))
|
Utils.swallowError(brokerPartitionInfo.updateInfo(outstandingProduceRequests.map(_.topic).toSet))
|
||||||
remainingRetries -= 1
|
remainingRetries -= 1
|
||||||
ProducerStats.resendRate.mark()
|
producerStats.resendRate.mark()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if(outstandingProduceRequests.size > 0) {
|
if(outstandingProduceRequests.size > 0) {
|
||||||
ProducerStats.failedSendRate.mark()
|
producerStats.failedSendRate.mark()
|
||||||
error("Failed to send the following requests: " + outstandingProduceRequests)
|
error("Failed to send the following requests: " + outstandingProduceRequests)
|
||||||
throw new FailedToSendMessageException("Failed to send messages after " + config.producerRetries + " tries.", null)
|
throw new FailedToSendMessageException("Failed to send messages after " + config.producerRetries + " tries.", null)
|
||||||
}
|
}
|
||||||
|
@ -80,7 +84,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
|
||||||
try {
|
try {
|
||||||
for ((brokerid, messagesPerBrokerMap) <- partitionedData) {
|
for ((brokerid, messagesPerBrokerMap) <- partitionedData) {
|
||||||
if (logger.isTraceEnabled)
|
if (logger.isTraceEnabled)
|
||||||
messagesPerBrokerMap.foreach(partitionAndEvent =>
|
messagesPerBrokerMap.foreach(partitionAndEvent =>
|
||||||
trace("Handling event for Topic: %s, Broker: %d, Partitions: %s".format(partitionAndEvent._1, brokerid, partitionAndEvent._2)))
|
trace("Handling event for Topic: %s, Broker: %d, Partitions: %s".format(partitionAndEvent._1, brokerid, partitionAndEvent._2)))
|
||||||
val messageSetPerBroker = groupMessagesToSet(messagesPerBrokerMap)
|
val messageSetPerBroker = groupMessagesToSet(messagesPerBrokerMap)
|
||||||
|
|
||||||
|
@ -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)))
|
serializedMessages += KeyedMessage[K,Message](topic = e.topic, key = null.asInstanceOf[K], message = new Message(bytes = encoder.toBytes(e.message)))
|
||||||
} catch {
|
} catch {
|
||||||
case t =>
|
case t =>
|
||||||
ProducerStats.serializationErrorRate.mark()
|
producerStats.serializationErrorRate.mark()
|
||||||
if (isSync) {
|
if (isSync) {
|
||||||
throw t
|
throw t
|
||||||
} else {
|
} else {
|
||||||
|
@ -171,7 +175,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
|
||||||
debug("Broker partitions registered for topic: %s are %s"
|
debug("Broker partitions registered for topic: %s are %s"
|
||||||
.format(m.topic, topicPartitionsList.map(p => p.partitionId).mkString(",")))
|
.format(m.topic, topicPartitionsList.map(p => p.partitionId).mkString(",")))
|
||||||
val totalNumPartitions = topicPartitionsList.length
|
val totalNumPartitions = topicPartitionsList.length
|
||||||
if(totalNumPartitions == 0)
|
if(totalNumPartitions == 0)
|
||||||
throw new NoBrokersForPartitionException("Partition key = " + m.key)
|
throw new NoBrokersForPartitionException("Partition key = " + m.key)
|
||||||
topicPartitionsList
|
topicPartitionsList
|
||||||
}
|
}
|
||||||
|
@ -187,10 +191,10 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
|
||||||
if(numPartitions <= 0)
|
if(numPartitions <= 0)
|
||||||
throw new UnknownTopicOrPartitionException("Invalid number of partitions: " + numPartitions +
|
throw new UnknownTopicOrPartitionException("Invalid number of partitions: " + numPartitions +
|
||||||
"\n Valid values are > 0")
|
"\n Valid values are > 0")
|
||||||
val partition =
|
val partition =
|
||||||
if(key == null)
|
if(key == null)
|
||||||
Utils.abs(counter.getAndIncrement()) % numPartitions
|
Utils.abs(partitionCounter.getAndIncrement()) % numPartitions
|
||||||
else
|
else
|
||||||
partitioner.partition(key, numPartitions)
|
partitioner.partition(key, numPartitions)
|
||||||
if(partition < 0 || partition >= numPartitions)
|
if(partition < 0 || partition >= numPartitions)
|
||||||
throw new UnknownTopicOrPartitionException("Invalid partition id : " + partition +
|
throw new UnknownTopicOrPartitionException("Invalid partition id : " + partition +
|
||||||
|
@ -210,20 +214,21 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
|
||||||
warn("Failed to send to broker %d with data %s".format(brokerId, messagesPerTopic))
|
warn("Failed to send to broker %d with data %s".format(brokerId, messagesPerTopic))
|
||||||
messagesPerTopic.keys.toSeq
|
messagesPerTopic.keys.toSeq
|
||||||
} else if(messagesPerTopic.size > 0) {
|
} else if(messagesPerTopic.size > 0) {
|
||||||
val topicPartitionDataPairs = messagesPerTopic.toSeq.map {
|
val producerRequest = new ProducerRequest(correlationCounter.getAndIncrement(), config.clientId, config.requiredAcks,
|
||||||
case (topicAndPartition, messages) =>
|
config.requestTimeoutMs, messagesPerTopic)
|
||||||
(topicAndPartition, messages)
|
|
||||||
}
|
|
||||||
val producerRequest = new ProducerRequest(config.correlationId, config.clientId, config.requiredAcks,
|
|
||||||
config.requestTimeoutMs, Map(topicPartitionDataPairs:_*))
|
|
||||||
try {
|
try {
|
||||||
val syncProducer = producerPool.getProducer(brokerId)
|
val syncProducer = producerPool.getProducer(brokerId)
|
||||||
val response = syncProducer.send(producerRequest)
|
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))
|
.format(messagesPerTopic, brokerId, syncProducer.config.host, syncProducer.config.port))
|
||||||
if (response.status.size != producerRequest.data.size)
|
if (response.status.size != producerRequest.data.size)
|
||||||
throw new KafkaException("Incomplete response (%s) for producer request (%s)"
|
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
|
response.status.filter(_._2.error != ErrorMapping.NoError).toSeq
|
||||||
.map(partitionStatus => partitionStatus._1)
|
.map(partitionStatus => partitionStatus._1)
|
||||||
} catch {
|
} catch {
|
||||||
|
@ -238,33 +243,33 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
|
||||||
|
|
||||||
private def groupMessagesToSet(messagesPerTopicAndPartition: Map[TopicAndPartition, Seq[KeyedMessage[K,Message]]]) = {
|
private def groupMessagesToSet(messagesPerTopicAndPartition: Map[TopicAndPartition, Seq[KeyedMessage[K,Message]]]) = {
|
||||||
/** enforce the compressed.topics config here.
|
/** enforce the compressed.topics config here.
|
||||||
* If the compression codec is anything other than NoCompressionCodec,
|
* If the compression codec is anything other than NoCompressionCodec,
|
||||||
* Enable compression only for specified topics if any
|
* 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 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 NoCompressionCodec, compression is disabled for all topics
|
||||||
*/
|
*/
|
||||||
|
|
||||||
val messagesPerTopicPartition = messagesPerTopicAndPartition.map { case (topicAndPartition, messages) =>
|
val messagesPerTopicPartition = messagesPerTopicAndPartition.map { case (topicAndPartition, messages) =>
|
||||||
val rawMessages = messages.map(_.message)
|
val rawMessages = messages.map(_.message)
|
||||||
( topicAndPartition,
|
( topicAndPartition,
|
||||||
config.compressionCodec match {
|
config.compressionCodec match {
|
||||||
case NoCompressionCodec =>
|
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: _*)
|
new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*)
|
||||||
case _ =>
|
case _ =>
|
||||||
config.compressedTopics.size match {
|
config.compressedTopics.size match {
|
||||||
case 0 =>
|
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))
|
.format(messages.size, config.compressionCodec.codec, topicAndPartition))
|
||||||
new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*)
|
new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*)
|
||||||
case _ =>
|
case _ =>
|
||||||
if(config.compressedTopics.contains(topicAndPartition.topic)) {
|
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))
|
.format(messages.size, config.compressionCodec.codec, topicAndPartition))
|
||||||
new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*)
|
new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*)
|
||||||
}
|
}
|
||||||
else {
|
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))
|
.format(messages.size, topicAndPartition, config.compressedTopics.toString))
|
||||||
new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*)
|
new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*)
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,7 @@ package kafka.producer.async
|
||||||
|
|
||||||
import kafka.utils.{SystemTime, Logging}
|
import kafka.utils.{SystemTime, Logging}
|
||||||
import java.util.concurrent.{TimeUnit, CountDownLatch, BlockingQueue}
|
import java.util.concurrent.{TimeUnit, CountDownLatch, BlockingQueue}
|
||||||
import collection.mutable.ListBuffer
|
import collection.mutable.ArrayBuffer
|
||||||
import kafka.producer.KeyedMessage
|
import kafka.producer.KeyedMessage
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
|
@ -28,12 +28,13 @@ class ProducerSendThread[K,V](val threadName: String,
|
||||||
val queue: BlockingQueue[KeyedMessage[K,V]],
|
val queue: BlockingQueue[KeyedMessage[K,V]],
|
||||||
val handler: EventHandler[K,V],
|
val handler: EventHandler[K,V],
|
||||||
val queueTime: Long,
|
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 shutdownLatch = new CountDownLatch(1)
|
||||||
private val shutdownCommand = new KeyedMessage[K,V]("shutdown", null.asInstanceOf[K], null.asInstanceOf[V])
|
private val shutdownCommand = new KeyedMessage[K,V]("shutdown", null.asInstanceOf[K], null.asInstanceOf[V])
|
||||||
|
|
||||||
newGauge("ProducerQueueSize-" + getId,
|
newGauge(clientId + "-ProducerQueueSize-" + getId,
|
||||||
new Gauge[Int] {
|
new Gauge[Int] {
|
||||||
def getValue = queue.size
|
def getValue = queue.size
|
||||||
})
|
})
|
||||||
|
@ -57,7 +58,7 @@ class ProducerSendThread[K,V](val threadName: String,
|
||||||
|
|
||||||
private def processEvents() {
|
private def processEvents() {
|
||||||
var lastSend = SystemTime.milliseconds
|
var lastSend = SystemTime.milliseconds
|
||||||
var events = new ListBuffer[KeyedMessage[K,V]]
|
var events = new ArrayBuffer[KeyedMessage[K,V]]
|
||||||
var full: Boolean = false
|
var full: Boolean = false
|
||||||
|
|
||||||
// drain the queue until you get a shutdown command
|
// 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
|
// if either queue time has reached or batch size has reached, dispatch to event handler
|
||||||
tryToHandle(events)
|
tryToHandle(events)
|
||||||
lastSend = SystemTime.milliseconds
|
lastSend = SystemTime.milliseconds
|
||||||
events = new ListBuffer[KeyedMessage[K,V]]
|
events = new ArrayBuffer[KeyedMessage[K,V]]
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// send the last batch of events
|
// send the last batch of events
|
||||||
|
|
|
@ -17,7 +17,6 @@
|
||||||
|
|
||||||
package kafka.serializer
|
package kafka.serializer
|
||||||
|
|
||||||
import kafka.message._
|
|
||||||
import kafka.utils.VerifiableProperties
|
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
|
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
|
* The string encoder translates strings into bytes. It uses UTF8 by default but takes
|
||||||
* an optional property serializer.encoding to control this.
|
* an optional property serializer.encoding to control this.
|
||||||
|
|
|
@ -18,8 +18,6 @@
|
||||||
package kafka.serializer
|
package kafka.serializer
|
||||||
|
|
||||||
import kafka.utils.VerifiableProperties
|
import kafka.utils.VerifiableProperties
|
||||||
import kafka.message._
|
|
||||||
import kafka.utils.Utils
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An encoder is a method of turning objects into byte arrays.
|
* An encoder is a method of turning objects into byte arrays.
|
||||||
|
|
|
@ -19,7 +19,7 @@ package kafka.server
|
||||||
|
|
||||||
import kafka.cluster.Broker
|
import kafka.cluster.Broker
|
||||||
import kafka.consumer.SimpleConsumer
|
import kafka.consumer.SimpleConsumer
|
||||||
import kafka.common.{TopicAndPartition, ErrorMapping}
|
import kafka.common.{ClientIdAndBroker, TopicAndPartition, ErrorMapping}
|
||||||
import collection.mutable
|
import collection.mutable
|
||||||
import kafka.message.ByteBufferMessageSet
|
import kafka.message.ByteBufferMessageSet
|
||||||
import kafka.message.MessageAndOffset
|
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,
|
abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int,
|
||||||
fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1)
|
fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1)
|
||||||
extends ShutdownableThread(name) {
|
extends ShutdownableThread(name) {
|
||||||
|
|
||||||
private val partitionMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset map
|
private val partitionMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset map
|
||||||
private val partitionMapLock = new ReentrantLock
|
private val partitionMapLock = new ReentrantLock
|
||||||
private val partitionMapCond = partitionMapLock.newCondition()
|
private val partitionMapCond = partitionMapLock.newCondition()
|
||||||
val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize)
|
val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, clientId)
|
||||||
val fetcherMetrics = FetcherStat.getFetcherStat(name + "-" + sourceBroker.id)
|
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 */
|
/* callbacks to be defined in subclass */
|
||||||
|
|
||||||
|
@ -117,7 +120,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke
|
||||||
case None => currentOffset.get
|
case None => currentOffset.get
|
||||||
}
|
}
|
||||||
partitionMap.put(topicAndPartition, newOffset)
|
partitionMap.put(topicAndPartition, newOffset)
|
||||||
FetcherLagMetrics.getFetcherLagMetrics(topic, partitionId).lag = partitionData.hw - newOffset
|
fetcherLagStats.getFetcherLagStats(topic, partitionId).lag = partitionData.hw - newOffset
|
||||||
fetcherMetrics.byteRate.mark(validBytes)
|
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
|
// 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)
|
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)
|
private[this] var lagVal = new AtomicLong(-1L)
|
||||||
newGauge(
|
newGauge(
|
||||||
name._1 + "-" + name._2 + "-ConsumerLag",
|
metricId + "-ConsumerLag",
|
||||||
new Gauge[Long] {
|
new Gauge[Long] {
|
||||||
def getValue = lagVal.get
|
def getValue = lagVal.get
|
||||||
}
|
}
|
||||||
|
@ -198,25 +201,34 @@ class FetcherLagMetrics(name: (String, Int)) extends KafkaMetricsGroup {
|
||||||
def lag = lagVal.get
|
def lag = lagVal.get
|
||||||
}
|
}
|
||||||
|
|
||||||
object FetcherLagMetrics {
|
class FetcherLagStats(metricId: ClientIdAndBroker) {
|
||||||
private val valueFactory = (k: (String, Int)) => new FetcherLagMetrics(k)
|
private val valueFactory = (k: ClientIdBrokerTopicPartition) => new FetcherLagMetrics(k)
|
||||||
private val stats = new Pool[(String, Int), FetcherLagMetrics](Some(valueFactory))
|
private val stats = new Pool[ClientIdBrokerTopicPartition, FetcherLagMetrics](Some(valueFactory))
|
||||||
|
|
||||||
def getFetcherLagMetrics(topic: String, partitionId: Int): FetcherLagMetrics = {
|
def getFetcherLagStats(topic: String, partitionId: Int): FetcherLagMetrics = {
|
||||||
stats.getAndMaybePut( (topic, partitionId) )
|
stats.getAndMaybePut(new ClientIdBrokerTopicPartition(metricId.clientId, metricId.brokerInfo, topic, partitionId))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class FetcherStat(name: String) extends KafkaMetricsGroup {
|
class FetcherMetrics(metricId: ClientIdBrokerTopic) extends KafkaMetricsGroup {
|
||||||
val requestRate = newMeter(name + "RequestsPerSec", "requests", TimeUnit.SECONDS)
|
val requestRate = newMeter(metricId + "-RequestsPerSec", "requests", TimeUnit.SECONDS)
|
||||||
val byteRate = newMeter(name + "BytesPerSec", "bytes", TimeUnit.SECONDS)
|
val byteRate = newMeter(metricId + "-BytesPerSec", "bytes", TimeUnit.SECONDS)
|
||||||
}
|
}
|
||||||
|
|
||||||
object FetcherStat {
|
class FetcherStats(metricId: ClientIdAndBroker) {
|
||||||
private val valueFactory = (k: String) => new FetcherStat(k)
|
private val valueFactory = (k: ClientIdBrokerTopic) => new FetcherMetrics(k)
|
||||||
private val stats = new Pool[String, FetcherStat](Some(valueFactory))
|
private val stats = new Pool[ClientIdBrokerTopic, FetcherMetrics](Some(valueFactory))
|
||||||
|
|
||||||
def getFetcherStat(name: String): FetcherStat = {
|
def getFetcherStats(name: String): FetcherMetrics = {
|
||||||
stats.getAndMaybePut(name)
|
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)
|
||||||
|
}
|
||||||
|
|
||||||
|
|
|
@ -30,7 +30,7 @@ import java.io._
|
||||||
*/
|
*/
|
||||||
|
|
||||||
object HighwaterMarkCheckpoint {
|
object HighwaterMarkCheckpoint {
|
||||||
val highWatermarkFileName = ".highwatermark"
|
val highWatermarkFileName = "replication-offset-checkpoint"
|
||||||
val currentHighwaterMarkFileVersion = 0
|
val currentHighwaterMarkFileVersion = 0
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -41,8 +41,10 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
val zkClient: ZkClient,
|
val zkClient: ZkClient,
|
||||||
brokerId: Int) extends Logging {
|
brokerId: Int) extends Logging {
|
||||||
|
|
||||||
private val producerRequestPurgatory = new ProducerRequestPurgatory
|
private val producerRequestPurgatory =
|
||||||
private val fetchRequestPurgatory = new FetchRequestPurgatory(requestChannel)
|
new ProducerRequestPurgatory(replicaManager.config.producerRequestPurgatoryPurgeInterval)
|
||||||
|
private val fetchRequestPurgatory =
|
||||||
|
new FetchRequestPurgatory(requestChannel, replicaManager.config.fetchRequestPurgatoryPurgeInterval)
|
||||||
private val delayedRequestMetrics = new DelayedRequestMetrics
|
private val delayedRequestMetrics = new DelayedRequestMetrics
|
||||||
|
|
||||||
private val requestLogger = Logger.getLogger("kafka.request.logger")
|
private val requestLogger = Logger.getLogger("kafka.request.logger")
|
||||||
|
@ -71,16 +73,16 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
case (topicAndPartition, data) =>
|
case (topicAndPartition, data) =>
|
||||||
(topicAndPartition, ProducerResponseStatus(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1l))
|
(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)))
|
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
|
||||||
error("error when handling request %s".format(apiRequest), e)
|
error("error when handling request %s".format(apiRequest), e)
|
||||||
case RequestKeys.FetchKey =>
|
case RequestKeys.FetchKey =>
|
||||||
val apiRequest = request.requestObj.asInstanceOf[FetchRequest]
|
val apiRequest = request.requestObj.asInstanceOf[FetchRequest]
|
||||||
val fetchResponsePartitionData = apiRequest.requestInfo.map {
|
val fetchResponsePartitionData = apiRequest.requestInfo.map {
|
||||||
case (topicAndPartition, data) =>
|
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)))
|
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(errorResponse)))
|
||||||
error("error when handling request %s".format(apiRequest), e)
|
error("error when handling request %s".format(apiRequest), e)
|
||||||
case RequestKeys.OffsetsKey =>
|
case RequestKeys.OffsetsKey =>
|
||||||
|
@ -89,7 +91,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
case (topicAndPartition, partitionOffsetRequest) =>
|
case (topicAndPartition, partitionOffsetRequest) =>
|
||||||
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), null))
|
(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)))
|
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
|
||||||
error("error when handling request %s".format(apiRequest), e)
|
error("error when handling request %s".format(apiRequest), e)
|
||||||
case RequestKeys.MetadataKey =>
|
case RequestKeys.MetadataKey =>
|
||||||
|
@ -97,7 +99,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
val topicMeatadata = apiRequest.topics.map {
|
val topicMeatadata = apiRequest.topics.map {
|
||||||
topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
|
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)))
|
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
|
||||||
error("error when handling request %s".format(apiRequest), e)
|
error("error when handling request %s".format(apiRequest), e)
|
||||||
case RequestKeys.LeaderAndIsrKey =>
|
case RequestKeys.LeaderAndIsrKey =>
|
||||||
|
@ -105,7 +107,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
val responseMap = apiRequest.partitionStateInfos.map {
|
val responseMap = apiRequest.partitionStateInfos.map {
|
||||||
case (topicAndPartition, partitionAndState) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
|
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)))
|
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
|
||||||
error("error when handling request %s".format(apiRequest), e)
|
error("error when handling request %s".format(apiRequest), e)
|
||||||
case RequestKeys.StopReplicaKey =>
|
case RequestKeys.StopReplicaKey =>
|
||||||
|
@ -114,7 +116,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
|
case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
|
||||||
}.toMap
|
}.toMap
|
||||||
error("error when handling request %s".format(apiRequest), e)
|
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)))
|
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
|
||||||
}
|
}
|
||||||
} finally
|
} finally
|
||||||
|
@ -128,7 +130,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
trace("Handling leader and ISR request " + leaderAndIsrRequest)
|
trace("Handling leader and ISR request " + leaderAndIsrRequest)
|
||||||
try {
|
try {
|
||||||
val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest)
|
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)))
|
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse)))
|
||||||
} catch {
|
} catch {
|
||||||
case e: KafkaStorageException =>
|
case e: KafkaStorageException =>
|
||||||
|
@ -145,7 +147,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
trace("Handling stop replica request " + stopReplicaRequest)
|
trace("Handling stop replica request " + stopReplicaRequest)
|
||||||
|
|
||||||
val (response, error) = replicaManager.stopReplicas(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)))
|
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse)))
|
||||||
|
|
||||||
replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads()
|
replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads()
|
||||||
|
@ -162,7 +164,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
// send any newly unblocked responses
|
// send any newly unblocked responses
|
||||||
for(fetchReq <- satisfied) {
|
for(fetchReq <- satisfied) {
|
||||||
val topicData = readMessageSets(fetchReq.fetch)
|
val topicData = readMessageSets(fetchReq.fetch)
|
||||||
val response = FetchResponse(FetchRequest.CurrentVersion, fetchReq.fetch.correlationId, topicData)
|
val response = FetchResponse(fetchReq.fetch.correlationId, topicData)
|
||||||
requestChannel.sendResponse(new RequestChannel.Response(fetchReq.request, new FetchResponseSend(response)))
|
requestChannel.sendResponse(new RequestChannel.Response(fetchReq.request, new FetchResponseSend(response)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -193,7 +195,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
allPartitionHaveReplicationFactorOne ||
|
allPartitionHaveReplicationFactorOne ||
|
||||||
numPartitionsInError == produceRequest.numPartitions) {
|
numPartitionsInError == produceRequest.numPartitions) {
|
||||||
val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap
|
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)))
|
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
|
||||||
} else {
|
} else {
|
||||||
// create a list of (topic, partition) pairs to use as keys for this delayed request
|
// create a list of (topic, partition) pairs to use as keys for this delayed request
|
||||||
|
@ -238,8 +240,8 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
private def appendToLocalLog(partitionAndData: Map[TopicAndPartition, MessageSet]): Iterable[ProduceResult] = {
|
private def appendToLocalLog(partitionAndData: Map[TopicAndPartition, MessageSet]): Iterable[ProduceResult] = {
|
||||||
trace("Append [%s] to local log ".format(partitionAndData.toString))
|
trace("Append [%s] to local log ".format(partitionAndData.toString))
|
||||||
partitionAndData.map {case (topicAndPartition, messages) =>
|
partitionAndData.map {case (topicAndPartition, messages) =>
|
||||||
BrokerTopicStat.getBrokerTopicStat(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes)
|
||||||
BrokerTopicStat.getBrokerAllTopicStat.bytesInRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerAllTopicStats.bytesInRate.mark(messages.sizeInBytes)
|
||||||
|
|
||||||
try {
|
try {
|
||||||
val localReplica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition)
|
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)
|
val info = log.append(messages.asInstanceOf[ByteBufferMessageSet], assignOffsets = true)
|
||||||
|
|
||||||
// update stats
|
// update stats
|
||||||
BrokerTopicStat.getBrokerTopicStat(topicAndPartition.topic).messagesInRate.mark(info.count)
|
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(info.count)
|
||||||
BrokerTopicStat.getBrokerAllTopicStat.messagesInRate.mark(info.count)
|
BrokerTopicStats.getBrokerAllTopicStats.messagesInRate.mark(info.count)
|
||||||
|
|
||||||
// we may need to increment high watermark since ISR could be down to 1
|
// we may need to increment high watermark since ISR could be down to 1
|
||||||
localReplica.partition.maybeIncrementLeaderHW(localReplica)
|
localReplica.partition.maybeIncrementLeaderHW(localReplica)
|
||||||
|
@ -261,8 +263,8 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
Runtime.getRuntime.halt(1)
|
Runtime.getRuntime.halt(1)
|
||||||
null
|
null
|
||||||
case e =>
|
case e =>
|
||||||
BrokerTopicStat.getBrokerTopicStat(topicAndPartition.topic).failedProduceRequestRate.mark()
|
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark()
|
||||||
BrokerTopicStat.getBrokerAllTopicStat.failedProduceRequestRate.mark()
|
BrokerTopicStats.getBrokerAllTopicStats.failedProduceRequestRate.mark()
|
||||||
error("Error processing ProducerRequest on %s:%d".format(topicAndPartition.topic, topicAndPartition.partition), e)
|
error("Error processing ProducerRequest on %s:%d".format(topicAndPartition.topic, topicAndPartition.partition), e)
|
||||||
new ProduceResult(topicAndPartition, e)
|
new ProduceResult(topicAndPartition, e)
|
||||||
}
|
}
|
||||||
|
@ -298,7 +300,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
bytesReadable >= fetchRequest.minBytes ||
|
bytesReadable >= fetchRequest.minBytes ||
|
||||||
fetchRequest.numPartitions <= 0) {
|
fetchRequest.numPartitions <= 0) {
|
||||||
debug("Returning fetch response %s for fetch request with correlation id %d".format(dataRead.values.map(_.error).mkString(","), fetchRequest.correlationId))
|
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)))
|
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response)))
|
||||||
} else {
|
} else {
|
||||||
debug("Putting fetch request into purgatory")
|
debug("Putting fetch request into purgatory")
|
||||||
|
@ -329,22 +331,21 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
val partitionData =
|
val partitionData =
|
||||||
try {
|
try {
|
||||||
val (messages, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId)
|
val (messages, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId)
|
||||||
BrokerTopicStat.getBrokerTopicStat(topic).bytesOutRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(messages.sizeInBytes)
|
||||||
BrokerTopicStat.getBrokerAllTopicStat.bytesOutRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerAllTopicStats.bytesOutRate.mark(messages.sizeInBytes)
|
||||||
if (!isFetchFromFollower) {
|
if (!isFetchFromFollower) {
|
||||||
new FetchResponsePartitionData(ErrorMapping.NoError, offset, highWatermark, messages)
|
new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages)
|
||||||
} else {
|
} else {
|
||||||
debug("Leader %d for topic %s partition %d received fetch request from follower %d"
|
debug("Leader %d for topic %s partition %d received fetch request from follower %d"
|
||||||
.format(brokerId, topic, partition, fetchRequest.replicaId))
|
.format(brokerId, topic, partition, fetchRequest.replicaId))
|
||||||
new FetchResponsePartitionData(ErrorMapping.NoError, offset, highWatermark, messages)
|
new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages)
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case t: Throwable =>
|
case t: Throwable =>
|
||||||
BrokerTopicStat.getBrokerTopicStat(topic).failedFetchRequestRate.mark()
|
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
|
||||||
BrokerTopicStat.getBrokerAllTopicStat.failedFetchRequestRate.mark()
|
BrokerTopicStats.getBrokerAllTopicStats.failedFetchRequestRate.mark()
|
||||||
error("error when processing request " + (topic, partition, offset, fetchSize), t)
|
error("error when processing request " + (topic, partition, offset, fetchSize), t)
|
||||||
new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]),
|
new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty)
|
||||||
offset, -1L, MessageSet.Empty)
|
|
||||||
}
|
}
|
||||||
(TopicAndPartition(topic, partition), partitionData)
|
(TopicAndPartition(topic, partition), partitionData)
|
||||||
}
|
}
|
||||||
|
@ -418,7 +419,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil) )
|
(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)))
|
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -496,9 +497,13 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
try {
|
try {
|
||||||
/* check if auto creation of topics is turned on */
|
/* check if auto creation of topics is turned on */
|
||||||
if (config.autoCreateTopics) {
|
if (config.autoCreateTopics) {
|
||||||
CreateTopicCommand.createTopic(zkClient, topicAndMetadata.topic, config.numPartitions, config.defaultReplicationFactor)
|
try {
|
||||||
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
|
CreateTopicCommand.createTopic(zkClient, topicAndMetadata.topic, config.numPartitions, config.defaultReplicationFactor)
|
||||||
.format(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)
|
val newTopicMetadata = AdminUtils.fetchTopicMetadataFromZk(topicAndMetadata.topic, zkClient)
|
||||||
topicsMetadata += newTopicMetadata
|
topicsMetadata += newTopicMetadata
|
||||||
newTopicMetadata.errorCode match {
|
newTopicMetadata.errorCode match {
|
||||||
|
@ -516,7 +521,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
topicsMetadata.foreach(metadata => trace("Sending topic metadata " + metadata.toString))
|
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)))
|
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
|
* 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)
|
this.logIdent = "[FetchRequestPurgatory-%d] ".format(brokerId)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -573,7 +579,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
debug("Expiring fetch request %s.".format(delayed.fetch))
|
debug("Expiring fetch request %s.".format(delayed.fetch))
|
||||||
try {
|
try {
|
||||||
val topicData = readMessageSets(delayed.fetch)
|
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
|
val fromFollower = delayed.fetch.isFromFollower
|
||||||
delayedRequestMetrics.recordDelayedFetchExpired(fromFollower)
|
delayedRequestMetrics.recordDelayedFetchExpired(fromFollower)
|
||||||
requestChannel.sendResponse(new RequestChannel.Response(delayed.request, new FetchResponseSend(response)))
|
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))
|
(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(
|
requestChannel.sendResponse(new RequestChannel.Response(
|
||||||
request, new BoundedByteBufferSend(response)))
|
request, new BoundedByteBufferSend(response)))
|
||||||
|
@ -692,7 +698,8 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
/**
|
/**
|
||||||
* A holding pen for produce requests waiting to be satisfied.
|
* 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)
|
this.logIdent = "[ProducerRequestPurgatory-%d] ".format(brokerId)
|
||||||
|
|
||||||
protected def checkSatisfied(followerFetchRequestKey: RequestKey,
|
protected def checkSatisfied(followerFetchRequestKey: RequestKey,
|
||||||
|
|
|
@ -166,5 +166,11 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
|
||||||
|
|
||||||
/* the frequency with which the highwater mark is saved out to disk */
|
/* the frequency with which the highwater mark is saved out to disk */
|
||||||
val highWaterMarkCheckpointIntervalMs = props.getLong("replica.highwatermark.checkpoint.ms", 5000L)
|
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)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -79,14 +79,14 @@ class BrokerTopicMetrics(name: String) extends KafkaMetricsGroup {
|
||||||
val failedFetchRequestRate = newMeter(name + "FailedFetchRequestsPerSec", "requests", TimeUnit.SECONDS)
|
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 valueFactory = (k: String) => new BrokerTopicMetrics(k)
|
||||||
private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory))
|
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 + "-")
|
stats.getAndMaybePut(topic + "-")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,7 @@ import kafka.utils._
|
||||||
import java.util.concurrent._
|
import java.util.concurrent._
|
||||||
import atomic.AtomicBoolean
|
import atomic.AtomicBoolean
|
||||||
import org.I0Itec.zkclient.ZkClient
|
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
|
* 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.
|
* Forces some dynamic jmx beans to be registered on server startup.
|
||||||
*/
|
*/
|
||||||
private def registerStats() {
|
private def registerStats() {
|
||||||
BrokerTopicStat.getBrokerAllTopicStat()
|
BrokerTopicStats.getBrokerAllTopicStats()
|
||||||
ControllerStat.offlinePartitionRate
|
ControllerStats.offlinePartitionRate
|
||||||
ControllerStat.uncleanLeaderElectionRate
|
ControllerStats.uncleanLeaderElectionRate
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -43,8 +43,7 @@ class KafkaZooKeeper(config: KafkaConfig) extends Logging {
|
||||||
private def registerBrokerInZk() {
|
private def registerBrokerInZk() {
|
||||||
info("Registering broker " + brokerIdPath)
|
info("Registering broker " + brokerIdPath)
|
||||||
val hostName = config.hostName
|
val hostName = config.hostName
|
||||||
val creatorId = hostName + "-" + System.currentTimeMillis
|
ZkUtils.registerBrokerInZk(zkClient, config.brokerId, hostName, config.port)
|
||||||
ZkUtils.registerBrokerInZk(zkClient, config.brokerId, hostName, creatorId, config.port)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -28,7 +28,7 @@ class ReplicaFetcherThread(name:String,
|
||||||
brokerConfig: KafkaConfig,
|
brokerConfig: KafkaConfig,
|
||||||
replicaMgr: ReplicaManager)
|
replicaMgr: ReplicaManager)
|
||||||
extends AbstractFetcherThread(name = name,
|
extends AbstractFetcherThread(name = name,
|
||||||
clientId = FetchRequest.ReplicaFetcherClientId + "- %s:%d".format(sourceBroker.host, sourceBroker.port) ,
|
clientId = FetchRequest.ReplicaFetcherClientId,
|
||||||
sourceBroker = sourceBroker,
|
sourceBroker = sourceBroker,
|
||||||
socketTimeout = brokerConfig.replicaSocketTimeoutMs,
|
socketTimeout = brokerConfig.replicaSocketTimeoutMs,
|
||||||
socketBufferSize = brokerConfig.replicaSocketBufferSize,
|
socketBufferSize = brokerConfig.replicaSocketBufferSize,
|
||||||
|
|
|
@ -18,13 +18,13 @@
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
import java.util.LinkedList
|
|
||||||
import java.util.concurrent._
|
import java.util.concurrent._
|
||||||
import java.util.concurrent.atomic._
|
import java.util.concurrent.atomic._
|
||||||
import kafka.network._
|
import kafka.network._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import com.yammer.metrics.core.Gauge
|
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
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.
|
* 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 */
|
/* a list of requests watching each key */
|
||||||
private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers))
|
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(
|
newGauge(
|
||||||
"NumDelayedRequests",
|
"NumDelayedRequests",
|
||||||
new Gauge[Int] {
|
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)
|
private val expirationThread = Utils.daemonThread("request-expiration-task", expiredRequestReaper)
|
||||||
expirationThread.start()
|
expirationThread.start()
|
||||||
|
|
||||||
|
def purgeSatisfied() {
|
||||||
|
expiredRequestReaper.forcePurge()
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add a new delayed request watching the contained keys
|
* Add a new delayed request watching the contained keys
|
||||||
*/
|
*/
|
||||||
def watch(delayedRequest: T) {
|
def watch(delayedRequest: T) {
|
||||||
|
if (requestCounter.getAndIncrement() >= purgeInterval) {
|
||||||
|
requestCounter.set(0)
|
||||||
|
purgeSatisfied()
|
||||||
|
}
|
||||||
|
|
||||||
for(key <- delayedRequest.keys) {
|
for(key <- delayedRequest.keys) {
|
||||||
var lst = watchersFor(key)
|
var lst = watchersFor(key)
|
||||||
lst.add(delayedRequest)
|
lst.add(delayedRequest)
|
||||||
|
@ -125,37 +144,29 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
|
||||||
*/
|
*/
|
||||||
private class Watchers {
|
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 */
|
def numRequests = requests.size
|
||||||
var liveCount = 0.0
|
|
||||||
|
|
||||||
def add(t: T) {
|
def add(t: T) {
|
||||||
synchronized {
|
synchronized {
|
||||||
requests.add(t)
|
requests.add(t)
|
||||||
liveCount += 1
|
|
||||||
maybePurge()
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def maybePurge() {
|
def purgeSatisfied(): Int = {
|
||||||
if(requests.size > CleanupThresholdSize && liveCount / requests.size < CleanupThresholdPrct) {
|
synchronized {
|
||||||
val iter = requests.iterator()
|
val iter = requests.iterator()
|
||||||
|
var purged = 0
|
||||||
while(iter.hasNext) {
|
while(iter.hasNext) {
|
||||||
val curr = iter.next
|
val curr = iter.next
|
||||||
if(curr.satisfied.get())
|
if(curr.satisfied.get()) {
|
||||||
iter.remove()
|
iter.remove()
|
||||||
|
purged += 1
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
purged
|
||||||
}
|
|
||||||
|
|
||||||
def decLiveCount() {
|
|
||||||
synchronized {
|
|
||||||
liveCount -= 1
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -177,7 +188,6 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
|
||||||
val updated = curr.satisfied.compareAndSet(false, true)
|
val updated = curr.satisfied.compareAndSet(false, true)
|
||||||
if(updated == true) {
|
if(updated == true) {
|
||||||
response += curr
|
response += curr
|
||||||
liveCount -= 1
|
|
||||||
expiredRequestReaper.satisfyRequest()
|
expiredRequestReaper.satisfyRequest()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -193,17 +203,16 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
|
||||||
*/
|
*/
|
||||||
private class ExpiredRequestReaper extends Runnable with Logging {
|
private class ExpiredRequestReaper extends Runnable with Logging {
|
||||||
this.logIdent = "ExpiredRequestReaper-%d ".format(brokerId)
|
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 delayed = new DelayQueue[T]
|
||||||
private val running = new AtomicBoolean(true)
|
private val running = new AtomicBoolean(true)
|
||||||
private val shutdownLatch = new CountDownLatch(1)
|
private val shutdownLatch = new CountDownLatch(1)
|
||||||
private val needsPurge = new AtomicBoolean(false)
|
|
||||||
/* The count of elements in the delay queue that are unsatisfied */
|
/* The count of elements in the delay queue that are unsatisfied */
|
||||||
private [kafka] val unsatisfied = new AtomicInteger(0)
|
private [kafka] val unsatisfied = new AtomicInteger(0)
|
||||||
|
|
||||||
|
def numRequests = delayed.size()
|
||||||
|
|
||||||
/** Main loop for the expiry thread */
|
/** Main loop for the expiry thread */
|
||||||
def run() {
|
def run() {
|
||||||
while(running.get) {
|
while(running.get) {
|
||||||
|
@ -214,10 +223,10 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case ie: InterruptedException =>
|
case ie: InterruptedException =>
|
||||||
if(needsPurge.getAndSet(false)) {
|
val purged = purgeSatisfied()
|
||||||
val purged = purgeSatisfied()
|
debug("Purged %d requests from delay queue.".format(purged))
|
||||||
debug("Forced purge of " + purged + " requests from delay queue.")
|
val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum
|
||||||
}
|
debug("Purged %d (watcher) requests.".format(numPurgedFromWatchers))
|
||||||
case e: Exception =>
|
case e: Exception =>
|
||||||
error("Error in long poll expiry thread: ", e)
|
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) {
|
def enqueue(t: T) {
|
||||||
delayed.add(t)
|
delayed.add(t)
|
||||||
unsatisfied.incrementAndGet()
|
unsatisfied.incrementAndGet()
|
||||||
if(unsatisfied.get > CleanupThresholdSize && unsatisfied.get / delayed.size.toDouble < CleanupThresholdPrct)
|
|
||||||
forcePurge()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def forcePurge() {
|
def forcePurge() {
|
||||||
needsPurge.set(true)
|
|
||||||
expirationThread.interrupt()
|
expirationThread.interrupt()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -259,8 +265,6 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
|
||||||
val updated = curr.satisfied.compareAndSet(false, true)
|
val updated = curr.satisfied.compareAndSet(false, true)
|
||||||
if(updated) {
|
if(updated) {
|
||||||
unsatisfied.getAndDecrement()
|
unsatisfied.getAndDecrement()
|
||||||
for(key <- curr.keys)
|
|
||||||
watchersFor(key).decLiveCount()
|
|
||||||
return curr
|
return curr
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -284,4 +288,4 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0) exten
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -41,7 +41,7 @@ object ConsumerOffsetChecker extends Logging {
|
||||||
val brokerInfo = ZkUtils.readDataMaybeNull(zkClient, "/brokers/ids/%s".format(bid))._1
|
val brokerInfo = ZkUtils.readDataMaybeNull(zkClient, "/brokers/ids/%s".format(bid))._1
|
||||||
val consumer = brokerInfo match {
|
val consumer = brokerInfo match {
|
||||||
case BrokerIpPattern(ip, port) =>
|
case BrokerIpPattern(ip, port) =>
|
||||||
Some(new SimpleConsumer(ip, port.toInt, 10000, 100000))
|
Some(new SimpleConsumer(ip, port.toInt, 10000, 100000, "ConsumerOffsetChecker"))
|
||||||
case _ =>
|
case _ =>
|
||||||
error("Could not parse broker info %s".format(brokerInfo))
|
error("Could not parse broker info %s".format(brokerInfo))
|
||||||
None
|
None
|
||||||
|
|
|
@ -57,8 +57,8 @@ object DumpLogSegments {
|
||||||
val verifyOnly = if(options.has(verifyOpt)) true else false
|
val verifyOnly = if(options.has(verifyOpt)) true else false
|
||||||
val files = options.valueOf(filesOpt).split(",")
|
val files = options.valueOf(filesOpt).split(",")
|
||||||
|
|
||||||
val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Int, Int)]]
|
val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Long, Long)]]
|
||||||
val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Int, Int)]]
|
val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Long, Long)]]
|
||||||
|
|
||||||
for(arg <- files) {
|
for(arg <- files) {
|
||||||
val file = new File(arg)
|
val file = new File(arg)
|
||||||
|
@ -89,7 +89,7 @@ object DumpLogSegments {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* print out the contents of the index */
|
/* 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 startOffset = file.getName().split("\\.")(0).toLong
|
||||||
val logFileName = file.getAbsolutePath.split("\\.")(0) + Log.LogFileSuffix
|
val logFileName = file.getAbsolutePath.split("\\.")(0) + Log.LogFileSuffix
|
||||||
val logFile = new File(logFileName)
|
val logFile = new File(logFileName)
|
||||||
|
@ -100,8 +100,8 @@ object DumpLogSegments {
|
||||||
val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, messageSet.sizeInBytes())
|
val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, messageSet.sizeInBytes())
|
||||||
val messageAndOffset = partialFileMessageSet.head
|
val messageAndOffset = partialFileMessageSet.head
|
||||||
if(messageAndOffset.offset != entry.offset + index.baseOffset) {
|
if(messageAndOffset.offset != entry.offset + index.baseOffset) {
|
||||||
var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getName, List[(Int, Int)]())
|
var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getName, List[(Long, Long)]())
|
||||||
misMatchesSeq ::=((entry.offset + index.baseOffset, messageAndOffset.offset).asInstanceOf[(Int, Int)])
|
misMatchesSeq ::=(entry.offset + index.baseOffset, messageAndOffset.offset)
|
||||||
misMatchesForIndexFilesMap.put(file.getName, misMatchesSeq)
|
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
|
// 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 */
|
/* 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
|
val startOffset = file.getName().split("\\.")(0).toLong
|
||||||
println("Starting offset: " + startOffset)
|
println("Starting offset: " + startOffset)
|
||||||
val messageSet = new FileMessageSet(file)
|
val messageSet = new FileMessageSet(file)
|
||||||
|
@ -126,8 +126,8 @@ object DumpLogSegments {
|
||||||
lastOffset = messageAndOffset.offset
|
lastOffset = messageAndOffset.offset
|
||||||
// If it's uncompressed message, its offset must be lastOffset + 1 no matter last message is compressed or uncompressed
|
// 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) {
|
else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) {
|
||||||
var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getName, List[(Int, Int)]())
|
var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getName, List[(Long, Long)]())
|
||||||
nonConsecutivePairsSeq ::=((lastOffset, messageAndOffset.offset).asInstanceOf[(Int, Int)])
|
nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset)
|
||||||
nonConsecutivePairsForLogFilesMap.put(file.getName, nonConsecutivePairsSeq)
|
nonConsecutivePairsForLogFilesMap.put(file.getName, nonConsecutivePairsSeq)
|
||||||
}
|
}
|
||||||
lastOffset = messageAndOffset.offset
|
lastOffset = messageAndOffset.offset
|
||||||
|
|
|
@ -67,7 +67,7 @@ object GetOffsetShell {
|
||||||
val partition = options.valueOf(partitionOpt).intValue
|
val partition = options.valueOf(partitionOpt).intValue
|
||||||
var time = options.valueOf(timeOpt).longValue
|
var time = options.valueOf(timeOpt).longValue
|
||||||
val nOffsets = options.valueOf(nOffsetsOpt).intValue
|
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 topicAndPartition = TopicAndPartition(topic, partition)
|
||||||
val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets)))
|
val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets)))
|
||||||
val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets
|
val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets
|
||||||
|
|
|
@ -17,7 +17,6 @@
|
||||||
|
|
||||||
package kafka.tools
|
package kafka.tools
|
||||||
|
|
||||||
import kafka.message.Message
|
|
||||||
import joptsimple.OptionParser
|
import joptsimple.OptionParser
|
||||||
import kafka.utils.{Utils, CommandLineUtils, Logging}
|
import kafka.utils.{Utils, CommandLineUtils, Logging}
|
||||||
import kafka.producer.{KeyedMessage, ProducerConfig, Producer}
|
import kafka.producer.{KeyedMessage, ProducerConfig, Producer}
|
||||||
|
|
|
@ -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()
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -24,7 +24,7 @@ import kafka.producer.{KeyedMessage, ProducerConfig, Producer}
|
||||||
import kafka.consumer._
|
import kafka.consumer._
|
||||||
import kafka.utils.{Logging, ZkUtils}
|
import kafka.utils.{Logging, ZkUtils}
|
||||||
import kafka.api.OffsetRequest
|
import kafka.api.OffsetRequest
|
||||||
import kafka.message.{CompressionCodec, Message}
|
import kafka.message.CompressionCodec
|
||||||
|
|
||||||
object ReplayLogProducer extends Logging {
|
object ReplayLogProducer extends Logging {
|
||||||
|
|
||||||
|
|
|
@ -19,12 +19,10 @@ package kafka.tools
|
||||||
|
|
||||||
import joptsimple._
|
import joptsimple._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import kafka.producer.ProducerConfig
|
|
||||||
import kafka.consumer._
|
import kafka.consumer._
|
||||||
import kafka.client.ClientUtils
|
import kafka.client.ClientUtils
|
||||||
import kafka.api.{OffsetRequest, FetchRequestBuilder, Request}
|
import kafka.api.{OffsetRequest, FetchRequestBuilder, Request}
|
||||||
import kafka.cluster.Broker
|
import kafka.cluster.Broker
|
||||||
import java.util.Properties
|
|
||||||
import scala.collection.JavaConversions._
|
import scala.collection.JavaConversions._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -74,7 +72,7 @@ object SimpleConsumerShell extends Logging {
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("class")
|
.describedAs("class")
|
||||||
.ofType(classOf[String])
|
.ofType(classOf[String])
|
||||||
.defaultsTo(classOf[NewlineMessageFormatter].getName)
|
.defaultsTo(classOf[DefaultMessageFormatter].getName)
|
||||||
val messageFormatterArgOpt = parser.accepts("property")
|
val messageFormatterArgOpt = parser.accepts("property")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("prop")
|
.describedAs("prop")
|
||||||
|
@ -127,7 +125,7 @@ object SimpleConsumerShell extends Logging {
|
||||||
// getting topic metadata
|
// getting topic metadata
|
||||||
info("Getting topic metatdata...")
|
info("Getting topic metatdata...")
|
||||||
val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt))
|
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)) {
|
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.err.println(("Error: no valid topic metadata for topic: %s, " + "what we get from server is only: %s").format(topic, topicsMetadata))
|
||||||
System.exit(1)
|
System.exit(1)
|
||||||
|
@ -167,7 +165,7 @@ object SimpleConsumerShell extends Logging {
|
||||||
System.exit(1)
|
System.exit(1)
|
||||||
}
|
}
|
||||||
if(startingOffset < 0)
|
if(startingOffset < 0)
|
||||||
startingOffset = SimpleConsumer.earliestOrLatestOffset(fetchTargetBroker, topic, partitionId, startingOffset, false)
|
startingOffset = SimpleConsumer.earliestOrLatestOffset(fetchTargetBroker, topic, partitionId, startingOffset, clientId, false)
|
||||||
|
|
||||||
// initializing formatter
|
// initializing formatter
|
||||||
val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter]
|
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]"
|
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))
|
.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() {
|
val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() {
|
||||||
def run() {
|
def run() {
|
||||||
var offset = startingOffset
|
var offset = startingOffset
|
||||||
|
|
|
@ -65,7 +65,7 @@ object UpdateOffsetsInZK {
|
||||||
|
|
||||||
ZkUtils.getBrokerInfo(zkClient, broker) match {
|
ZkUtils.getBrokerInfo(zkClient, broker) match {
|
||||||
case Some(brokerInfo) =>
|
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 topicAndPartition = TopicAndPartition(topic, partition)
|
||||||
val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1)))
|
val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1)))
|
||||||
val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
|
val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
|
||||||
|
|
|
@ -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 -")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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 ....
|
* key value pairs. the format of allCSVal is key1:val1, key2:val2 ....
|
||||||
*/
|
*/
|
||||||
def parseCsvMap(str: String): Map[String, String] = {
|
def parseCsvMap(str: String): Map[String, String] = {
|
||||||
|
|
|
@ -180,11 +180,11 @@ object ZkUtils extends Logging {
|
||||||
replicas.contains(brokerId.toString)
|
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 brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id
|
||||||
val broker = new Broker(id, creator, host, port)
|
val broker = new Broker(id, host, port)
|
||||||
try {
|
try {
|
||||||
createEphemeralPathExpectConflict(zkClient, brokerIdPath, broker.getZKString)
|
createEphemeralPathExpectConflict(zkClient, brokerIdPath, broker.getZkString)
|
||||||
} catch {
|
} catch {
|
||||||
case e: ZkNodeExistsException =>
|
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.")
|
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.")
|
||||||
|
|
|
@ -17,7 +17,6 @@
|
||||||
|
|
||||||
package kafka
|
package kafka
|
||||||
|
|
||||||
import message.Message
|
|
||||||
import org.apache.log4j.PropertyConfigurator
|
import org.apache.log4j.PropertyConfigurator
|
||||||
import kafka.utils.Logging
|
import kafka.utils.Logging
|
||||||
import serializer.Encoder
|
import serializer.Encoder
|
||||||
|
|
|
@ -18,7 +18,6 @@
|
||||||
package kafka
|
package kafka
|
||||||
|
|
||||||
import consumer._
|
import consumer._
|
||||||
import message.Message
|
|
||||||
import utils.Utils
|
import utils.Utils
|
||||||
import java.util.concurrent.CountDownLatch
|
import java.util.concurrent.CountDownLatch
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,7 @@ import org.scalatest.junit.JUnit3Suite
|
||||||
import kafka.zk.ZooKeeperTestHarness
|
import kafka.zk.ZooKeeperTestHarness
|
||||||
import kafka.server.KafkaConfig
|
import kafka.server.KafkaConfig
|
||||||
import kafka.utils.{ZkUtils, TestUtils}
|
import kafka.utils.{ZkUtils, TestUtils}
|
||||||
import kafka.common.{ErrorMapping, TopicAndPartition}
|
import kafka.common.{TopicExistsException, ErrorMapping, TopicAndPartition}
|
||||||
|
|
||||||
|
|
||||||
class AdminTest extends JUnit3Suite with ZooKeeperTestHarness {
|
class AdminTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
|
@ -170,7 +170,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
AdminUtils.createTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient)
|
AdminUtils.createTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient)
|
||||||
fail("shouldn't be able to create a topic already exists")
|
fail("shouldn't be able to create a topic already exists")
|
||||||
} catch {
|
} catch {
|
||||||
case e: AdministrationException => // this is good
|
case e: TopicExistsException => // this is good
|
||||||
case e2 => throw e2
|
case e2 => throw e2
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -374,33 +374,34 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
var controllerId = ZkUtils.getController(zkClient)
|
var controllerId = ZkUtils.getController(zkClient)
|
||||||
var controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
|
var controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
|
||||||
var partitionsRemaining = controller.shutdownBroker(2)
|
var partitionsRemaining = controller.shutdownBroker(2)
|
||||||
assertEquals(0, partitionsRemaining)
|
try {
|
||||||
var topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
|
assertEquals(0, partitionsRemaining)
|
||||||
var leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
|
var topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
|
||||||
assertTrue(leaderAfterShutdown != leaderBeforeShutdown)
|
var leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
|
||||||
assertEquals(2, topicMetadata.partitionsMetadata.head.isr.size)
|
assertTrue(leaderAfterShutdown != leaderBeforeShutdown)
|
||||||
|
|
||||||
leaderBeforeShutdown = leaderAfterShutdown
|
leaderBeforeShutdown = leaderAfterShutdown
|
||||||
controllerId = ZkUtils.getController(zkClient)
|
controllerId = ZkUtils.getController(zkClient)
|
||||||
controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
|
controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
|
||||||
partitionsRemaining = controller.shutdownBroker(1)
|
partitionsRemaining = controller.shutdownBroker(1)
|
||||||
assertEquals(0, partitionsRemaining)
|
assertEquals(0, partitionsRemaining)
|
||||||
topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
|
topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
|
||||||
leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
|
leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
|
||||||
assertTrue(leaderAfterShutdown != leaderBeforeShutdown)
|
assertTrue(leaderAfterShutdown != leaderBeforeShutdown)
|
||||||
assertEquals(1, topicMetadata.partitionsMetadata.head.isr.size)
|
assertEquals(1, controller.controllerContext.allLeaders(TopicAndPartition("test", 1)).leaderAndIsr.isr.size)
|
||||||
|
|
||||||
leaderBeforeShutdown = leaderAfterShutdown
|
leaderBeforeShutdown = leaderAfterShutdown
|
||||||
controllerId = ZkUtils.getController(zkClient)
|
controllerId = ZkUtils.getController(zkClient)
|
||||||
controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
|
controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
|
||||||
partitionsRemaining = controller.shutdownBroker(0)
|
partitionsRemaining = controller.shutdownBroker(0)
|
||||||
assertEquals(1, partitionsRemaining)
|
assertEquals(1, partitionsRemaining)
|
||||||
topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
|
topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient)
|
||||||
leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
|
leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id
|
||||||
assertTrue(leaderAfterShutdown == leaderBeforeShutdown)
|
assertTrue(leaderAfterShutdown == leaderBeforeShutdown)
|
||||||
assertEquals(1, topicMetadata.partitionsMetadata.head.isr.size)
|
assertEquals(1, controller.controllerContext.allLeaders(TopicAndPartition("test", 1)).leaderAndIsr.isr.size)
|
||||||
|
} finally {
|
||||||
servers.foreach(_.shutdown())
|
servers.foreach(_.shutdown())
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def checkIfReassignPartitionPathExists(): Boolean = {
|
private def checkIfReassignPartitionPathExists(): Boolean = {
|
||||||
|
|
|
@ -75,10 +75,11 @@ object SerializationTestUtils{
|
||||||
TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100)
|
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 brokers = List(new Broker(0, "localhost", 1011), new Broker(1, "localhost", 1012), new Broker(2, "localhost", 1013))
|
||||||
private val partitionMetaData1 = new PartitionMetadata(1, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty)
|
private val partitionMetaData0 = new PartitionMetadata(0, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 0)
|
||||||
private val partitionMetaData2 = new PartitionMetadata(2, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty)
|
private val partitionMetaData1 = new PartitionMetadata(1, Some(brokers.head), replicas = brokers, isr = brokers.tail, errorCode = 1)
|
||||||
private val partitionMetaData3 = new PartitionMetadata(3, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty)
|
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 partitionMetaDataSeq = Seq(partitionMetaData0, partitionMetaData1, partitionMetaData2, partitionMetaData3)
|
||||||
private val topicmetaData1 = new TopicMetadata(topic1, partitionMetaDataSeq)
|
private val topicmetaData1 = new TopicMetadata(topic1, partitionMetaDataSeq)
|
||||||
private val topicmetaData2 = new TopicMetadata(topic2, partitionMetaDataSeq)
|
private val topicmetaData2 = new TopicMetadata(topic2, partitionMetaDataSeq)
|
||||||
|
@ -104,7 +105,7 @@ object SerializationTestUtils{
|
||||||
def createTestStopReplicaResponse() : StopReplicaResponse = {
|
def createTestStopReplicaResponse() : StopReplicaResponse = {
|
||||||
val responseMap = Map(((topic1, 0), ErrorMapping.NoError),
|
val responseMap = Map(((topic1, 0), ErrorMapping.NoError),
|
||||||
((topic2, 0), ErrorMapping.NoError))
|
((topic2, 0), ErrorMapping.NoError))
|
||||||
new StopReplicaResponse(1, responseMap.toMap)
|
new StopReplicaResponse(0, responseMap.toMap)
|
||||||
}
|
}
|
||||||
|
|
||||||
def createTestProducerRequest: ProducerRequest = {
|
def createTestProducerRequest: ProducerRequest = {
|
||||||
|
@ -112,7 +113,7 @@ object SerializationTestUtils{
|
||||||
}
|
}
|
||||||
|
|
||||||
def createTestProducerResponse: ProducerResponse =
|
def createTestProducerResponse: ProducerResponse =
|
||||||
ProducerResponse(1, 1, Map(
|
ProducerResponse(1, Map(
|
||||||
TopicAndPartition(topic1, 0) -> ProducerResponseStatus(0.toShort, 10001),
|
TopicAndPartition(topic1, 0) -> ProducerResponseStatus(0.toShort, 10001),
|
||||||
TopicAndPartition(topic2, 0) -> ProducerResponseStatus(0.toShort, 20001)
|
TopicAndPartition(topic2, 0) -> ProducerResponseStatus(0.toShort, 20001)
|
||||||
))
|
))
|
||||||
|
@ -122,7 +123,7 @@ object SerializationTestUtils{
|
||||||
}
|
}
|
||||||
|
|
||||||
def createTestFetchResponse: FetchResponse = {
|
def createTestFetchResponse: FetchResponse = {
|
||||||
FetchResponse(1, 1, topicDataFetchResponse)
|
FetchResponse(1, topicDataFetchResponse)
|
||||||
}
|
}
|
||||||
|
|
||||||
def createTestOffsetRequest = new OffsetRequest(
|
def createTestOffsetRequest = new OffsetRequest(
|
||||||
|
@ -131,17 +132,17 @@ object SerializationTestUtils{
|
||||||
)
|
)
|
||||||
|
|
||||||
def createTestOffsetResponse: OffsetResponse = {
|
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)))
|
TopicAndPartition(topic1, 1) -> PartitionOffsetsResponse(ErrorMapping.NoError, Seq(1000l, 2000l, 3000l, 4000l)))
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
def createTestTopicMetadataRequest: TopicMetadataRequest = {
|
def createTestTopicMetadataRequest: TopicMetadataRequest = {
|
||||||
new TopicMetadataRequest(1, "client 1", Seq(topic1, topic2))
|
new TopicMetadataRequest(1, "client 1", Seq(topic1, topic2), 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
def createTestTopicMetadataResponse: TopicMetadataResponse = {
|
def createTestTopicMetadataResponse: TopicMetadataResponse = {
|
||||||
new TopicMetadataResponse(1, Seq(topicmetaData1, topicmetaData2))
|
new TopicMetadataResponse(Seq(topicmetaData1, topicmetaData2), 1)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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.")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -15,11 +15,11 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package kafka.utils
|
package unit.kafka.common
|
||||||
|
|
||||||
import junit.framework.Assert._
|
import junit.framework.Assert._
|
||||||
import collection.mutable.ArrayBuffer
|
import collection.mutable.ArrayBuffer
|
||||||
import kafka.common.InvalidTopicException
|
import kafka.common.{Topic, InvalidTopicException}
|
||||||
import org.junit.Test
|
import org.junit.Test
|
||||||
|
|
||||||
class TopicTest {
|
class TopicTest {
|
||||||
|
@ -34,7 +34,7 @@ class TopicTest {
|
||||||
invalidTopicNames += longName
|
invalidTopicNames += longName
|
||||||
val badChars = Array('/', '\\', ',', '\0', ':', "\"", '\'', ';', '*', '?', '.')
|
val badChars = Array('/', '\\', ',', '\0', ':', "\"", '\'', ';', '*', '?', '.')
|
||||||
for (weirdChar <- badChars) {
|
for (weirdChar <- badChars) {
|
||||||
invalidTopicNames += "Is" + weirdChar + "funny"
|
invalidTopicNames += "Is" + weirdChar + "illegal"
|
||||||
}
|
}
|
||||||
|
|
||||||
for (i <- 0 until invalidTopicNames.size) {
|
for (i <- 0 until invalidTopicNames.size) {
|
|
@ -47,7 +47,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness {
|
||||||
val group = "group1"
|
val group = "group1"
|
||||||
val consumer0 = "consumer0"
|
val consumer0 = "consumer0"
|
||||||
val consumedOffset = 5
|
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 queue = new LinkedBlockingQueue[FetchedDataChunk]
|
||||||
val topicInfos = configs.map(c => new PartitionTopicInfo(topic,
|
val topicInfos = configs.map(c => new PartitionTopicInfo(topic,
|
||||||
c.brokerId,
|
c.brokerId,
|
||||||
|
@ -55,7 +55,8 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness {
|
||||||
queue,
|
queue,
|
||||||
new AtomicLong(consumedOffset),
|
new AtomicLong(consumedOffset),
|
||||||
new AtomicLong(0),
|
new AtomicLong(0),
|
||||||
new AtomicInteger(0)))
|
new AtomicInteger(0),
|
||||||
|
""))
|
||||||
val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0))
|
val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0))
|
||||||
|
|
||||||
override def setUp() {
|
override def setUp() {
|
||||||
|
@ -78,8 +79,9 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness {
|
||||||
consumerConfig.consumerTimeoutMs,
|
consumerConfig.consumerTimeoutMs,
|
||||||
new StringDecoder(),
|
new StringDecoder(),
|
||||||
new StringDecoder(),
|
new StringDecoder(),
|
||||||
enableShallowIterator = false)
|
enableShallowIterator = false,
|
||||||
var receivedMessages = (0 until 5).map(i => iter.next.message).toList
|
clientId = "")
|
||||||
|
val receivedMessages = (0 until 5).map(i => iter.next.message).toList
|
||||||
|
|
||||||
assertFalse(iter.hasNext)
|
assertFalse(iter.hasNext)
|
||||||
assertEquals(1, queue.size) // This is only the shutdown command.
|
assertEquals(1, queue.size) // This is only the shutdown command.
|
||||||
|
|
|
@ -24,7 +24,6 @@ import kafka.server._
|
||||||
import org.apache.log4j.{Level, Logger}
|
import org.apache.log4j.{Level, Logger}
|
||||||
import org.scalatest.junit.JUnit3Suite
|
import org.scalatest.junit.JUnit3Suite
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import kafka.message.Message
|
|
||||||
import kafka.serializer._
|
import kafka.serializer._
|
||||||
import kafka.producer.{Producer, KeyedMessage}
|
import kafka.producer.{Producer, KeyedMessage}
|
||||||
|
|
||||||
|
|
|
@ -23,7 +23,6 @@ import scala.collection._
|
||||||
import junit.framework.Assert._
|
import junit.framework.Assert._
|
||||||
|
|
||||||
import kafka.cluster._
|
import kafka.cluster._
|
||||||
import kafka.message._
|
|
||||||
import kafka.server._
|
import kafka.server._
|
||||||
import org.scalatest.junit.JUnit3Suite
|
import org.scalatest.junit.JUnit3Suite
|
||||||
import kafka.consumer._
|
import kafka.consumer._
|
||||||
|
@ -41,7 +40,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
|
||||||
yield new KafkaConfig(props)
|
yield new KafkaConfig(props)
|
||||||
val messages = new mutable.HashMap[Int, Seq[Array[Byte]]]
|
val messages = new mutable.HashMap[Int, Seq[Array[Byte]]]
|
||||||
val topic = "topic"
|
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 shutdown = ZookeeperConsumerConnector.shutdownCommand
|
||||||
val queue = new LinkedBlockingQueue[FetchedDataChunk]
|
val queue = new LinkedBlockingQueue[FetchedDataChunk]
|
||||||
val topicInfos = configs.map(c => new PartitionTopicInfo(topic,
|
val topicInfos = configs.map(c => new PartitionTopicInfo(topic,
|
||||||
|
@ -50,7 +49,8 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
|
||||||
queue,
|
queue,
|
||||||
new AtomicLong(0),
|
new AtomicLong(0),
|
||||||
new AtomicLong(0),
|
new AtomicLong(0),
|
||||||
new AtomicInteger(0)))
|
new AtomicInteger(0),
|
||||||
|
""))
|
||||||
|
|
||||||
var fetcher: ConsumerFetcherManager = null
|
var fetcher: ConsumerFetcherManager = null
|
||||||
|
|
||||||
|
@ -84,7 +84,9 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
|
||||||
def sendMessages(messagesPerNode: Int): Int = {
|
def sendMessages(messagesPerNode: Int): Int = {
|
||||||
var count = 0
|
var count = 0
|
||||||
for(conf <- configs) {
|
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
|
val ms = 0.until(messagesPerNode).map(x => (conf.brokerId * 5 + x).toString.getBytes).toArray
|
||||||
messages += conf.brokerId -> ms
|
messages += conf.brokerId -> ms
|
||||||
producer.send(ms.map(m => KeyedMessage[String, Array[Byte]](topic, topic, m)):_*)
|
producer.send(ms.map(m => KeyedMessage[String, Array[Byte]](topic, topic, m)):_*)
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
package kafka.integration
|
package kafka.integration
|
||||||
|
|
||||||
import kafka.api.FetchRequestBuilder
|
import kafka.api.FetchRequestBuilder
|
||||||
import kafka.message.{Message, ByteBufferMessageSet}
|
import kafka.message.ByteBufferMessageSet
|
||||||
import kafka.server.{KafkaRequestHandler, KafkaConfig}
|
import kafka.server.{KafkaRequestHandler, KafkaConfig}
|
||||||
import org.apache.log4j.{Level, Logger}
|
import org.apache.log4j.{Level, Logger}
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
|
|
|
@ -25,7 +25,6 @@ import java.util.Properties
|
||||||
import kafka.utils.Utils
|
import kafka.utils.Utils
|
||||||
import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
|
import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
|
||||||
import kafka.serializer._
|
import kafka.serializer._
|
||||||
import kafka.message.Message
|
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import org.apache.log4j.{Level, Logger}
|
import org.apache.log4j.{Level, Logger}
|
||||||
import org.I0Itec.zkclient.ZkClient
|
import org.I0Itec.zkclient.ZkClient
|
||||||
|
|
|
@ -21,7 +21,6 @@ import kafka.consumer.SimpleConsumer
|
||||||
import org.scalatest.junit.JUnit3Suite
|
import org.scalatest.junit.JUnit3Suite
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import kafka.producer.{ProducerConfig, Producer}
|
import kafka.producer.{ProducerConfig, Producer}
|
||||||
import kafka.message.Message
|
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import kafka.serializer._
|
import kafka.serializer._
|
||||||
|
|
||||||
|
@ -44,10 +43,7 @@ trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarnes
|
||||||
props.put("producer.request.required.acks", "-1")
|
props.put("producer.request.required.acks", "-1")
|
||||||
props.put("serializer.class", classOf[StringEncoder].getName.toString)
|
props.put("serializer.class", classOf[StringEncoder].getName.toString)
|
||||||
producer = new Producer(new ProducerConfig(props))
|
producer = new Producer(new ProducerConfig(props))
|
||||||
consumer = new SimpleConsumer(host,
|
consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "")
|
||||||
port,
|
|
||||||
1000000,
|
|
||||||
64*1024)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override def tearDown() {
|
override def tearDown() {
|
||||||
|
|
|
@ -97,7 +97,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
private def mockLogManagerAndTestTopic(topic: String): Seq[TopicMetadata] = {
|
private def mockLogManagerAndTestTopic(topic: String): Seq[TopicMetadata] = {
|
||||||
// topic metadata request only requires 1 call from the replica manager
|
// topic metadata request only requires 1 call from the replica manager
|
||||||
val replicaManager = EasyMock.createMock(classOf[ReplicaManager])
|
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)
|
EasyMock.replay(replicaManager)
|
||||||
|
|
||||||
// create a topic metadata request
|
// create a topic metadata request
|
||||||
|
|
|
@ -29,7 +29,7 @@ import kafka.producer.KeyedMessage
|
||||||
import kafka.javaapi.producer.Producer
|
import kafka.javaapi.producer.Producer
|
||||||
import kafka.utils.IntEncoder
|
import kafka.utils.IntEncoder
|
||||||
import kafka.utils.TestUtils._
|
import kafka.utils.TestUtils._
|
||||||
import kafka.utils.{Utils, Logging, TestUtils}
|
import kafka.utils.{Logging, TestUtils}
|
||||||
import kafka.consumer.{KafkaStream, ConsumerConfig}
|
import kafka.consumer.{KafkaStream, ConsumerConfig}
|
||||||
import kafka.zk.ZooKeeperTestHarness
|
import kafka.zk.ZooKeeperTestHarness
|
||||||
|
|
||||||
|
|
|
@ -438,6 +438,33 @@ class LogTest extends JUnitSuite {
|
||||||
log.append(set)
|
log.append(set)
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
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
|
* Verify that truncation works correctly after re-opening the log
|
||||||
|
|
|
@ -24,7 +24,6 @@ import kafka.server.{KafkaConfig, KafkaServer}
|
||||||
import kafka.utils.{TestUtils, Utils, Logging}
|
import kafka.utils.{TestUtils, Utils, Logging}
|
||||||
import junit.framework.Assert._
|
import junit.framework.Assert._
|
||||||
import kafka.api.FetchRequestBuilder
|
import kafka.api.FetchRequestBuilder
|
||||||
import kafka.message.Message
|
|
||||||
import kafka.producer.async.MissingConfigException
|
import kafka.producer.async.MissingConfigException
|
||||||
import kafka.serializer.Encoder
|
import kafka.serializer.Encoder
|
||||||
import kafka.zk.ZooKeeperTestHarness
|
import kafka.zk.ZooKeeperTestHarness
|
||||||
|
@ -57,7 +56,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with
|
||||||
logDirZk = new File(logDirZkPath)
|
logDirZk = new File(logDirZkPath)
|
||||||
config = new KafkaConfig(propsZk)
|
config = new KafkaConfig(propsZk)
|
||||||
serverZk = TestUtils.createServer(config);
|
serverZk = TestUtils.createServer(config);
|
||||||
simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64*1024)
|
simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64*1024, "")
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue