merge from 0.8 and resolve conflicts

This commit is contained in:
Jun Rao 2013-01-11 18:25:56 -08:00
commit 9249b76d1d
98 changed files with 854 additions and 758 deletions

View File

@ -23,7 +23,7 @@ zk.connect=127.0.0.1:2181
zk.connectiontimeout.ms=1000000
#consumer group id
groupid=test-consumer-group
group.id=test-consumer-group
#consumer timeout
#consumer.timeout.ms=5000

View File

@ -4,30 +4,53 @@
# 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.
log4j.rootLogger=INFO, stdout
log4j.rootLogger=OFF, stdout
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n
#log4j.appender.fileAppender=org.apache.log4j.FileAppender
#log4j.appender.fileAppender.File=kafka-request.log
#log4j.appender.fileAppender.layout=org.apache.log4j.PatternLayout
#log4j.appender.fileAppender.layout.ConversionPattern= %-4r [%t] %-5p %c %x - %m%n
log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender
log4j.appender.kafkaAppender.DatePattern='.'yyyy-MM-dd-HH
log4j.appender.kafkaAppender.File=server.log
log4j.appender.kafkaAppender.layout=org.apache.log4j.PatternLayout
log4j.appender.kafkaAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
log4j.appender.stateChangeAppender=org.apache.log4j.DailyRollingFileAppender
log4j.appender.stateChangeAppender.DatePattern='.'yyyy-MM-dd-HH
log4j.appender.stateChangeAppender.File=state-change.log
log4j.appender.stateChangeAppender.layout=org.apache.log4j.PatternLayout
log4j.appender.stateChangeAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
log4j.appender.requestAppender=org.apache.log4j.DailyRollingFileAppender
log4j.appender.requestAppender.DatePattern='.'yyyy-MM-dd-HH
log4j.appender.requestAppender.File=kafka-request.log
log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout
log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
# Turn on all our debugging info
log4j.logger.kafka.perf=DEBUG
log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG
#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender
#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender
#log4j.logger.kafka.perf=DEBUG, kafkaAppender
#log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG, kafkaAppender
#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG
log4j.logger.kafka=ERROR
log4j.logger.kafka.perf=DEBUG
log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG
log4j.logger.kafka.network.RequestChannel$=TRACE, requestAppender
log4j.additivity.kafka.network.RequestChannel$=false
#log4j.logger.kafka.server.KafkaApis=TRACE, requestAppender
#log4j.additivity.kafka.server.KafkaApis=false
log4j.logger.kafka.request.logger=TRACE, requestAppender
log4j.additivity.kafka.request.logger=false
log4j.logger.kafka.controller=TRACE, stateChangeAppender
log4j.additivity.kafka.controller=false

View File

@ -36,35 +36,18 @@ serializer.class=kafka.serializer.StringEncoder
# allow topic level compression
#compressed.topics=
# max message size; messages larger than that size are discarded; default is 1000000
#max.message.size=
############################# Async Producer #############################
# maximum time, in milliseconds, for buffering data on the producer queue
#queue.time=
#queue.buffering.max.ms=
# the maximum size of the blocking queue for buffering on the producer
#queue.size=
#queue.buffering.max.messages=
# Timeout for event enqueue:
# 0: events will be enqueued immediately or dropped if the queue is full
# -ve: enqueue will block indefinitely if the queue is full
# +ve: enqueue will block up to this many milliseconds if the queue is full
#queue.enqueueTimeout.ms=
#queue.enqueue.timeout.ms=
# the number of messages batched at the producer
#batch.size=
# the callback handler for one or multiple events
#callback.handler=
# properties required to initialize the callback handler
#callback.handler.props=
# the handler for events
#event.handler=
# properties required to initialize the event handler
#event.handler.props=
#batch.num.messages=

View File

@ -17,7 +17,7 @@
############################# Server Basics #############################
# The id of the broker. This must be set to a unique integer for each broker.
brokerid=0
broker.id=0
############################# Socket Server Settings #############################
@ -27,22 +27,22 @@ port=9092
# Hostname the broker will bind to and advertise to producers and consumers.
# If not set, the server will bind to all interfaces and advertise the value returned from
# from java.net.InetAddress.getCanonicalHostName().
#hostname=localhost
#host.name=localhost
# The number of threads handling network requests
network.threads=2
num.network.threads=2
# The number of threads doing disk I/O
io.threads=2
num.io.threads=2
# The send buffer (SO_SNDBUF) used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# The receive buffer (SO_RCVBUF) used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# The maximum size of a request that the socket server will accept (protection against OOM)
max.socket.request.bytes=104857600
socket.request.max.bytes=104857600
############################# Log Basics #############################
@ -54,9 +54,6 @@ log.dir=/tmp/kafka-logs
# for consumption, but also mean more files.
num.partitions=1
# Overrides for for the default given by num.partitions on a per-topic basis
#topic.partition.count.map=topic1:3, topic2:4
############################# Log Flush Policy #############################
# The following configurations control the flush of data to disk. This is the most
@ -69,16 +66,13 @@ num.partitions=1
# every N messages (or both). This can be done globally and overridden on a per-topic basis.
# The number of messages to accept before forcing a flush of data to disk
log.flush.interval=10000
log.flush.interval.messages=10000
# The maximum amount of time a message can sit in a log before we force a flush
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# Per-topic overrides for log.default.flush.interval.ms
#topic.flush.intervals.ms=topic1:1000, topic2:3000
# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
log.default.flush.scheduler.interval.ms=1000
# Per-topic overrides for log.flush.interval.ms
#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
############################# Log Retention Policy #############################
@ -91,11 +85,11 @@ log.default.flush.scheduler.interval.ms=1000
log.retention.hours=168
# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
# segments don't drop below log.retention.size.
#log.retention.size=1073741824
# segments don't drop below log.retention.bytes.
#log.retention.bytes=1073741824
# The maximum size of a log segment file. When this size is reached a new log segment will be created.
log.file.size=536870912
log.segment.bytes=536870912
# The interval at which log segments are checked to see if they can be deleted according
# to the retention policies

View File

@ -71,7 +71,7 @@ public class DataGenerator {
System.out.println("server uri:" + _uri.toString());
Properties producerProps = new Properties();
producerProps.put("broker.list", String.format("%s:%d", _uri.getHost(), _uri.getPort()));
producerProps.put("buffer.size", String.valueOf(TCP_BUFFER_SIZE));
producerProps.put("send.buffer.bytes", String.valueOf(TCP_BUFFER_SIZE));
producerProps.put("connect.timeout.ms", String.valueOf(CONNECT_TIMEOUT));
producerProps.put("reconnect.interval", String.valueOf(RECONNECT_INTERVAL));

View File

@ -119,10 +119,9 @@ public class KafkaOutputFormat<W extends BytesWritable> extends OutputFormat<Nul
job.setInt("kafka.output.compression_codec", compressionCodec);
props.setProperty("producer.type", producerType);
props.setProperty("buffer.size", Integer.toString(bufSize));
props.setProperty("send.buffer.bytes", Integer.toString(bufSize));
props.setProperty("connect.timeout.ms", Integer.toString(timeout));
props.setProperty("reconnect.interval", Integer.toString(interval));
props.setProperty("max.message.size", Integer.toString(maxSize));
props.setProperty("compression.codec", Integer.toString(compressionCodec));
if (uri.getScheme().equals("kafka")) {

View File

@ -120,8 +120,9 @@ case class LeaderAndIsrRequest (versionId: Short,
controllerEpoch: Int)
extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) {
def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], liveBrokers: Set[Broker], controllerEpoch: Int) = {
this(LeaderAndIsrRequest.CurrentVersion, 0, LeaderAndIsrRequest.DefaultClientId, LeaderAndIsrRequest.DefaultAckTimeout,
def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], liveBrokers: Set[Broker],
controllerEpoch: Int, correlationId: Int) = {
this(LeaderAndIsrRequest.CurrentVersion, correlationId, LeaderAndIsrRequest.DefaultClientId, LeaderAndIsrRequest.DefaultAckTimeout,
partitionStateInfos, liveBrokers, controllerEpoch)
}

View File

@ -23,7 +23,6 @@ import scala.collection.Map
import kafka.common.TopicAndPartition
import kafka.api.ApiUtils._
object ProducerRequest {
val CurrentVersion = 0.shortValue

View File

@ -59,8 +59,8 @@ case class StopReplicaRequest(versionId: Short,
controllerEpoch: Int)
extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) {
def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerEpoch: Int) = {
this(StopReplicaRequest.CurrentVersion, 0, StopReplicaRequest.DefaultClientId, StopReplicaRequest.DefaultAckTimeout,
def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerEpoch: Int, correlationId: Int) = {
this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId, StopReplicaRequest.DefaultAckTimeout,
deletePartitions, partitions, controllerEpoch)
}

View File

@ -39,30 +39,32 @@ object TopicMetadataRequest extends Logging {
val topics = new ListBuffer[String]()
for(i <- 0 until numTopics)
topics += readShortString(buffer)
val topicsList = topics.toList
debug("topic = %s".format(topicsList.head))
new TopicMetadataRequest(versionId, clientId, topics.toList, correlationId)
new TopicMetadataRequest(versionId, correlationId, clientId, topics.toList)
}
}
case class TopicMetadataRequest(val versionId: Short,
val correlationId: Int,
val clientId: String,
val topics: Seq[String],
val correlationId: Int)
val topics: Seq[String])
extends RequestOrResponse(Some(RequestKeys.MetadataKey)){
def this(topics: Seq[String]) =
this(TopicMetadataRequest.CurrentVersion, TopicMetadataRequest.DefaultClientId, topics, 0)
def this(topics: Seq[String], correlationId: Int) =
this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics)
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId) // correlation id not set yet
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(topics.size)
topics.foreach(topic => writeShortString(buffer, topic))
}
def sizeInBytes(): Int = {
2 + 4 + shortStringLength(clientId) + 4 /* number of topics */ + topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */
2 + /* version id */
4 + /* correlation id */
shortStringLength(clientId) + /* client id */
4 + /* number of topics */
topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */
}
}

View File

@ -20,22 +20,23 @@ object ClientUtils extends Logging{
* @param producerConfig The producer's config
* @return topic metadata response
*/
def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], producerConfig: ProducerConfig): TopicMetadataResponse = {
def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = {
var fetchMetaDataSucceeded: Boolean = false
var i: Int = 0
val topicMetadataRequest = new TopicMetadataRequest(topics.toSeq)
val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, producerConfig.clientId, topics.toSeq)
var topicMetadataResponse: TopicMetadataResponse = null
var t: Throwable = null
while(i < brokers.size && !fetchMetaDataSucceeded) {
val producer: SyncProducer = ProducerPool.createSyncProducer(producerConfig, brokers(i))
info("Fetching metadata for topic %s".format(topics))
info("Fetching metadata with correlation id %d for %d topic(s) %s".format(correlationId, topics.size, topics))
try {
topicMetadataResponse = producer.send(topicMetadataRequest)
fetchMetaDataSucceeded = true
}
catch {
case e =>
warn("fetching topic metadata for topics [%s] from broker [%s] failed".format(topics, brokers(i).toString), e)
warn("Fetching topic metadata with correlation id %d for topics [%s] from broker [%s] failed"
.format(correlationId, topics, brokers(i).toString), e)
t = e
} finally {
i = i + 1
@ -44,6 +45,8 @@ object ClientUtils extends Logging{
}
if(!fetchMetaDataSucceeded){
throw new KafkaException("fetching topic metadata for topics [%s] from broker [%s] failed".format(topics, brokers), t)
} else {
debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics))
}
return topicMetadataResponse
}
@ -58,9 +61,9 @@ object ClientUtils extends Logging{
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)
props.put("client.id", clientId)
val producerConfig = new ProducerConfig(props)
fetchTopicMetadata(topics, brokers, producerConfig)
fetchTopicMetadata(topics, brokers, producerConfig, 0)
}
/**

View File

@ -261,11 +261,11 @@ class Partition(val topic: String,
.format(topic, partitionId, oldHighWatermark, newHighWatermark, allLogEndOffsets.mkString(",")))
}
def maybeShrinkIsr(replicaMaxLagTimeMs: Long, replicaMaxLagBytes: Long) {
def maybeShrinkIsr(replicaMaxLagTimeMs: Long, replicaMaxLagMessages: Long) {
leaderIsrUpdateLock synchronized {
leaderReplicaIfLocal() match {
case Some(leaderReplica) =>
val outOfSyncReplicas = getOutOfSyncReplicas(leaderReplica, replicaMaxLagTimeMs, replicaMaxLagBytes)
val outOfSyncReplicas = getOutOfSyncReplicas(leaderReplica, replicaMaxLagTimeMs, replicaMaxLagMessages)
if(outOfSyncReplicas.size > 0) {
val newInSyncReplicas = inSyncReplicas -- outOfSyncReplicas
assert(newInSyncReplicas.size > 0)
@ -281,12 +281,12 @@ class Partition(val topic: String,
}
}
def getOutOfSyncReplicas(leaderReplica: Replica, keepInSyncTimeMs: Long, keepInSyncBytes: Long): Set[Replica] = {
def getOutOfSyncReplicas(leaderReplica: Replica, keepInSyncTimeMs: Long, keepInSyncMessages: Long): Set[Replica] = {
/**
* there are two cases that need to be handled here -
* 1. Stuck followers: If the leo of the replica is less than the leo of leader and the leo hasn't been updated
* for keepInSyncTimeMs ms, the follower is stuck and should be removed from the ISR
* 2. Slow followers: If the leo of the slowest follower is behind the leo of the leader by keepInSyncBytes, the
* 2. Slow followers: If the leo of the slowest follower is behind the leo of the leader by keepInSyncMessages, the
* follower is not catching up and should be removed from the ISR
**/
val leaderLogEndOffset = leaderReplica.logEndOffset
@ -298,7 +298,7 @@ class Partition(val topic: String,
val stuckReplicas = possiblyStuckReplicas.filter(r => r.logEndOffsetUpdateTimeMs < (time.milliseconds - keepInSyncTimeMs))
debug("Stuck replicas for topic %s partition %d are %s".format(topic, partitionId, stuckReplicas.map(_.brokerId).mkString(",")))
// Case 2 above
val slowReplicas = candidateReplicas.filter(r => r.logEndOffset >= 0 && (leaderLogEndOffset - r.logEndOffset) > keepInSyncBytes)
val slowReplicas = candidateReplicas.filter(r => r.logEndOffset >= 0 && (leaderLogEndOffset - r.logEndOffset) > keepInSyncMessages)
debug("Slow replicas for topic %s partition %d are %s".format(topic, partitionId, slowReplicas.map(_.brokerId).mkString(",")))
stuckReplicas ++ slowReplicas
}

View File

@ -144,14 +144,14 @@ object ConsoleConsumer extends Logging {
}
val props = new Properties()
props.put("groupid", options.valueOf(groupIdOpt))
props.put("socket.buffersize", options.valueOf(socketBufferSizeOpt).toString)
props.put("fetch.size", options.valueOf(fetchSizeOpt).toString)
props.put("min.fetch.bytes", options.valueOf(minFetchBytesOpt).toString)
props.put("max.fetch.wait.ms", options.valueOf(maxWaitMsOpt).toString)
props.put("autocommit.enable", "true")
props.put("autocommit.interval.ms", options.valueOf(autoCommitIntervalOpt).toString)
props.put("autooffset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest")
props.put("group.id", options.valueOf(groupIdOpt))
props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString)
props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString)
props.put("fetch.min.bytes", options.valueOf(minFetchBytesOpt).toString)
props.put("fetch.wait.max.ms", options.valueOf(maxWaitMsOpt).toString)
props.put("auto.commit.enable", "true")
props.put("auto.commit.interval.ms", options.valueOf(autoCommitIntervalOpt).toString)
props.put("auto.offset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest")
props.put("zk.connect", options.valueOf(zkConnectOpt))
props.put("consumer.timeout.ms", options.valueOf(consumerTimeoutMsOpt).toString)
val config = new ConsumerConfig(props)
@ -209,7 +209,7 @@ object ConsoleConsumer extends Logging {
} catch {
case e => error("Error processing message, stopping consumer: ", e)
}
System.out.println("Consumed %d messages".format(numMessages))
System.err.println("Consumed %d messages".format(numMessages))
System.out.flush()
formatter.close()
connector.shutdown()

View File

@ -52,11 +52,11 @@ object ConsumerConfig extends Config {
}
def validateClientId(clientId: String) {
validateChars("clientid", clientId)
validateChars("client.id", clientId)
}
def validateGroupId(groupId: String) {
validateChars("groupid", groupId)
validateChars("group.id", groupId)
}
def validateAutoOffsetReset(autoOffsetReset: String) {
@ -77,38 +77,38 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
}
/** a string that uniquely identifies a set of consumers within the same consumer group */
val groupId = props.getString("groupid")
val groupId = props.getString("group.id")
/** consumer id: generated automatically if not set.
* Set this explicitly for only testing purpose. */
val consumerId: Option[String] = Option(props.getString("consumerid", null))
val consumerId: Option[String] = Option(props.getString("consumer.id", null))
/** the socket timeout for network requests. The actual timeout set will be max.fetch.wait + socket.timeout.ms. */
val socketTimeoutMs = props.getInt("socket.timeout.ms", SocketTimeout)
/** the socket receive buffer for network requests */
val socketBufferSize = props.getInt("socket.buffersize", SocketBufferSize)
val socketReceiveBufferBytes = props.getInt("socket.receive.buffer.bytes", SocketBufferSize)
/** the number of byes of messages to attempt to fetch */
val fetchSize = props.getInt("fetch.size", FetchSize)
val fetchMessageMaxBytes = props.getInt("fetch.message.max.bytes", FetchSize)
/** if true, periodically commit to zookeeper the offset of messages already fetched by the consumer */
val autoCommit = props.getBoolean("autocommit.enable", AutoCommit)
val autoCommitEnable = props.getBoolean("auto.commit.enable", AutoCommit)
/** the frequency in ms that the consumer offsets are committed to zookeeper */
val autoCommitIntervalMs = props.getInt("autocommit.interval.ms", AutoCommitInterval)
val autoCommitIntervalMs = props.getInt("auto.commit.interval.ms", AutoCommitInterval)
/** max number of messages buffered for consumption */
val maxQueuedChunks = props.getInt("queuedchunks.max", MaxQueuedChunks)
val queuedMaxMessages = props.getInt("queued.max.messages", MaxQueuedChunks)
/** max number of retries during rebalance */
val maxRebalanceRetries = props.getInt("rebalance.retries.max", MaxRebalanceRetries)
val rebalanceMaxRetries = props.getInt("rebalance.max.retries", MaxRebalanceRetries)
/** the minimum amount of data the server should return for a fetch request. If insufficient data is available the request will block */
val minFetchBytes = props.getInt("min.fetch.bytes", MinFetchBytes)
val fetchMinBytes = props.getInt("fetch.min.bytes", MinFetchBytes)
/** the maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediate satisfy min.fetch.bytes */
val maxFetchWaitMs = props.getInt("max.fetch.wait.ms", MaxFetchWaitMs)
/** the maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy fetch.min.bytes */
val fetchWaitMaxMs = props.getInt("fetch.wait.max.ms", MaxFetchWaitMs)
/** backoff time between retries during rebalance */
val rebalanceBackoffMs = props.getInt("rebalance.backoff.ms", zkSyncTimeMs)
@ -120,7 +120,7 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
smallest : automatically reset the offset to the smallest offset
largest : automatically reset the offset to the largest offset
anything else: throw exception to the consumer */
val autoOffsetReset = props.getString("autooffset.reset", AutoOffsetReset)
val autoOffsetReset = props.getString("auto.offset.reset", AutoOffsetReset)
/** throw a timeout exception to the consumer if no message is available for consumption after the specified interval */
val consumerTimeoutMs = props.getInt("consumer.timeout.ms", ConsumerTimeoutMs)
@ -129,12 +129,12 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
* Typically, it's only used for mirroring raw messages from one kafka cluster to another to save the
* overhead of decompression.
* */
val enableShallowIterator = props.getBoolean("shallowiterator.enable", false)
val shallowIteratorEnable = props.getBoolean("shallow.iterator.enable", false)
/**
* Client id is specified by the kafka consumer client, used to distinguish different clients
*/
val clientId = props.getString("clientid", groupId)
val clientId = props.getString("client.id", groupId)
validate(this)
}

View File

@ -30,14 +30,14 @@ class ConsumerFetcherThread(name: String,
partitionMap: Map[TopicAndPartition, PartitionTopicInfo],
val consumerFetcherManager: ConsumerFetcherManager)
extends AbstractFetcherThread(name = name,
clientId = config.clientId,
clientId = config.clientId + "-" + name,
sourceBroker = sourceBroker,
socketTimeout = config.socketTimeoutMs,
socketBufferSize = config.socketBufferSize,
fetchSize = config.fetchSize,
socketBufferSize = config.socketReceiveBufferBytes,
fetchSize = config.fetchMessageMaxBytes,
fetcherBrokerId = Request.OrdinaryConsumerId,
maxWait = config.maxFetchWaitMs,
minBytes = config.minFetchBytes) {
maxWait = config.fetchWaitMaxMs,
minBytes = config.fetchMinBytes) {
// process fetched data
def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: FetchResponsePartitionData) {

View File

@ -112,7 +112,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
connectZk()
createFetcher()
if (config.autoCommit) {
if (config.autoCommitEnable) {
scheduler.startup
info("starting auto committer every " + config.autoCommitIntervalMs + " ms")
scheduler.schedule("kafka-consumer-autocommit",
@ -163,14 +163,14 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
if (wildcardTopicWatcher != null)
wildcardTopicWatcher.shutdown()
try {
if (config.autoCommit)
if (config.autoCommitEnable)
scheduler.shutdown()
fetcher match {
case Some(f) => f.shutdown
case None =>
}
sendShutdownToAllQueues()
if (config.autoCommit)
if (config.autoCommitEnable)
commitOffsets()
if (zkClient != null) {
zkClient.close()
@ -197,9 +197,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
// make a list of (queue,stream) pairs, one pair for each threadId
val queuesAndStreams = topicThreadIds.values.map(threadIdSet =>
threadIdSet.map(_ => {
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks)
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.queuedMaxMessages)
val stream = new KafkaStream[K,V](
queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.enableShallowIterator, config.clientId)
queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.shallowIteratorEnable, config.clientId)
(queue, stream)
})
).flatten.toList
@ -368,7 +368,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
def syncedRebalance() {
rebalanceLock synchronized {
for (i <- 0 until config.maxRebalanceRetries) {
for (i <- 0 until config.rebalanceMaxRetries) {
info("begin rebalancing consumer " + consumerIdString + " try #" + i)
var done = false
val cluster = getCluster(zkClient)
@ -396,7 +396,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
}
}
throw new ConsumerRebalanceFailedException(consumerIdString + " can't rebalance after " + config.maxRebalanceRetries +" retries")
throw new ConsumerRebalanceFailedException(consumerIdString + " can't rebalance after " + config.rebalanceMaxRetries +" retries")
}
private def rebalance(cluster: Cluster): Boolean = {
@ -613,7 +613,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
queue,
consumedOffset,
fetchedOffset,
new AtomicInteger(config.fetchSize),
new AtomicInteger(config.fetchMessageMaxBytes),
config.clientId)
partTopicInfoMap.put(partition, partTopicInfo)
debug(partTopicInfo + " selected new offset " + offset)
@ -712,12 +712,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private val wildcardQueuesAndStreams = (1 to numStreams)
.map(e => {
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks)
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.queuedMaxMessages)
val stream = new KafkaStream[K,V](queue,
config.consumerTimeoutMs,
keyDecoder,
valueDecoder,
config.enableShallowIterator,
config.shallowIteratorEnable,
config.clientId)
(queue, stream)
}).toList

View File

@ -184,13 +184,13 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques
}
}
def sendRequestsToBrokers(controllerEpoch: Int, liveBrokers: Set[Broker]) {
def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int, liveBrokers: Set[Broker]) {
leaderAndIsrRequestMap.foreach { m =>
val broker = m._1
val partitionStateInfos = m._2.toMap
val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet
val leaders = liveBrokers.filter(b => leaderIds.contains(b.id))
val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerEpoch)
val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerEpoch, correlationId)
debug("The leaderAndIsr request sent to broker %d is %s".format(broker, leaderAndIsrRequest))
sendRequest(broker, leaderAndIsrRequest, null)
}
@ -203,7 +203,7 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques
debug("The stop replica request (delete = %s) sent to broker %d is %s"
.format(deletePartitions, broker, replicas.mkString(",")))
sendRequest(broker, new StopReplicaRequest(deletePartitions,
Set.empty[(String, Int)] ++ replicas, controllerEpoch), null)
Set.empty[(String, Int)] ++ replicas, controllerEpoch, correlationId), null)
}
}
m.clear()

View File

@ -34,6 +34,7 @@ import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient}
import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException}
import scala.Some
import kafka.common.TopicAndPartition
import java.util.concurrent.atomic.AtomicInteger
class ControllerContext(val zkClient: ZkClient,
var controllerChannelManager: ControllerChannelManager = null,
@ -42,6 +43,7 @@ class ControllerContext(val zkClient: ZkClient,
val brokerShutdownLock: Object = new Object,
var epoch: Int = KafkaController.InitialControllerEpoch - 1,
var epochZkVersion: Int = KafkaController.InitialControllerEpochZkVersion - 1,
val correlationId: AtomicInteger = new AtomicInteger(0),
var allTopics: Set[String] = Set.empty,
var partitionReplicaAssignment: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map.empty,
var allLeaders: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty,
@ -186,7 +188,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
}
}
}
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.liveBrokers)
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers)
val partitionsRemaining = replicatedPartitionsBrokerLeads().toSet
debug("Remaining partitions to move on broker %d: %s".format(id, partitionsRemaining.mkString(",")))

View File

@ -85,9 +85,10 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
if(partitionState.equals(OfflinePartition) || partitionState.equals(NewPartition))
handleStateChange(topicAndPartition.topic, topicAndPartition.partition, OnlinePartition, offlinePartitionSelector)
}
brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.liveBrokers)
brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers)
} catch {
case e => error("Error while moving some partitions to the online state", e)
// TODO: It is not enough to bail out and log an error, it is important to trigger leader election for those partitions
}
}
@ -104,7 +105,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
partitions.foreach { topicAndPartition =>
handleStateChange(topicAndPartition.topic, topicAndPartition.partition, targetState, leaderSelector)
}
brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.liveBrokers)
brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers)
}catch {
case e => error("Error while moving some partitions to %s state".format(targetState), e)
}

View File

@ -83,7 +83,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
try {
brokerRequestBatch.newBatch()
replicas.foreach(r => handleStateChange(r.topic, r.partition, r.replica, targetState))
brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.liveBrokers)
brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers)
}catch {
case e => error("Error while moving some replicas to %s state".format(targetState), e)
}

View File

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

View File

@ -52,8 +52,11 @@ class FileMessageSet private[kafka](val file: File,
new AtomicInteger(math.min(channel.size().toInt, end) - start)
/* if this is not a slice, update the file pointer to the end of the file */
if (!isSlice)
if (!isSlice) {
info("Creating or reloading log segment %s".format(file.getAbsolutePath))
/* set the file position to the last byte in the file */
channel.position(channel.size)
}
/**
* Create a file message set with no slicing.

View File

@ -371,11 +371,17 @@ class Log(val dir: File,
*/
private def maybeRoll(): LogSegment = {
val segment = activeSegment
if ((segment.size > maxSegmentSize) ||
(segment.size > 0 && time.milliseconds - segment.created > rollIntervalMs) ||
segment.index.isFull)
if (segment.size > maxSegmentSize) {
info("Rolling %s due to full data log".format(name))
roll()
else
} else if (segment.size > 0 && time.milliseconds - segment.created > rollIntervalMs) {
info("Rolling %s due to time based rolling".format(name))
roll()
} else if (segment.index.isFull) {
info("Rolling %s due to full index maxIndexSize = %d, entries = %d, maxEntries = %d"
.format(name, segment.index.maxIndexSize, segment.index.entries(), segment.index.maxEntries))
roll()
} else
segment
}
@ -398,7 +404,7 @@ class Log(val dir: File,
file.delete()
}
debug("Rolling log '" + name + "' to " + logFile.getName + " and " + indexFile.getName)
info("Rolling log '" + name + "' to " + logFile.getName + " and " + indexFile.getName)
segments.lastEntry() match {
case null =>
case entry => entry.getValue.index.trimToValidSize()
@ -446,7 +452,7 @@ class Log(val dir: File,
logSegments.foreach(_.delete())
Utils.rm(dir)
}
/**
* Truncate this log so that it ends with the greatest offset < targetOffset.
* @param targetOffset The offset to truncate to, an upper bound on all offsets in the log after truncation is complete.

View File

@ -44,15 +44,15 @@ class LogManager(val config: KafkaConfig,
val LockFile = ".lock"
val InitialTaskDelayMs = 30*1000
val logDirs: Array[File] = config.logDirs.map(new File(_)).toArray
private val logFileSizeMap = config.logFileSizeMap
private val logFlushInterval = config.flushInterval
private val logFlushIntervals = config.flushIntervalMap
private val logFileSizeMap = config.logSegmentBytesPerTopicMap
private val logFlushInterval = config.logFlushIntervalMessages
private val logFlushIntervals = config.logFlushIntervalMsPerTopicMap
private val logCreationLock = new Object
private val logRetentionSizeMap = config.logRetentionSizeMap
private val logRetentionMsMap = config.logRetentionHoursMap.map(e => (e._1, e._2 * 60 * 60 * 1000L)) // convert hours to ms
private val logRollMsMap = config.logRollHoursMap.map(e => (e._1, e._2 * 60 * 60 * 1000L))
private val logRetentionSizeMap = config.logRetentionBytesPerTopicMap
private val logRetentionMsMap = config.logRetentionHoursPerTopicMap.map(e => (e._1, e._2 * 60 * 60 * 1000L)) // convert hours to ms
private val logRollMsMap = config.logRollHoursPerTopicMap.map(e => (e._1, e._2 * 60 * 60 * 1000L))
private val logRollDefaultIntervalMs = 1000L * 60 * 60 * config.logRollHours
private val logCleanupIntervalMs = 1000L * 60 * config.logCleanupIntervalMinutes
private val logCleanupIntervalMs = 1000L * 60 * config.logCleanupIntervalMins
private val logCleanupDefaultAgeMs = 1000L * 60 * 60 * config.logRetentionHours
this.logIdent = "[Log Manager on Broker " + config.brokerId + "] "
@ -115,15 +115,15 @@ class LogManager(val config: KafkaConfig,
info("Loading log '" + dir.getName + "'")
val topicPartition = parseTopicPartitionName(dir.getName)
val rollIntervalMs = logRollMsMap.get(topicPartition.topic).getOrElse(this.logRollDefaultIntervalMs)
val maxLogFileSize = logFileSizeMap.get(topicPartition.topic).getOrElse(config.logFileSize)
val maxLogFileSize = logFileSizeMap.get(topicPartition.topic).getOrElse(config.logSegmentBytes)
val log = new Log(dir,
scheduler,
maxLogFileSize,
config.maxMessageSize,
maxLogFileSize,
config.messageMaxBytes,
logFlushInterval,
rollIntervalMs,
needsRecovery,
config.logIndexMaxSizeBytes,
config.logIndexSizeMaxBytes,
config.logIndexIntervalBytes,
config.logDeleteDelayMs,
time)
@ -149,11 +149,11 @@ class LogManager(val config: KafkaConfig,
period = logCleanupIntervalMs,
TimeUnit.MILLISECONDS)
info("Starting log flusher with a default period of %d ms with the following overrides: %s."
.format(config.defaultFlushIntervalMs, logFlushIntervals.map(e => e._1.toString + "=" + e._2.toString).mkString(", ")))
.format(config.logFlushIntervalMs, logFlushIntervals.map(e => e._1.toString + "=" + e._2.toString).mkString(", ")))
scheduler.schedule("kafka-log-flusher",
flushDirtyLogs,
delay = InitialTaskDelayMs,
period = config.flushSchedulerThreadRate,
period = config.logFlushSchedulerIntervalMs,
TimeUnit.MILLISECONDS)
}
}
@ -198,15 +198,15 @@ class LogManager(val config: KafkaConfig,
val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition)
dir.mkdirs()
val rollIntervalMs = logRollMsMap.get(topicAndPartition.topic).getOrElse(this.logRollDefaultIntervalMs)
val maxLogFileSize = logFileSizeMap.get(topicAndPartition.topic).getOrElse(config.logFileSize)
val maxLogFileSize = logFileSizeMap.get(topicAndPartition.topic).getOrElse(config.logSegmentBytes)
log = new Log(dir,
scheduler,
maxLogFileSize,
config.maxMessageSize,
config.messageMaxBytes,
logFlushInterval,
rollIntervalMs,
needsRecovery = false,
config.logIndexMaxSizeBytes,
config.logIndexSizeMaxBytes,
config.logIndexIntervalBytes,
config.logDeleteDelayMs,
time)
@ -252,7 +252,7 @@ class LogManager(val config: KafkaConfig,
*/
private def cleanupSegmentsToMaintainSize(log: Log): Int = {
val topic = parseTopicPartitionName(log.dir.getName).topic
val maxLogRetentionSize = logRetentionSizeMap.get(topic).getOrElse(config.logRetentionSize)
val maxLogRetentionSize = logRetentionSizeMap.get(topic).getOrElse(config.logRetentionBytes)
if(maxLogRetentionSize < 0 || log.size < maxLogRetentionSize)
return 0
var diff = log.size - maxLogRetentionSize
@ -313,7 +313,7 @@ class LogManager(val config: KafkaConfig,
try {
val timeSinceLastFlush = time.milliseconds - log.lastFlushTime
var logFlushInterval = config.defaultFlushIntervalMs
var logFlushInterval = config.logFlushIntervalMs
if(logFlushIntervals.contains(topicAndPartition.topic))
logFlushInterval = logFlushIntervals(topicAndPartition.topic)
debug("Checking if flush is needed on " + topicAndPartition.topic + " flush interval " + logFlushInterval +

View File

@ -83,7 +83,10 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
Utils.swallow(raf.close())
}
}
info("Created index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d"
.format(file.getAbsolutePath, maxEntries, maxIndexSize, entries(), lastOffset))
/**
* The maximum number of eight-byte entries this index can hold
*/
@ -94,7 +97,7 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
/* the last offset in the index */
var lastOffset = readLastOffset()
/**
* The last offset written to the index
*/
@ -274,6 +277,7 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
* Delete this index file
*/
def delete(): Boolean = {
info("Deleting index " + this.file.getAbsolutePath)
this.file.delete()
}

View File

@ -46,7 +46,12 @@ object RequestChannel extends Logging {
val requestId = buffer.getShort()
val requestObj: RequestOrResponse = RequestKeys.deserializerForKey(requestId)(buffer)
buffer.rewind()
trace("Received request: %s".format(requestObj))
buffer.getShort
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = ApiUtils.readShortString(buffer)
buffer.rewind()
trace("Received request v%d with correlation id %d from client %s: %s".format(versionId, correlationId, clientId, requestObj))
def updateRequestMetrics() {
val endTimeMs = SystemTime.milliseconds
@ -75,8 +80,8 @@ object RequestChannel extends Logging {
m.responseSendTimeHist.update(responseSendTime)
m.totalTimeHist.update(totalTime)
}
trace("Completed request: %s totalTime:%d queueTime:%d localTime:%d remoteTime:%d sendTime:%d"
.format(requestObj, totalTime, queueTime, apiLocalTime, apiRemoteTime, responseSendTime))
trace("Completed request v%d with correlation id %d and client %s: %s, totalTime:%d, queueTime:%d, localTime:%d, remoteTime:%d, sendTime:%d"
.format(versionId, correlationId, clientId, requestObj, totalTime, queueTime, apiLocalTime, apiRemoteTime, responseSendTime))
}
}

View File

@ -319,7 +319,7 @@ private[kafka] class Processor(val id: Int,
} else if(receive.complete) {
val req = RequestChannel.Request(processor = id, requestKey = key, buffer = receive.buffer, startTimeMs = time.milliseconds)
requestChannel.sendRequest(req)
trace("Recieved request, sending for processing by handler: " + req)
trace("Received request, sending for processing by handler: " + req)
key.attach(null)
} else {
// more reading to be done

View File

@ -37,7 +37,7 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig,
* @return a sequence of (brokerId, numPartitions). Returns a zero-length
* sequence if no brokers are available.
*/
def getBrokerPartitionInfo(topic: String): Seq[PartitionAndLeader] = {
def getBrokerPartitionInfo(topic: String, correlationId: Int): Seq[PartitionAndLeader] = {
debug("Getting broker partition info for topic %s".format(topic))
// check if the cache has metadata for this topic
val topicMetadata = topicPartitionInfo.get(topic)
@ -46,7 +46,7 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig,
case Some(m) => m
case None =>
// refresh the topic metadata cache
updateInfo(Set(topic))
updateInfo(Set(topic), correlationId)
val topicMetadata = topicPartitionInfo.get(topic)
topicMetadata match {
case Some(m) => m
@ -70,9 +70,9 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig,
* It updates the cache by issuing a get topic metadata request to a random broker.
* @param topics the topics for which the metadata is to be fetched
*/
def updateInfo(topics: Set[String]) {
def updateInfo(topics: Set[String], correlationId: Int) {
var topicsMetadata: Seq[TopicMetadata] = Nil
val topicMetadataResponse = ClientUtils.fetchTopicMetadata(topics, brokers, producerConfig)
val topicMetadataResponse = ClientUtils.fetchTopicMetadata(topics, brokers, producerConfig, correlationId)
topicsMetadata = topicMetadataResponse.topicsMetadata
// throw partition specific exception
topicsMetadata.foreach(tmd =>{

View File

@ -50,6 +50,27 @@ object ConsoleProducer {
.describedAs("timeout_ms")
.ofType(classOf[java.lang.Long])
.defaultsTo(1000)
val queueSizeOpt = parser.accepts("queue-size", "If set and the producer is running in asynchronous mode, this gives the maximum amount of " +
" messages will queue awaiting suffient batch size.")
.withRequiredArg
.describedAs("queue_size")
.ofType(classOf[java.lang.Long])
.defaultsTo(10000)
val queueEnqueueTimeoutMsOpt = parser.accepts("queue-enqueuetimeout-ms", "Timeout for event enqueue")
.withRequiredArg
.describedAs("queue enqueuetimeout ms")
.ofType(classOf[java.lang.Long])
.defaultsTo(0)
val requestRequiredAcksOpt = parser.accepts("request-required-acks", "The required acks of the producer requests")
.withRequiredArg
.describedAs("request required acks")
.ofType(classOf[java.lang.Integer])
.defaultsTo(0)
val requestTimeoutMsOpt = parser.accepts("request-timeout-ms", "The ack timeout of the producer requests. Value must be non-negative and non-zero")
.withRequiredArg
.describedAs("request timeout ms")
.ofType(classOf[java.lang.Integer])
.defaultsTo(1500)
val valueEncoderOpt = parser.accepts("value-serializer", "The class name of the message encoder implementation to use for serializing values.")
.withRequiredArg
.describedAs("encoder_class")
@ -88,6 +109,10 @@ object ConsoleProducer {
val compress = options.has(compressOpt)
val batchSize = options.valueOf(batchSizeOpt)
val sendTimeout = options.valueOf(sendTimeoutOpt)
val queueSize = options.valueOf(queueSizeOpt)
val queueEnqueueTimeoutMs = options.valueOf(queueEnqueueTimeoutMsOpt)
val requestRequiredAcks = options.valueOf(requestRequiredAcksOpt)
val requestTimeoutMs = options.valueOf(requestTimeoutMsOpt)
val keyEncoderClass = options.valueOf(keyEncoderOpt)
val valueEncoderClass = options.valueOf(valueEncoderOpt)
val readerClass = options.valueOf(messageReaderOpt)
@ -100,8 +125,12 @@ object ConsoleProducer {
props.put("compression.codec", codec.toString)
props.put("producer.type", if(sync) "sync" else "async")
if(options.has(batchSizeOpt))
props.put("batch.size", batchSize.toString)
props.put("queue.time", sendTimeout.toString)
props.put("batch.num.messages", batchSize.toString)
props.put("queue.buffering.max.ms", sendTimeout.toString)
props.put("queue.buffering.max.messages", queueSize.toString)
props.put("queue.enqueueTimeout.ms", queueEnqueueTimeoutMs.toString)
props.put("request.required.acks", requestRequiredAcks.toString)
props.put("request.timeout.ms", requestTimeoutMs.toString)
props.put("key.serializer.class", keyEncoderClass)
props.put("serializer.class", valueEncoderClass)
@ -122,6 +151,7 @@ object ConsoleProducer {
if(message != null)
producer.send(message)
} while(message != null)
System.exit(0)
}
def parseLineReaderArgs(args: Iterable[String]): Properties = {
@ -163,21 +193,22 @@ object ConsoleProducer {
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,
(reader.readLine(), parseKey) match {
case (null, _) => null
case (line, true) =>
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)
}
case (line, false) =>
new KeyedMessage(topic, line)
}
}
}

View File

@ -24,9 +24,6 @@ private class DefaultPartitioner[T](props: VerifiableProperties = null) extends
private val random = new java.util.Random
def partition(key: T, numPartitions: Int): Int = {
if(key == null)
random.nextInt(numPartitions)
else
Utils.abs(key.hashCode) % numPartitions
Utils.abs(key.hashCode) % numPartitions
}
}

View File

@ -73,8 +73,8 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging {
//These have default values in ProducerConfig and AsyncProducerConfig. We don't care if they're not specified
if(producerType != null) props.put("producer.type", producerType)
if(compressionCodec != null) props.put("compression.codec", compressionCodec)
if(enqueueTimeout != null) props.put("queue.enqueueTimeout.ms", enqueueTimeout)
if(queueSize != null) props.put("queue.size", queueSize)
if(enqueueTimeout != null) props.put("queue.enqueue.timeout.ms", enqueueTimeout)
if(queueSize != null) props.put("queue.buffering.max.messages", queueSize)
val config : ProducerConfig = new ProducerConfig(props)
producer = new Producer[String, String](config)
LogLog.debug("Kafka producer connected to " + config.brokerList)

View File

@ -31,7 +31,7 @@ class Producer[K,V](config: ProducerConfig,
extends Logging {
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.queueBufferingMaxMessages)
private val random = new Random
private var sync: Boolean = true
@ -44,8 +44,8 @@ class Producer[K,V](config: ProducerConfig,
producerSendThread = new ProducerSendThread[K,V]("ProducerSendThread-" + asyncProducerID,
queue,
eventHandler,
config.queueTime,
config.batchSize,
config.queueBufferingMaxMs,
config.batchNumMessages,
config.clientId)
producerSendThread.start()
}
@ -87,17 +87,17 @@ class Producer[K,V](config: ProducerConfig,
private def asyncSend(messages: Seq[KeyedMessage[K,V]]) {
for (message <- messages) {
val added = config.enqueueTimeoutMs match {
val added = config.queueEnqueueTimeoutMs match {
case 0 =>
queue.offer(message)
case _ =>
try {
config.enqueueTimeoutMs < 0 match {
config.queueEnqueueTimeoutMs < 0 match {
case true =>
queue.put(message)
true
case _ =>
queue.offer(message, config.enqueueTimeoutMs, TimeUnit.MILLISECONDS)
queue.offer(message, config.queueEnqueueTimeoutMs, TimeUnit.MILLISECONDS)
}
}
catch {

View File

@ -26,12 +26,12 @@ import kafka.common.{InvalidConfigException, Config}
object ProducerConfig extends Config {
def validate(config: ProducerConfig) {
validateClientId(config.clientId)
validateBatchSize(config.batchSize, config.queueSize)
validateBatchSize(config.batchNumMessages, config.queueBufferingMaxMessages)
validateProducerType(config.producerType)
}
def validateClientId(clientId: String) {
validateChars("clientid", clientId)
validateChars("client.id", clientId)
}
def validateBatchSize(batchSize: Int, queueSize: Int) {
@ -101,17 +101,26 @@ class ProducerConfig private (val props: VerifiableProperties)
*/
val compressedTopics = Utils.parseCsvList(props.getString("compressed.topics", null))
/**
* If a request fails it is possible to have the producer automatically retry. This is controlled by this setting.
* Note that not all errors mean that the message was lost--for example if the network connection is lost we will
* get a socket exception--in this case enabling retries can result in duplicate messages.
*/
val producerRetries = props.getInt("producer.num.retries", 3)
/** The leader may be unavailable transiently, which can fail the sending of a message.
* This property specifies the number of retries when such failures occur.
*/
val messageSendMaxRetries = props.getInt("message.send.max.retries", 3)
/** Before each retry, the producer refreshes the metadata of relevant topics. Since leader
* election takes a bit of time, this property specifies the amount of time that the producer
* waits before refreshing the metadata.
*/
val retryBackoffMs = props.getInt("retry.backoff.ms", 100)
/**
* The amount of time to wait in between retries
* The producer generally refreshes the topic metadata from brokers when there is a failure
* (partition missing, leader not available...). It will also poll regularly (default: every 10min
* so 600000ms). If you set this to a negative value, metadata will only get refreshed on failure.
* If you set this to zero, the metadata will get refreshed after each message sent (not recommended)
* Important note: the refresh happen only AFTER the message is sent, so if the producer never sends
* a message the metadata is never refreshed
*/
val producerRetryBackoffMs = props.getInt("producer.retry.backoff.ms", 100)
val topicMetadataRefreshIntervalMs = props.getInt("topic.metadata.refresh.interval.ms", 600000)
validate(this)
}

View File

@ -36,7 +36,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging {
private val lock = new Object()
@volatile private var shutdown: Boolean = false
private val blockingChannel = new BlockingChannel(config.host, config.port, BlockingChannel.UseDefaultBufferSize,
config.bufferSize, config.requestTimeoutMs)
config.sendBufferBytes, config.requestTimeoutMs)
val brokerInfo = "host_%s-port_%s".format(config.host, config.port)
val producerRequestStats = ProducerRequestStatsRegistry.getProducerRequestStats(config.clientId)

View File

@ -36,24 +36,22 @@ class SyncProducerConfig private (val props: VerifiableProperties) extends SyncP
trait SyncProducerConfigShared {
val props: VerifiableProperties
val bufferSize = props.getInt("buffer.size", 100*1024)
val maxMessageSize = props.getInt("max.message.size", 1000000)
val sendBufferBytes = props.getInt("send.buffer.bytes", 100*1024)
/* the client application sending the producer requests */
val clientId = props.getString("clientid", SyncProducerConfig.DefaultClientId)
val clientId = props.getString("client.id", SyncProducerConfig.DefaultClientId)
/*
* The required acks of the producer requests - negative value means ack
* after the replicas in ISR have caught up to the leader's offset
* corresponding to this produce request.
*/
val requiredAcks = props.getShort("producer.request.required.acks", SyncProducerConfig.DefaultRequiredAcks)
val requestRequiredAcks = props.getShort("request.required.acks", SyncProducerConfig.DefaultRequiredAcks)
/*
* The ack timeout of the producer requests. Value must be non-negative and non-zero
*/
val requestTimeoutMs = props.getIntInRange("producer.request.timeout.ms", SyncProducerConfig.DefaultAckTimeoutMs,
val requestTimeoutMs = props.getIntInRange("request.timeout.ms", SyncProducerConfig.DefaultAckTimeoutMs,
(1, Integer.MAX_VALUE))
}

View File

@ -22,10 +22,10 @@ trait AsyncProducerConfig {
val props: VerifiableProperties
/* maximum time, in milliseconds, for buffering data on the producer queue */
val queueTime = props.getInt("queue.time", 5000)
val queueBufferingMaxMs = props.getInt("queue.buffering.max.ms", 5000)
/** the maximum size of the blocking queue for buffering on the producer */
val queueSize = props.getInt("queue.size", 10000)
val queueBufferingMaxMessages = props.getInt("queue.buffering.max.messages", 10000)
/**
* Timeout for event enqueue:
@ -33,10 +33,10 @@ trait AsyncProducerConfig {
* -ve: enqueue will block indefinitely if the queue is full
* +ve: enqueue will block up to this many milliseconds if the queue is full
*/
val enqueueTimeoutMs = props.getInt("queue.enqueueTimeout.ms", 0)
val queueEnqueueTimeoutMs = props.getInt("queue.enqueue.timeout.ms", 0)
/** the number of messages batched at the producer */
val batchSize = props.getInt("batch.size", 200)
val batchNumMessages = props.getInt("batch.num.messages", 200)
/** the serializer class for values */
val serializerClass = props.getString("serializer.class", "kafka.serializer.DefaultEncoder")

View File

@ -21,13 +21,12 @@ import kafka.common._
import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
import kafka.producer._
import kafka.serializer.Encoder
import kafka.utils.{Utils, Logging}
import kafka.utils.{Utils, Logging, SystemTime}
import scala.collection.{Seq, Map}
import scala.collection.mutable.{ArrayBuffer, HashMap}
import scala.collection.mutable.{ArrayBuffer, HashMap, Set}
import java.util.concurrent.atomic._
import kafka.api.{TopicMetadata, ProducerRequest}
class DefaultEventHandler[K,V](config: ProducerConfig,
private val partitioner: Partitioner[K],
private val encoder: Encoder[V],
@ -38,11 +37,15 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
val isSync = ("sync" == config.producerType)
val partitionCounter = new AtomicInteger(0)
val correlationCounter = new AtomicInteger(0)
val correlationId = new AtomicInteger(0)
val brokerPartitionInfo = new BrokerPartitionInfo(config, producerPool, topicPartitionInfos)
private val lock = new Object()
private val topicMetadataRefreshInterval = config.topicMetadataRefreshIntervalMs
private var lastTopicMetadataRefreshTime = 0L
private val topicMetadataToRefresh = Set.empty[String]
private val producerStats = ProducerStatsRegistry.getProducerStats(config.clientId)
private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(config.clientId)
@ -56,22 +59,32 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
producerTopicStats.getProducerAllTopicStats.byteRate.mark(dataSize)
}
var outstandingProduceRequests = serializedData
var remainingRetries = config.producerRetries + 1
var remainingRetries = config.messageSendMaxRetries + 1
val correlationIdStart = correlationId.get()
while (remainingRetries > 0 && outstandingProduceRequests.size > 0) {
topicMetadataToRefresh ++= outstandingProduceRequests.map(_.topic)
if (topicMetadataRefreshInterval >= 0 &&
SystemTime.milliseconds - lastTopicMetadataRefreshTime > topicMetadataRefreshInterval) {
Utils.swallowError(brokerPartitionInfo.updateInfo(topicMetadataToRefresh.toSet, correlationId.getAndIncrement))
topicMetadataToRefresh.clear
lastTopicMetadataRefreshTime = SystemTime.milliseconds
}
outstandingProduceRequests = dispatchSerializedData(outstandingProduceRequests)
if (outstandingProduceRequests.size > 0) {
// back off and update the topic metadata cache before attempting another send operation
Thread.sleep(config.producerRetryBackoffMs)
Thread.sleep(config.retryBackoffMs)
// 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, correlationId.getAndIncrement))
remainingRetries -= 1
producerStats.resendRate.mark()
}
}
if(outstandingProduceRequests.size > 0) {
producerStats.failedSendRate.mark()
error("Failed to send the following requests: " + outstandingProduceRequests)
throw new FailedToSendMessageException("Failed to send messages after " + config.producerRetries + " tries.", null)
val correlationIdEnd = correlationId.get()
error("Failed to send the following requests with correlation ids in [%d,%d]: %s".format(correlationIdStart, correlationIdEnd-1, outstandingProduceRequests))
throw new FailedToSendMessageException("Failed to send messages after " + config.messageSendMaxRetries + " tries.", null)
}
}
}
@ -133,9 +146,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
try {
for (message <- messages) {
val topicPartitionsList = getPartitionListForTopic(message)
val totalNumPartitions = topicPartitionsList.length
val partitionIndex = getPartition(message.key, totalNumPartitions)
val partitionIndex = getPartition(message.key, topicPartitionsList)
val brokerPartition = topicPartitionsList(partitionIndex)
// postpone the failure until the send operation, so that requests for other brokers are handled correctly
@ -170,8 +181,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
}
private def getPartitionListForTopic(m: KeyedMessage[K,Message]): Seq[PartitionAndLeader] = {
debug("Getting the number of broker partitions registered for topic: " + m.topic)
val topicPartitionsList = brokerPartitionInfo.getBrokerPartitionInfo(m.topic)
val topicPartitionsList = brokerPartitionInfo.getBrokerPartitionInfo(m.topic, correlationId.getAndIncrement)
debug("Broker partitions registered for topic: %s are %s"
.format(m.topic, topicPartitionsList.map(p => p.partitionId).mkString(",")))
val totalNumPartitions = topicPartitionsList.length
@ -184,17 +194,24 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
* Retrieves the partition id and throws an UnknownTopicOrPartitionException if
* the value of partition is not between 0 and numPartitions-1
* @param key the partition key
* @param numPartitions the total number of available partitions
* @param topicPartitionList the list of available partitions
* @return the partition id
*/
private def getPartition(key: K, numPartitions: Int): Int = {
private def getPartition(key: K, topicPartitionList: Seq[PartitionAndLeader]): Int = {
val numPartitions = topicPartitionList.size
if(numPartitions <= 0)
throw new UnknownTopicOrPartitionException("Invalid number of partitions: " + numPartitions +
"\n Valid values are > 0")
val partition =
if(key == null)
Utils.abs(partitionCounter.getAndIncrement()) % numPartitions
else
if(key == null) {
// If the key is null, we don't really need a partitioner so we just send to the next
// available partition
val availablePartitions = topicPartitionList.filter(_.leaderBrokerIdOpt.isDefined)
if (availablePartitions.isEmpty)
throw new LeaderNotAvailableException("No leader for any partition")
val index = Utils.abs(partitionCounter.getAndIncrement()) % availablePartitions.size
availablePartitions(index).partitionId
} else
partitioner.partition(key, numPartitions)
if(partition < 0 || partition >= numPartitions)
throw new UnknownTopicOrPartitionException("Invalid partition id : " + partition +
@ -214,13 +231,17 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
warn("Failed to send to broker %d with data %s".format(brokerId, messagesPerTopic))
messagesPerTopic.keys.toSeq
} else if(messagesPerTopic.size > 0) {
val producerRequest = new ProducerRequest(correlationCounter.getAndIncrement(), config.clientId, config.requiredAcks,
val currentCorrelationId = correlationId.getAndIncrement
val producerRequest = new ProducerRequest(currentCorrelationId, config.clientId, config.requestRequiredAcks,
config.requestTimeoutMs, messagesPerTopic)
var failedTopicPartitions = Seq.empty[TopicAndPartition]
try {
val syncProducer = producerPool.getProducer(brokerId)
debug("Producer sending messages with correlation id %d for topics %s to broker %d on %s:%d"
.format(currentCorrelationId, messagesPerTopic, brokerId, syncProducer.config.host, syncProducer.config.port))
val response = syncProducer.send(producerRequest)
debug("Producer sent messages for topics %s to broker %d on %s:%d"
.format(messagesPerTopic, brokerId, syncProducer.config.host, syncProducer.config.port))
debug("Producer sent messages with correlation id %d for topics %s to broker %d on %s:%d"
.format(currentCorrelationId, messagesPerTopic, brokerId, syncProducer.config.host, syncProducer.config.port))
if (response.status.size != producerRequest.data.size)
throw new KafkaException("Incomplete response (%s) for producer request (%s)"
.format(response, producerRequest))
@ -229,11 +250,16 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
successfullySentData.foreach(m => messagesPerTopic(m._1).foreach(message =>
trace("Successfully sent message: %s".format(Utils.readString(message.message.payload)))))
}
response.status.filter(_._2.error != ErrorMapping.NoError).toSeq
.map(partitionStatus => partitionStatus._1)
failedTopicPartitions = response.status.filter(_._2.error != ErrorMapping.NoError).toSeq
.map(partitionStatus => partitionStatus._1)
if(failedTopicPartitions.size > 0)
error("Produce request with correlation id %d failed due to response %s. List of failed topic partitions is %s"
.format(currentCorrelationId, response.toString, failedTopicPartitions.mkString(",")))
failedTopicPartitions
} catch {
case t: Throwable =>
warn("failed to send to broker %d with data %s".format(brokerId, messagesPerTopic), t)
warn("Failed to send producer request with correlation id %d to broker %d with data %s"
.format(currentCorrelationId, brokerId, messagesPerTopic), t)
messagesPerTopic.keys.toSeq
}
} else {

View File

@ -66,7 +66,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke
}
override def doWork() {
val fetchRequestuilder = new FetchRequestBuilder().
val fetchRequestBuilder = new FetchRequestBuilder().
clientId(clientId).
replicaId(fetcherBrokerId).
maxWait(maxWait).
@ -78,14 +78,14 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke
partitionMapCond.await()
partitionMap.foreach {
case((topicAndPartition, offset)) =>
fetchRequestuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition,
fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition,
offset, fetchSize)
}
} finally {
partitionMapLock.unlock()
}
val fetchRequest = fetchRequestuilder.build()
val fetchRequest = fetchRequestBuilder.build()
val partitionsWithError = new mutable.HashSet[TopicAndPartition]
var response: FetchResponse = null
try {

View File

@ -22,7 +22,7 @@ import kafka.api._
import kafka.message._
import kafka.network._
import kafka.log._
import kafka.utils.{Pool, SystemTime, Logging, ZkUtils, ZKGroupTopicDirs}
import kafka.utils.ZKGroupTopicDirs
import org.apache.log4j.Logger
import scala.collection._
import kafka.network.RequestChannel.Response
@ -31,6 +31,7 @@ import java.util.concurrent.atomic._
import kafka.metrics.KafkaMetricsGroup
import org.I0Itec.zkclient.ZkClient
import kafka.common._
import kafka.utils.{ZkUtils, Pool, SystemTime, Logging}
/**
@ -43,9 +44,9 @@ class KafkaApis(val requestChannel: RequestChannel,
val config: KafkaConfig) extends Logging {
private val producerRequestPurgatory =
new ProducerRequestPurgatory(replicaManager.config.producerRequestPurgatoryPurgeInterval)
new ProducerRequestPurgatory(replicaManager.config.producerPurgatoryPurgeIntervalRequests)
private val fetchRequestPurgatory =
new FetchRequestPurgatory(requestChannel, replicaManager.config.fetchRequestPurgatoryPurgeInterval)
new FetchRequestPurgatory(requestChannel, replicaManager.config.fetchPurgatoryPurgeIntervalRequests)
private val delayedRequestMetrics = new DelayedRequestMetrics
private val requestLogger = Logger.getLogger("kafka.request.logger")
@ -74,7 +75,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val apiRequest = request.requestObj.asInstanceOf[ProducerRequest]
val producerResponseStatus = apiRequest.data.map {
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.correlationId, producerResponseStatus)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
@ -148,8 +149,8 @@ class KafkaApis(val requestChannel: RequestChannel,
def handleLeaderAndIsrRequest(request: RequestChannel.Request) {
val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest]
if(requestLogger.isTraceEnabled)
requestLogger.trace("Handling leader and ISR request " + leaderAndIsrRequest)
trace("Handling leader and ISR request " + leaderAndIsrRequest)
requestLogger.trace("Handling LeaderAndIsrRequest v%d with correlation id %d from client %s: %s"
.format(leaderAndIsrRequest.versionId, leaderAndIsrRequest.correlationId, leaderAndIsrRequest.clientId, leaderAndIsrRequest.toString))
try {
val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest)
val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error)
@ -165,8 +166,8 @@ class KafkaApis(val requestChannel: RequestChannel,
def handleStopReplicaRequest(request: RequestChannel.Request) {
val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest]
if(requestLogger.isTraceEnabled)
requestLogger.trace("Handling stop replica request " + stopReplicaRequest)
trace("Handling stop replica request " + stopReplicaRequest)
requestLogger.trace("Handling StopReplicaRequest v%d with correlation id %d from client %s: %s"
.format(stopReplicaRequest.versionId, stopReplicaRequest.correlationId, stopReplicaRequest.clientId, stopReplicaRequest.toString))
val (response, error) = replicaManager.stopReplicas(stopReplicaRequest)
val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, response.toMap, error)
@ -198,10 +199,10 @@ class KafkaApis(val requestChannel: RequestChannel,
val produceRequest = request.requestObj.asInstanceOf[ProducerRequest]
val sTime = SystemTime.milliseconds
if(requestLogger.isTraceEnabled)
requestLogger.trace("Handling producer request " + request.toString)
trace("Handling producer request " + request.toString)
requestLogger.trace("Handling ProducerRequest v%d with correlation id %d from client %s: %s"
.format(produceRequest.versionId, produceRequest.correlationId, produceRequest.clientId, produceRequest.toString))
val localProduceResults = appendToLocalLog(produceRequest.data)
val localProduceResults = appendToLocalLog(produceRequest)
debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime))
val numPartitionsInError = localProduceResults.count(_.error.isDefined)
@ -259,7 +260,8 @@ class KafkaApis(val requestChannel: RequestChannel,
/**
* Helper method for handling a parsed producer request
*/
private def appendToLocalLog(partitionAndData: Map[TopicAndPartition, MessageSet]): Iterable[ProduceResult] = {
private def appendToLocalLog(producerRequest: ProducerRequest): Iterable[ProduceResult] = {
val partitionAndData: Map[TopicAndPartition, MessageSet] = producerRequest.data
trace("Append [%s] to local log ".format(partitionAndData.toString))
partitionAndData.map {case (topicAndPartition, messages) =>
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes)
@ -287,7 +289,8 @@ class KafkaApis(val requestChannel: RequestChannel,
case e =>
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicStats.failedProduceRequestRate.mark()
error("Error processing ProducerRequest on %s:%d".format(topicAndPartition.topic, topicAndPartition.partition), e)
error("Error processing ProducerRequest with correlation id %d from client %s on %s:%d"
.format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition.topic, topicAndPartition.partition), e)
new ProduceResult(topicAndPartition, e)
}
}
@ -299,8 +302,8 @@ class KafkaApis(val requestChannel: RequestChannel,
def handleFetchRequest(request: RequestChannel.Request) {
val fetchRequest = request.requestObj.asInstanceOf[FetchRequest]
if(requestLogger.isTraceEnabled)
requestLogger.trace("Handling fetch request " + fetchRequest.toString)
trace("Handling fetch request " + fetchRequest.toString)
requestLogger.trace("Handling FetchRequest v%d with correlation id %d from client %s: %s"
.format(fetchRequest.versionId, fetchRequest.correlationId, fetchRequest.clientId, fetchRequest.toString))
if(fetchRequest.isFromFollower) {
maybeUpdatePartitionHw(fetchRequest)
@ -321,7 +324,8 @@ class KafkaApis(val requestChannel: RequestChannel,
if(fetchRequest.maxWait <= 0 ||
bytesReadable >= fetchRequest.minBytes ||
fetchRequest.numPartitions <= 0) {
debug("Returning fetch response %s for fetch request with correlation id %d".format(dataRead.values.map(_.error).mkString(","), fetchRequest.correlationId))
debug("Returning fetch response %s for fetch request with correlation id %d to client %s"
.format(dataRead.values.map(_.error).mkString(","), fetchRequest.correlationId, fetchRequest.clientId))
val response = new FetchResponse(fetchRequest.correlationId, dataRead)
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response)))
} else {
@ -408,8 +412,8 @@ class KafkaApis(val requestChannel: RequestChannel,
def handleOffsetRequest(request: RequestChannel.Request) {
val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest]
if(requestLogger.isTraceEnabled)
requestLogger.trace("Handling offset request " + offsetRequest.toString)
trace("Handling offset request " + offsetRequest.toString)
requestLogger.trace("Handling OffsetRequest v%d with correlation id %d from client %s: %s"
.format(offsetRequest.versionId, offsetRequest.correlationId, offsetRequest.clientId, offsetRequest.toString))
val responseMap = offsetRequest.requestInfo.map(elem => {
val (topicAndPartition, partitionOffsetRequestInfo) = elem
@ -504,12 +508,17 @@ class KafkaApis(val requestChannel: RequestChannel,
def handleTopicMetadataRequest(request: RequestChannel.Request) {
val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest]
if(requestLogger.isTraceEnabled)
requestLogger.trace("Handling topic metadata request " + metadataRequest.toString())
trace("Handling topic metadata request " + metadataRequest.toString())
requestLogger.trace("Handling TopicMetadataRequest v%d with correlation id %d from client %s: %s"
.format(metadataRequest.versionId, metadataRequest.correlationId, metadataRequest.clientId, metadataRequest.toString))
val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]()
val config = replicaManager.config
val uniqueTopics = metadataRequest.topics.toSet
val uniqueTopics = {
if(metadataRequest.topics.size > 0)
metadataRequest.topics.toSet
else
ZkUtils.getAllTopics(zkClient).toSet
}
val topicMetadataList = AdminUtils.fetchTopicMetadataFromZk(uniqueTopics, zkClient)
topicMetadataList.foreach(
topicAndMetadata => {
@ -518,7 +527,7 @@ class KafkaApis(val requestChannel: RequestChannel,
case ErrorMapping.UnknownTopicOrPartitionCode =>
try {
/* check if auto creation of topics is turned on */
if (config.autoCreateTopics) {
if (config.autoCreateTopicsEnable) {
try {
CreateTopicCommand.createTopic(zkClient, topicAndMetadata.topic, config.numPartitions, config.defaultReplicationFactor)
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
@ -542,6 +551,7 @@ class KafkaApis(val requestChannel: RequestChannel,
topicsMetadata += topicAndMetadata
}
})
trace("Sending topic metadata for correlation id %d to client %s".format(metadataRequest.correlationId, metadataRequest.clientId))
topicsMetadata.foreach(metadata => trace("Sending topic metadata " + metadata.toString))
val response = new TopicMetadataResponse(topicsMetadata.toSeq, metadataRequest.correlationId)
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))

View File

@ -36,40 +36,40 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
/*********** General Configuration ***********/
/* the broker id for this server */
val brokerId: Int = props.getIntInRange("brokerid", (0, Int.MaxValue))
val brokerId: Int = props.getIntInRange("broker.id", (0, Int.MaxValue))
/* the maximum size of message that the server can receive */
val maxMessageSize = props.getIntInRange("max.message.size", 1000000, (0, Int.MaxValue))
val messageMaxBytes = props.getIntInRange("message.max.bytes", 1000000, (0, Int.MaxValue))
/* the number of network threads that the server uses for handling network requests */
val numNetworkThreads = props.getIntInRange("network.threads", 3, (1, Int.MaxValue))
val numNetworkThreads = props.getIntInRange("num.network.threads", 3, (1, Int.MaxValue))
/* the number of io threads that the server uses for carrying out network requests */
val numIoThreads = props.getIntInRange("io.threads", 8, (1, Int.MaxValue))
val numIoThreads = props.getIntInRange("num.io.threads", 8, (1, Int.MaxValue))
/* the number of threads to use for various background processing tasks */
val backgroundThreads = props.getIntInRange("background.threads", 4, (1, Int.MaxValue))
/* the number of queued requests allowed before blocking the network threads */
val numQueuedRequests = props.getIntInRange("max.queued.requests", 500, (1, Int.MaxValue))
val queuedMaxRequests = props.getIntInRange("queued.max.requests", 500, (1, Int.MaxValue))
/*********** Socket Server Configuration ***********/
/* the port to listen and accept connections on */
val port: Int = props.getInt("port", 6667)
/* hostname of broker. If this is set, it will only bind to this address. If this is not set,
/* hostname of broker. If this is set, it will only bind to this address. If this is not set,
* it will bind to all interfaces, and publish one to ZK */
val hostName: String = props.getString("hostname", null)
val hostName: String = props.getString("host.name", null)
/* the SO_SNDBUFF buffer of the socket sever sockets */
val socketSendBuffer: Int = props.getInt("socket.send.buffer", 100*1024)
val socketSendBufferBytes: Int = props.getInt("socket.send.buffer.bytes", 100*1024)
/* the SO_RCVBUFF buffer of the socket sever sockets */
val socketReceiveBuffer: Int = props.getInt("socket.receive.buffer", 100*1024)
val socketReceiveBufferBytes: Int = props.getInt("socket.receive.buffer.bytes", 100*1024)
/* the maximum number of bytes in a socket request */
val maxSocketRequestSize: Int = props.getIntInRange("max.socket.request.bytes", 100*1024*1024, (1, Int.MaxValue))
val socketRequestMaxBytes: Int = props.getIntInRange("socket.request.max.bytes", 100*1024*1024, (1, Int.MaxValue))
/*********** Log Configuration ***********/
@ -77,58 +77,58 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
val numPartitions = props.getIntInRange("num.partitions", 1, (1, Int.MaxValue))
/* the directories in which the log data is kept */
val logDirs = Utils.parseCsvList(props.getString("log.directories", props.getString("log.dir", "")))
val logDirs = Utils.parseCsvList(props.getString("log.dirs", props.getString("log.dir", "/tmp/kafka-logs")))
require(logDirs.size > 0)
/* the maximum size of a single log file */
val logFileSize = props.getIntInRange("log.file.size", 1*1024*1024*1024, (Message.MinHeaderSize, Int.MaxValue))
val logSegmentBytes = props.getIntInRange("log.segment.bytes", 1*1024*1024*1024, (Message.MinHeaderSize, Int.MaxValue))
/* the maximum size of a single log file for some specific topic */
val logFileSizeMap = props.getMap("topic.log.file.size", _.toInt > 0).mapValues(_.toInt)
val logSegmentBytesPerTopicMap = props.getMap("log.segment.bytes.per.topic", _.toInt > 0).mapValues(_.toInt)
/* the maximum time before a new log segment is rolled out */
val logRollHours = props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue))
/* the number of hours before rolling out a new log segment for some specific topic */
val logRollHoursMap = props.getMap("topic.log.roll.hours", _.toInt > 0).mapValues(_.toInt)
val logRollHoursPerTopicMap = props.getMap("log.roll.hours.per.topic", _.toInt > 0).mapValues(_.toInt)
/* the number of hours to keep a log file before deleting it */
val logRetentionHours = props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue))
/* the number of hours to keep a log file before deleting it for some specific topic*/
val logRetentionHoursMap = props.getMap("topic.log.retention.hours", _.toInt > 0).mapValues(_.toInt)
val logRetentionHoursPerTopicMap = props.getMap("log.retention.hours.per.topic", _.toInt > 0).mapValues(_.toInt)
/* the maximum size of the log before deleting it */
val logRetentionSize = props.getLong("log.retention.size", -1)
val logRetentionBytes = props.getLong("log.retention.bytes", -1)
/* the maximum size of the log for some specific topic before deleting it */
val logRetentionSizeMap = props.getMap("topic.log.retention.size", _.toLong > 0).mapValues(_.toLong)
val logRetentionBytesPerTopicMap = props.getMap("log.retention.bytes.per.topic", _.toLong > 0).mapValues(_.toLong)
/* the frequency in minutes that the log cleaner checks whether any log is eligible for deletion */
val logCleanupIntervalMinutes = props.getIntInRange("log.cleanup.interval.mins", 10, (1, Int.MaxValue))
val logCleanupIntervalMins = props.getIntInRange("log.cleanup.interval.mins", 10, (1, Int.MaxValue))
/* the maximum size in bytes of the offset index */
val logIndexMaxSizeBytes = props.getIntInRange("log.index.max.size", 10*1024*1024, (4, Int.MaxValue))
val logIndexSizeMaxBytes = props.getIntInRange("log.index.size.max.bytes", 10*1024*1024, (4, Int.MaxValue))
/* the interval with which we add an entry to the offset index */
val logIndexIntervalBytes = props.getIntInRange("log.index.interval.bytes", 4096, (0, Int.MaxValue))
/* the number of messages accumulated on a log partition before messages are flushed to disk */
val flushInterval = props.getIntInRange("log.flush.interval", 500, (1, Int.MaxValue))
val logFlushIntervalMessages = props.getIntInRange("log.flush.interval.messages", 500, (1, Int.MaxValue))
val logDeleteDelayMs = props.getLongInRange("log.segment.delete.delay.ms", 60000, (0, Long.MaxValue))
/* the maximum time in ms that a message in selected topics is kept in memory before flushed to disk, e.g., topic1:3000,topic2: 6000 */
val flushIntervalMap = props.getMap("topic.flush.intervals.ms", _.toInt > 0).mapValues(_.toInt)
val logFlushIntervalMsPerTopicMap = props.getMap("log.flush.interval.ms.per.topic", _.toInt > 0).mapValues(_.toInt)
/* the frequency in ms that the log flusher checks whether any log needs to be flushed to disk */
val flushSchedulerThreadRate = props.getInt("log.default.flush.scheduler.interval.ms", 3000)
val logFlushSchedulerIntervalMs = props.getInt("log.flush.scheduler.interval.ms", 3000)
/* the maximum time in ms that a message in any topic is kept in memory before flushed to disk */
val defaultFlushIntervalMs = props.getInt("log.default.flush.interval.ms", flushSchedulerThreadRate)
val logFlushIntervalMs = props.getInt("log.flush.interval.ms", logFlushSchedulerIntervalMs)
/* enable auto creation of topic on the server */
val autoCreateTopics = props.getBoolean("auto.create.topics", true)
val autoCreateTopicsEnable = props.getBoolean("auto.create.topics.enable", true)
/*********** Replication configuration ***********/
@ -141,37 +141,39 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
/* default replication factors for automatically created topics */
val defaultReplicationFactor = props.getInt("default.replication.factor", 1)
val replicaMaxLagTimeMs = props.getLong("replica.max.lag.time.ms", 10000)
/* If a follower hasn't sent any fetch requests during this time, the leader will remove the follower from isr */
val replicaLagTimeMaxMs = props.getLong("replica.lag.time.max.ms", 10000)
val replicaMaxLagBytes = props.getLong("replica.max.lag.bytes", 4000)
/* If the lag in messages between a leader and a follower exceeds this number, the leader will remove the follower from isr */
val replicaLagMaxMessages = props.getLong("replica.lag.max.messages", 4000)
/* the socket timeout for network requests */
val replicaSocketTimeoutMs = props.getInt("replica.socket.timeout.ms", ConsumerConfig.SocketTimeout)
/* the socket receive buffer for network requests */
val replicaSocketBufferSize = props.getInt("replica.socket.buffersize", ConsumerConfig.SocketBufferSize)
val replicaSocketReceiveBufferBytes = props.getInt("replica.socket.receive.buffer.bytes", ConsumerConfig.SocketBufferSize)
/* the number of byes of messages to attempt to fetch */
val replicaFetchSize = props.getInt("replica.fetch.size", ConsumerConfig.FetchSize)
val replicaFetchMaxBytes = props.getInt("replica.fetch.max.bytes", ConsumerConfig.FetchSize)
/* max wait time for each fetcher request issued by follower replicas*/
val replicaMaxWaitTimeMs = props.getInt("replica.fetch.wait.time.ms", 500)
val replicaFetchWaitMaxMs = props.getInt("replica.fetch.wait.max.ms", 500)
/* minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs */
val replicaMinBytes = props.getInt("replica.fetch.min.bytes", 1)
val replicaFetchMinBytes = props.getInt("replica.fetch.min.bytes", 1)
/* number of fetcher threads used to replicate messages from a source broker.
* Increasing this value can increase the degree of I/O parallelism in the follower broker. */
val numReplicaFetchers = props.getInt("replica.fetchers", 1)
val numReplicaFetchers = props.getInt("num.replica.fetchers", 1)
/* the frequency with which the highwater mark is saved out to disk */
val highWaterMarkCheckpointIntervalMs = props.getLong("replica.highwatermark.checkpoint.ms", 5000L)
/* the frequency with which the high watermark is saved out to disk */
val replicaHighWatermarkCheckpointIntervalMs = props.getLong("replica.high.watermark.checkpoint.interval.ms", 5000L)
/* the purge interval (in number of requests) of the fetch request purgatory */
val fetchRequestPurgatoryPurgeInterval = props.getInt("fetch.purgatory.purge.interval", 10000)
val fetchPurgatoryPurgeIntervalRequests = props.getInt("fetch.purgatory.purge.interval.requests", 10000)
/* the purge interval (in number of requests) of the producer request purgatory */
val producerRequestPurgatoryPurgeInterval = props.getInt("producer.purgatory.purge.interval", 10000)
val producerPurgatoryPurgeIntervalRequests = props.getInt("producer.purgatory.purge.interval.requests", 10000)
/*********** Misc configuration ***********/

View File

@ -65,8 +65,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
config.hostName,
config.port,
config.numNetworkThreads,
config.numQueuedRequests,
config.maxSocketRequestSize)
config.queuedMaxRequests,
config.socketRequestMaxBytes)
socketServer.startup

View File

@ -31,11 +31,11 @@ class ReplicaFetcherThread(name:String,
clientId = FetchRequest.ReplicaFetcherClientId,
sourceBroker = sourceBroker,
socketTimeout = brokerConfig.replicaSocketTimeoutMs,
socketBufferSize = brokerConfig.replicaSocketBufferSize,
fetchSize = brokerConfig.replicaFetchSize,
socketBufferSize = brokerConfig.replicaSocketReceiveBufferBytes,
fetchSize = brokerConfig.replicaFetchMaxBytes,
fetcherBrokerId = brokerConfig.brokerId,
maxWait = brokerConfig.replicaMaxWaitTimeMs,
minBytes = brokerConfig.replicaMinBytes) {
maxWait = brokerConfig.replicaFetchWaitMaxMs,
minBytes = brokerConfig.replicaFetchMinBytes) {
// process fetched data
def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: FetchResponsePartitionData) {

View File

@ -72,7 +72,7 @@ class ReplicaManager(val config: KafkaConfig,
def startHighWaterMarksCheckPointThread() = {
if(highWatermarkCheckPointThreadStarted.compareAndSet(false, true))
scheduler.schedule("highwatermark-checkpoint", checkpointHighWatermarks, period = config.highWaterMarkCheckpointIntervalMs, unit = TimeUnit.MILLISECONDS)
scheduler.schedule("highwatermark-checkpoint", checkpointHighWatermarks, period = config.replicaHighWatermarkCheckpointIntervalMs, unit = TimeUnit.MILLISECONDS)
}
/**
@ -91,7 +91,7 @@ class ReplicaManager(val config: KafkaConfig,
def startup() {
// start ISR expiration thread
scheduler.schedule("isr-expiration", maybeShrinkIsr, period = config.replicaMaxLagTimeMs, unit = TimeUnit.MILLISECONDS)
scheduler.schedule("isr-expiration", maybeShrinkIsr, period = config.replicaLagTimeMaxMs, unit = TimeUnit.MILLISECONDS)
}
def stopReplica(topic: String, partitionId: Int, deletePartition: Boolean): Short = {
@ -244,7 +244,7 @@ class ReplicaManager(val config: KafkaConfig,
private def maybeShrinkIsr(): Unit = {
trace("Evaluating ISR list of partitions to see which replicas can be removed from the ISR")
leaderPartitionsLock synchronized {
leaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaMaxLagTimeMs, config.replicaMaxLagBytes))
leaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages))
}
}

View File

@ -29,66 +29,59 @@ import scala.collection._
object ConsumerOffsetChecker extends Logging {
private val consumerMap: mutable.Map[String, Option[SimpleConsumer]] = mutable.Map()
private val consumerMap: mutable.Map[Int, Option[SimpleConsumer]] = mutable.Map()
private val BidPidPattern = """(\d+)-(\d+)""".r
private val BrokerIpPattern = """^([^:]+):(\d+).*$""".r
// e.g., 127.0.0.1:9092:9999 (with JMX port)
private val BrokerIpPattern = """.*:([^:]+):(\d+$)""".r
// e.g., 127.0.0.1-1315436360737:127.0.0.1:9092
// e.g., host.domain.com-1315436360737:host.domain.com:9092
private def getConsumer(zkClient: ZkClient, bid: String): Option[SimpleConsumer] = {
private def getConsumer(zkClient: ZkClient, bid: Int): Option[SimpleConsumer] = {
val brokerInfo = ZkUtils.readDataMaybeNull(zkClient, "/brokers/ids/%s".format(bid))._1
val consumer = brokerInfo match {
case BrokerIpPattern(ip, port) =>
case Some(BrokerIpPattern(ip, port)) =>
Some(new SimpleConsumer(ip, port.toInt, 10000, 100000, "ConsumerOffsetChecker"))
case _ =>
error("Could not parse broker info %s".format(brokerInfo))
error("Could not parse broker info %s with regex %s".format(brokerInfo, BrokerIpPattern.toString()))
None
}
consumer
}
private def processPartition(zkClient: ZkClient,
group: String, topic: String, bidPid: String) {
group: String, topic: String, pid: Int) {
val offset = ZkUtils.readData(zkClient, "/consumers/%s/offsets/%s/%s".
format(group, topic, bidPid))._1.toLong
format(group, topic, pid))._1.toLong
val owner = ZkUtils.readDataMaybeNull(zkClient, "/consumers/%s/owners/%s/%s".
format(group, topic, bidPid))._1
println("%s,%s,%s (Group,Topic,BrokerId-PartitionId)".format(group, topic, bidPid))
println("%20s%s".format("Owner = ", owner))
println("%20s%d".format("Consumer offset = ", offset))
println("%20s%,d (%,.2fG)".format("= ", offset, offset / math.pow(1024, 3)))
format(group, topic, pid))._1
bidPid match {
case BidPidPattern(bid, pid) =>
val consumerOpt = consumerMap.getOrElseUpdate(
bid, getConsumer(zkClient, bid))
ZkUtils.getLeaderForPartition(zkClient, topic, pid) match {
case Some(bid) =>
val consumerOpt = consumerMap.getOrElseUpdate(bid, getConsumer(zkClient, bid))
consumerOpt match {
case Some(consumer) =>
val topicAndPartition = TopicAndPartition(topic, pid.toInt)
val topicAndPartition = TopicAndPartition(topic, pid)
val request =
OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1)))
val logSize = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
println("%20s%d".format("Log size = ", logSize))
println("%20s%,d (%,.2fG)".format("= ", logSize, logSize / math.pow(1024, 3)))
val lag = logSize - offset
println("%20s%d".format("Consumer lag = ", lag))
println("%20s%,d (%,.2fG)".format("= ", lag, lag / math.pow(1024, 3)))
println()
println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offset, logSize, lag,
owner match {case Some(ownerStr) => ownerStr case None => "none"}))
consumer.close()
case None => // ignore
}
case _ =>
error("Could not parse broker/partition pair %s".format(bidPid))
case None =>
error("No broker for partition %s - %s".format(topic, pid))
}
}
private def processTopic(zkClient: ZkClient, group: String, topic: String) {
val bidsPids = ZkUtils.getChildrenParentMayNotExist(
zkClient, "/consumers/%s/offsets/%s".format(group, topic)).toList
bidsPids.sorted.foreach {
bidPid => processPartition(zkClient, group, topic, bidPid)
val pidMap = ZkUtils.getPartitionsForTopics(zkClient, Seq(topic))
pidMap.get(topic) match {
case Some(pids) =>
pids.sorted.foreach {
pid => processPartition(zkClient, group, topic, pid)
}
case None => // ignore
}
}
@ -112,6 +105,7 @@ object ConsumerOffsetChecker extends Logging {
withRequiredArg().ofType(classOf[String])
val groupOpt = parser.accepts("group", "Consumer group.").
withRequiredArg().ofType(classOf[String])
parser.accepts("broker-info", "Print broker info")
parser.accepts("help", "Print this message.")
val options = parser.parse(args : _*)
@ -147,11 +141,14 @@ object ConsumerOffsetChecker extends Logging {
debug("zkConnect = %s; topics = %s; group = %s".format(
zkConnect, topicList.toString(), group))
println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format("Group", "Topic", "Pid", "Offset", "logSize", "Lag", "Owner"))
topicList.sorted.foreach {
topic => processTopic(zkClient, group, topic)
}
printBrokerInfo()
if (options.has("broker-info"))
printBrokerInfo();
}
finally {
for (consumerOpt <- consumerMap.values) {

View File

@ -182,9 +182,9 @@ public class KafkaMigrationTool
Properties kafkaConsumerProperties_07 = new Properties();
kafkaConsumerProperties_07.load(new FileInputStream(consumerConfigFile_07));
/** Disable shallow iteration because the message format is different between 07 and 08, we have to get each individual message **/
if(kafkaConsumerProperties_07.getProperty("shallowiterator.enable", "").equals("true")){
if(kafkaConsumerProperties_07.getProperty("shallow.iterator.enable", "").equals("true")){
logger.warn("Shallow iterator should not be used in the migration tool");
kafkaConsumerProperties_07.setProperty("shallowiterator.enable", "false");
kafkaConsumerProperties_07.setProperty("shallow.iterator.enable", "false");
}
Object consumerConfig_07 = ConsumerConfigConstructor_07.newInstance(kafkaConsumerProperties_07);

View File

@ -42,12 +42,12 @@ object ReplayLogProducer extends Logging {
// consumer properties
val consumerProps = new Properties
consumerProps.put("groupid", GroupId)
consumerProps.put("group.id", GroupId)
consumerProps.put("zk.connect", config.zkConnect)
consumerProps.put("consumer.timeout.ms", "10000")
consumerProps.put("autooffset.reset", OffsetRequest.SmallestTimeString)
consumerProps.put("fetch.size", (1024*1024).toString)
consumerProps.put("socket.buffer.size", (2 * 1024 * 1024).toString)
consumerProps.put("auto.offset.reset", OffsetRequest.SmallestTimeString)
consumerProps.put("fetch.message.max.bytes", (1024*1024).toString)
consumerProps.put("socket.receive.buffer.bytes", (2 * 1024 * 1024).toString)
val consumerConfig = new ConsumerConfig(consumerProps)
val consumerConnector: ConsumerConnector = Consumer.create(consumerConfig)
val topicMessageStreams = consumerConnector.createMessageStreams(Predef.Map(config.inputTopic -> config.numThreads))
@ -141,10 +141,10 @@ object ReplayLogProducer extends Logging {
val props = new Properties()
props.put("broker.list", config.brokerList)
props.put("reconnect.interval", Integer.MAX_VALUE.toString)
props.put("buffer.size", (64*1024).toString)
props.put("send.buffer.bytes", (64*1024).toString)
props.put("compression.codec", config.compressionCodec.codec.toString)
props.put("batch.size", config.batchSize.toString)
props.put("queue.enqueueTimeout.ms", "-1")
props.put("batch.num.messages", config.batchSize.toString)
props.put("queue.enqueue.timeout.ms", "-1")
if(config.isAsync)
props.put("producer.type", "async")

View File

@ -785,11 +785,11 @@ class ZKConfig(props: VerifiableProperties) {
val zkConnect = props.getString("zk.connect", null)
/** zookeeper session timeout */
val zkSessionTimeoutMs = props.getInt("zk.sessiontimeout.ms", 6000)
val zkSessionTimeoutMs = props.getInt("zk.session.timeout.ms", 6000)
/** the max time that the client waits to establish a connection to zookeeper */
val zkConnectionTimeoutMs = props.getInt("zk.connectiontimeout.ms",zkSessionTimeoutMs)
val zkConnectionTimeoutMs = props.getInt("zk.connection.timeout.ms",zkSessionTimeoutMs)
/** how far a ZK follower can be behind a ZK leader */
val zkSyncTimeMs = props.getInt("zk.synctime.ms", 2000)
val zkSyncTimeMs = props.getInt("zk.sync.time.ms", 2000)
}

View File

@ -35,9 +35,9 @@ object TestEndToEndLatency {
val topic = "test"
val consumerProps = new Properties()
consumerProps.put("groupid", topic)
consumerProps.put("group.id", topic)
consumerProps.put("auto.commit", "true")
consumerProps.put("autooffset.reset", "largest")
consumerProps.put("auto.offset.reset", "largest")
consumerProps.put("zk.connect", zkConnect)
consumerProps.put("socket.timeout.ms", 1201000.toString)

View File

@ -34,7 +34,7 @@ object TestLogPerformance {
val config = new KafkaConfig(props)
val dir = TestUtils.tempDir()
val scheduler = new KafkaScheduler(1)
val log = new Log(dir, scheduler, 50*1024*1024, config.maxMessageSize, 5000000, config.logRollHours*60*60*1000L, needsRecovery = false, segmentDeleteDelayMs = 0, time = SystemTime)
val log = new Log(dir, scheduler, 50*1024*1024, config.messageMaxBytes, 5000000, config.logRollHours*60*60*1000L, needsRecovery = false, segmentDeleteDelayMs = 0, time = SystemTime)
val bytes = new Array[Byte](messageSize)
new java.util.Random().nextBytes(bytes)
val message = new Message(bytes)

View File

@ -31,7 +31,7 @@ object TestZKConsumerOffsets {
val topic = args(1)
val autoOffsetReset = args(2)
val props = Utils.loadProps(args(0))
props.put("autooffset.reset", "largest")
props.put("auto.offset.reset", "largest")
val config = new ConsumerConfig(props)
val consumerConnector: ConsumerConnector = Consumer.create(config)

View File

@ -89,7 +89,7 @@ object SerializationTestUtils{
val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1)
val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, 3)),
((topic2, 0), PartitionStateInfo(leaderAndIsr2, 3)))
new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 1)
new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 1, 0)
}
def createTestLeaderAndIsrResponse() : LeaderAndIsrResponse = {
@ -99,7 +99,7 @@ object SerializationTestUtils{
}
def createTestStopReplicaRequest() : StopReplicaRequest = {
new StopReplicaRequest(controllerEpoch = 1, deletePartitions = true, partitions = collection.immutable.Set((topic1, 0), (topic2, 0)))
new StopReplicaRequest(controllerEpoch = 1, correlationId = 0, deletePartitions = true, partitions = collection.immutable.Set((topic1, 0), (topic2, 0)))
}
def createTestStopReplicaResponse() : StopReplicaResponse = {
@ -138,7 +138,7 @@ object SerializationTestUtils{
}
def createTestTopicMetadataRequest: TopicMetadataRequest = {
new TopicMetadataRequest(1, "client 1", Seq(topic1, topic2), 1)
new TopicMetadataRequest(1, 1, "client 1", Seq(topic1, topic2))
}
def createTestTopicMetadataResponse: TopicMetadataResponse = {

View File

@ -78,9 +78,9 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L
// update offset in zookeeper for consumer to jump "forward" in time
val dirs = new ZKGroupTopicDirs(group, topic)
var consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer)
consumerProps.put("autooffset.reset", resetTo)
consumerProps.put("auto.offset.reset", resetTo)
consumerProps.put("consumer.timeout.ms", "2000")
consumerProps.put("max.fetch.wait.ms", "0")
consumerProps.put("fetch.wait.max.ms", "0")
val consumerConfig = new ConsumerConfig(consumerProps)
TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0", offset)

View File

@ -35,12 +35,12 @@ trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarnes
val props = new Properties()
props.put("partitioner.class", "kafka.utils.StaticPartitioner")
props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs))
props.put("buffer.size", "65536")
props.put("send.buffer.bytes", "65536")
props.put("connect.timeout.ms", "100000")
props.put("reconnect.interval", "10000")
props.put("producer.retry.backoff.ms", "1000")
props.put("producer.num.retries", "3")
props.put("producer.request.required.acks", "-1")
props.put("retry.backoff.ms", "1000")
props.put("message.send.max.retries", "3")
props.put("request.required.acks", "-1")
props.put("serializer.class", classOf[StringEncoder].getName.toString)
producer = new Producer(new ProducerConfig(props))
consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "")

View File

@ -51,7 +51,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
CreateTopicCommand.createTopic(zkClient, topic, 1)
// create a topic metadata request
val topicMetadataRequest = new TopicMetadataRequest(List(topic))
val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0)
val serializedMetadataRequest = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes + 2)
topicMetadataRequest.writeTo(serializedMetadataRequest)
@ -70,7 +70,27 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
0 -> configs.head.brokerId
)
TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1)
val topicMetadata = mockLogManagerAndTestTopic(topic)
val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0)
val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest)
assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size)
assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic)
val partitionMetadata = topicMetadata.head.partitionsMetadata
assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size)
assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId)
assertEquals(1, partitionMetadata.head.replicas.size)
}
def testGetAllTopicMetadata {
// create topic
val topic = "test"
CreateTopicCommand.createTopic(zkClient, topic, 1)
// set up leader for topic partition 0
val leaderForPartitionMap = Map(
0 -> configs.head.brokerId
)
TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1)
val topicMetadataRequest = new TopicMetadataRequest(List(), 0)
val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest)
assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size)
assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic)
val partitionMetadata = topicMetadata.head.partitionsMetadata
@ -83,7 +103,8 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
// auto create topic
val topic = "test"
val topicMetadata = mockLogManagerAndTestTopic(topic)
val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0)
val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest)
assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size)
assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic)
val partitionMetadata = topicMetadata.head.partitionsMetadata
@ -94,16 +115,14 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
assertEquals(ErrorMapping.LeaderNotAvailableCode, partitionMetadata.head.errorCode)
}
private def mockLogManagerAndTestTopic(topic: String): Seq[TopicMetadata] = {
private def mockLogManagerAndTestTopic(request: TopicMetadataRequest): Seq[TopicMetadata] = {
// topic metadata request only requires 1 call from the replica manager
val replicaManager = EasyMock.createMock(classOf[ReplicaManager])
EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes()
EasyMock.replay(replicaManager)
// create a topic metadata request
val topicMetadataRequest = new TopicMetadataRequest(List(topic))
val serializedMetadataRequest = TestUtils.createRequestByteBuffer(topicMetadataRequest)
val serializedMetadataRequest = TestUtils.createRequestByteBuffer(request)
// create the kafka request handler
val requestChannel = new RequestChannel(2, 5)

View File

@ -39,8 +39,8 @@ class LogManagerTest extends JUnit3Suite {
override def setUp() {
super.setUp()
config = new KafkaConfig(TestUtils.createBrokerConfig(0, -1)) {
override val logFileSize = 1024
override val flushInterval = 10000
override val logSegmentBytes = 1024
override val logFlushIntervalMessages = 10000
override val logRetentionHours = maxLogAgeHours
}
logManager = new LogManager(config, time.scheduler, time)
@ -119,8 +119,8 @@ class LogManagerTest extends JUnit3Suite {
val props = TestUtils.createBrokerConfig(0, -1)
logManager.shutdown()
config = new KafkaConfig(props) {
override val logFileSize = (10 * (setSize - 1)) // each segment will be 10 messages
override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Long]
override val logSegmentBytes = (10 * (setSize - 1)) // each segment will be 10 messages
override val logRetentionBytes = (5 * 10 * setSize + 10).asInstanceOf[Long]
override val logRollHours = maxRollInterval
}
logManager = new LogManager(config, time.scheduler, time)
@ -139,7 +139,7 @@ class LogManagerTest extends JUnit3Suite {
}
// should be exactly 100 full segments + 1 new empty one
assertEquals("Check we have the expected number of segments.", numMessages * setSize / config.logFileSize, log.numberOfSegments)
assertEquals("Check we have the expected number of segments.", numMessages * setSize / config.logSegmentBytes, log.numberOfSegments)
// this cleanup shouldn't find any expired segments but should delete some to reduce size
time.sleep(logManager.InitialTaskDelayMs)
@ -165,9 +165,9 @@ class LogManagerTest extends JUnit3Suite {
val props = TestUtils.createBrokerConfig(0, -1)
logManager.shutdown()
config = new KafkaConfig(props) {
override val flushSchedulerThreadRate = 1000
override val defaultFlushIntervalMs = 1000
override val flushInterval = Int.MaxValue
override val logFlushSchedulerIntervalMs = 1000
override val logFlushIntervalMs = 1000
override val logFlushIntervalMessages = Int.MaxValue
}
logManager = new LogManager(config, time.scheduler, time)
logManager.startup
@ -191,7 +191,7 @@ class LogManagerTest extends JUnit3Suite {
val dirs = Seq(TestUtils.tempDir().getAbsolutePath,
TestUtils.tempDir().getAbsolutePath,
TestUtils.tempDir().getAbsolutePath)
props.put("log.directories", dirs.mkString(","))
props.put("log.dirs", dirs.mkString(","))
logManager.shutdown()
logManager = new LogManager(new KafkaConfig(props), time.scheduler, time)

View File

@ -65,7 +65,7 @@ class LogTest extends JUnitSuite {
val time: MockTime = new MockTime()
// create a log
val log = new Log(logDir, time.scheduler, 1000, config.maxMessageSize, 1000, rollMs, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 1000, config.messageMaxBytes, 1000, rollMs, needsRecovery = false, time = time)
time.sleep(rollMs + 1)
// segment age is less than its limit
@ -98,7 +98,7 @@ class LogTest extends JUnitSuite {
val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
// create a log
val log = new Log(logDir, time.scheduler, logFileSize, config.maxMessageSize, 1000, 10000, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time)
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
// segments expire in size
@ -114,7 +114,7 @@ class LogTest extends JUnitSuite {
@Test
def testLoadEmptyLog() {
createEmptyLogs(logDir, 0)
val log = new Log(logDir, time.scheduler, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 1024, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
log.append(TestUtils.singleMessageSet("test".getBytes))
}
@ -123,7 +123,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testAppendAndReadWithSequentialOffsets() {
val log = new Log(logDir, time.scheduler, 71, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 71, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray
for(i <- 0 until messages.length)
@ -142,7 +142,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testAppendAndReadWithNonSequentialOffsets() {
val log = new Log(logDir, time.scheduler, 71, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 71, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
val messages = messageIds.map(id => new Message(id.toString.getBytes))
@ -165,7 +165,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testReadAtLogGap() {
val log = new Log(logDir, time.scheduler, 300, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 300, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
// keep appending until we have two segments with only a single message in the second segment
while(log.numberOfSegments == 1)
@ -185,7 +185,7 @@ class LogTest extends JUnitSuite {
@Test
def testReadOutOfRange() {
createEmptyLogs(logDir, 1024)
val log = new Log(logDir, time.scheduler, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 1024, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1024, 1000).sizeInBytes)
try {
log.read(0, 1024)
@ -208,7 +208,7 @@ class LogTest extends JUnitSuite {
@Test
def testLogRolls() {
/* create a multipart log with 100 messages */
val log = new Log(logDir, time.scheduler, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 100, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val numMessages = 100
val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes))
val offsets = messageSets.map(log.append(_).firstOffset)
@ -232,8 +232,8 @@ class LogTest extends JUnitSuite {
@Test
def testCompressedMessages() {
/* this log should roll after every messageset */
val log = new Log(logDir, time.scheduler, 10, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 10, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
/* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)))
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes)))
@ -255,7 +255,7 @@ class LogTest extends JUnitSuite {
for(messagesToAppend <- List(0, 1, 25)) {
logDir.mkdirs()
// first test a log segment starting at 0
val log = new Log(logDir, time.scheduler, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, 100, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
for(i <- 0 until messagesToAppend)
log.append(TestUtils.singleMessageSet(i.toString.getBytes))
@ -311,7 +311,7 @@ class LogTest extends JUnitSuite {
val messageSize = 100
val segmentSize = 7 * messageSize
val indexInterval = 3 * messageSize
var log = new Log(logDir, time.scheduler, segmentSize, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
var log = new Log(logDir, time.scheduler, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
for(i <- 0 until numMessages)
log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(messageSize)))
assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset)
@ -320,14 +320,14 @@ class LogTest extends JUnitSuite {
log.close()
// test non-recovery case
log = new Log(logDir, time.scheduler, segmentSize, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
log = new Log(logDir, time.scheduler, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
assertEquals("Should have %d messages when log is reopened w/o recovery".format(numMessages), numMessages, log.logEndOffset)
assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset)
assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries)
log.close()
// test recovery case
log = new Log(logDir, time.scheduler, segmentSize, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
log = new Log(logDir, time.scheduler, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
assertEquals("Should have %d messages when log is reopened with recovery".format(numMessages), numMessages, log.logEndOffset)
assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset)
assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries)
@ -341,7 +341,7 @@ class LogTest extends JUnitSuite {
def testIndexRebuild() {
// publish the messages and close the log
val numMessages = 200
var log = new Log(logDir, time.scheduler, 200, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = 1, maxIndexSize = 4096)
var log = new Log(logDir, time.scheduler, 200, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = 1, maxIndexSize = 4096)
for(i <- 0 until numMessages)
log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10)))
val indexFiles = log.logSegments.map(_.index.file)
@ -351,7 +351,7 @@ class LogTest extends JUnitSuite {
indexFiles.foreach(_.delete())
// reopen the log
log = new Log(logDir, time.scheduler, 200, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = 1, maxIndexSize = 4096)
log = new Log(logDir, time.scheduler, 200, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = 1, maxIndexSize = 4096)
assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
for(i <- 0 until numMessages)
@ -370,7 +370,7 @@ class LogTest extends JUnitSuite {
val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
// create a log
val log = new Log(logDir, time.scheduler, logFileSize, config.maxMessageSize, 1000, 10000, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time)
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
for (i<- 1 to msgPerSeg)
@ -422,7 +422,7 @@ class LogTest extends JUnitSuite {
val setSize = set.sizeInBytes
val msgPerSeg = 10
val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
val log = new Log(logDir, time.scheduler, logFileSize, config.maxMessageSize, 1000, 10000, needsRecovery = false, time = time)
val log = new Log(logDir, time.scheduler, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time)
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
for (i<- 1 to msgPerSeg)
log.append(set)
@ -438,7 +438,7 @@ class LogTest extends JUnitSuite {
log.append(set)
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.
*/
@ -451,8 +451,8 @@ class LogTest extends JUnitSuite {
val log = new Log(logDir,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.maxMessageSize,
maxIndexSize = 1000,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 1,
needsRecovery = false)
@ -477,8 +477,8 @@ class LogTest extends JUnitSuite {
var log = new Log(logDir,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.maxMessageSize,
maxIndexSize = 1000,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
needsRecovery = true)
@ -489,8 +489,8 @@ class LogTest extends JUnitSuite {
log = new Log(logDir,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.maxMessageSize,
maxIndexSize = 1000,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
needsRecovery = true)
log.truncateTo(3)
@ -508,7 +508,7 @@ class LogTest extends JUnitSuite {
val log = new Log(logDir,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.maxMessageSize,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
segmentDeleteDelayMs = asyncDeleteMs,
@ -540,7 +540,7 @@ class LogTest extends JUnitSuite {
var log = new Log(logDir,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.maxMessageSize,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
needsRecovery = false)
@ -555,7 +555,7 @@ class LogTest extends JUnitSuite {
log = new Log(logDir,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.maxMessageSize,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
needsRecovery = false)

View File

@ -63,8 +63,8 @@ class AsyncProducerTest extends JUnit3Suite {
props.put("serializer.class", "kafka.serializer.StringEncoder")
props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs))
props.put("producer.type", "async")
props.put("queue.size", "10")
props.put("batch.size", "1")
props.put("queue.buffering.max.messages", "10")
props.put("batch.num.messages", "1")
val config = new ProducerConfig(props)
val produceData = getProduceData(12)
@ -87,7 +87,7 @@ class AsyncProducerTest extends JUnit3Suite {
props.put("serializer.class", "kafka.serializer.StringEncoder")
props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs))
props.put("producer.type", "async")
props.put("batch.size", "1")
props.put("batch.num.messages", "1")
val config = new ProducerConfig(props)
val produceData = getProduceData(10)
@ -191,7 +191,7 @@ class AsyncProducerTest extends JUnit3Suite {
producerPool = producerPool,
topicPartitionInfos = topicPartitionInfos)
val topic1Broker1Data =
val topic1Broker1Data =
ArrayBuffer[KeyedMessage[Int,Message]](new KeyedMessage[Int,Message]("topic1", 0, new Message("msg1".getBytes)),
new KeyedMessage[Int,Message]("topic1", 2, new Message("msg3".getBytes)))
val topic1Broker2Data = ArrayBuffer[KeyedMessage[Int,Message]](new KeyedMessage[Int,Message]("topic1", 3, new Message("msg4".getBytes)))
@ -358,7 +358,7 @@ class AsyncProducerTest extends JUnit3Suite {
val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder")
props.put("producer.type", "async")
props.put("batch.size", "5")
props.put("batch.num.messages", "5")
props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs))
val config = new ProducerConfig(props)
@ -410,12 +410,12 @@ class AsyncProducerTest extends JUnit3Suite {
// produce request for topic1 and partitions 0 and 1. Let the first request fail
// entirely. The second request will succeed for partition 1 but fail for partition 0.
// On the third try for partition 0, let it succeed.
val request1 = TestUtils.produceRequestWithAcks(List(topic1), List(0, 1), messagesToSet(msgs), acks = 0, correlationId = 0)
val request2 = TestUtils.produceRequestWithAcks(List(topic1), List(0, 1), messagesToSet(msgs), acks = 0, correlationId = 1)
val request1 = TestUtils.produceRequestWithAcks(List(topic1), List(0, 1), messagesToSet(msgs), acks = 0, correlationId = 11)
val request2 = TestUtils.produceRequestWithAcks(List(topic1), List(0, 1), messagesToSet(msgs), acks = 0, correlationId = 17)
val response1 = ProducerResponse(0,
Map((TopicAndPartition("topic1", 0), ProducerResponseStatus(ErrorMapping.NotLeaderForPartitionCode.toShort, 0L)),
(TopicAndPartition("topic1", 1), ProducerResponseStatus(ErrorMapping.NoError, 0L))))
val request3 = TestUtils.produceRequest(topic1, 0, messagesToSet(msgs), correlationId = 2)
val request3 = TestUtils.produceRequest(topic1, 0, messagesToSet(msgs), correlationId = 21)
val response2 = ProducerResponse(0,
Map((TopicAndPartition("topic1", 0), ProducerResponseStatus(ErrorMapping.NoError, 0L))))
val mockSyncProducer = EasyMock.createMock(classOf[SyncProducer])
@ -425,9 +425,7 @@ class AsyncProducerTest extends JUnit3Suite {
EasyMock.replay(mockSyncProducer)
val producerPool = EasyMock.createMock(classOf[ProducerPool])
EasyMock.expect(producerPool.getProducer(0)).andReturn(mockSyncProducer)
EasyMock.expect(producerPool.getProducer(0)).andReturn(mockSyncProducer)
EasyMock.expect(producerPool.getProducer(0)).andReturn(mockSyncProducer)
EasyMock.expect(producerPool.getProducer(0)).andReturn(mockSyncProducer).times(4)
EasyMock.expect(producerPool.close())
EasyMock.replay(producerPool)

View File

@ -140,13 +140,13 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
props1.put("serializer.class", "kafka.serializer.StringEncoder")
props1.put("partitioner.class", "kafka.utils.StaticPartitioner")
props1.put("broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
props1.put("producer.request.required.acks", "2")
props1.put("producer.request.timeout.ms", "1000")
props1.put("request.required.acks", "2")
props1.put("request.timeout.ms", "1000")
val props2 = new util.Properties()
props2.putAll(props1)
props2.put("producer.request.required.acks", "3")
props2.put("producer.request.timeout.ms", "1000")
props2.put("request.required.acks", "3")
props2.put("request.timeout.ms", "1000")
val producerConfig1 = new ProducerConfig(props1)
val producerConfig2 = new ProducerConfig(props2)
@ -198,8 +198,8 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder")
props.put("partitioner.class", "kafka.utils.StaticPartitioner")
props.put("producer.request.timeout.ms", "2000")
// props.put("producer.request.required.acks", "-1")
props.put("request.timeout.ms", "2000")
// props.put("request.required.acks", "-1")
props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
// create topic
@ -256,7 +256,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
val props = new Properties()
props.put("serializer.class", "kafka.serializer.StringEncoder")
props.put("partitioner.class", "kafka.utils.StaticPartitioner")
props.put("producer.request.timeout.ms", String.valueOf(timeoutMs))
props.put("request.timeout.ms", String.valueOf(timeoutMs))
props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
val config = new ProducerConfig(props)
@ -300,7 +300,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
// make sure we don't wait fewer than numRetries*timeoutMs milliseconds
// we do this because the DefaultEventHandler retries a number of times
assertTrue((t2-t1) >= timeoutMs*config.producerRetries)
assertTrue((t2-t1) >= timeoutMs*config.messageSendMaxRetries)
}
}

View File

@ -41,7 +41,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
val props = new Properties()
props.put("host", "localhost")
props.put("port", server.socketServer.port.toString)
props.put("buffer.size", "102400")
props.put("send.buffer.bytes", "102400")
props.put("connect.timeout.ms", "500")
props.put("reconnect.interval", "1000")
val producer = new SyncProducer(new SyncProducerConfig(props))
@ -77,10 +77,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
val props = new Properties()
props.put("host", "localhost")
props.put("port", server.socketServer.port.toString)
props.put("buffer.size", "102400")
props.put("send.buffer.bytes", "102400")
props.put("connect.timeout.ms", "300")
props.put("reconnect.interval", "500")
props.put("max.message.size", "100")
val correlationId = 0
val clientId = SyncProducerConfig.DefaultClientId
val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs
@ -98,12 +97,11 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
val props = new Properties()
props.put("host", "localhost")
props.put("port", server.socketServer.port.toString)
props.put("max.message.size", 50000.toString)
val producer = new SyncProducer(new SyncProducerConfig(props))
CreateTopicCommand.createTopic(zkClient, "test", 1, 1)
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0, 500)
val message1 = new Message(new Array[Byte](configs(0).maxMessageSize + 1))
val message1 = new Message(new Array[Byte](configs(0).messageMaxBytes + 1))
val messageSet1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message1)
val response1 = producer.send(TestUtils.produceRequest("test", 0, messageSet1))
@ -111,7 +109,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
Assert.assertEquals(ErrorMapping.MessageSizeTooLargeCode, response1.status(TopicAndPartition("test", 0)).error)
Assert.assertEquals(-1L, response1.status(TopicAndPartition("test", 0)).offset)
val safeSize = configs(0).maxMessageSize - Message.MessageOverhead - MessageSet.LogOverhead - 1
val safeSize = configs(0).messageMaxBytes - Message.MessageOverhead - MessageSet.LogOverhead - 1
val message2 = new Message(new Array[Byte](safeSize))
val messageSet2 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message2)
val response2 = producer.send(TestUtils.produceRequest("test", 0, messageSet2))
@ -127,10 +125,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
val props = new Properties()
props.put("host", "localhost")
props.put("port", server.socketServer.port.toString)
props.put("buffer.size", "102400")
props.put("send.buffer.bytes", "102400")
props.put("connect.timeout.ms", "300")
props.put("reconnect.interval", "500")
props.put("max.message.size", "100")
val producer = new SyncProducer(new SyncProducerConfig(props))
val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes))
@ -179,8 +176,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
val props = new Properties()
props.put("host", "localhost")
props.put("port", server.socketServer.port.toString)
props.put("buffer.size", "102400")
props.put("producer.request.timeout.ms", String.valueOf(timeoutMs))
props.put("send.buffer.bytes", "102400")
props.put("request.timeout.ms", String.valueOf(timeoutMs))
val producer = new SyncProducer(new SyncProducerConfig(props))
val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes))

View File

@ -29,8 +29,8 @@ class IsrExpirationTest extends JUnit3Suite {
var topicPartitionIsr: Map[(String, Int), Seq[Int]] = new HashMap[(String, Int), Seq[Int]]()
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) {
override val replicaMaxLagTimeMs = 100L
override val replicaMaxLagBytes = 10L
override val replicaLagTimeMaxMs = 100L
override val replicaLagMaxMessages = 10L
})
val topic = "foo"
@ -45,7 +45,7 @@ class IsrExpirationTest extends JUnit3Suite {
// let the follower catch up to 10
(partition0.assignedReplicas() - leaderReplica).foreach(r => r.logEndOffset = 10)
var partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaMaxLagTimeMs, configs.head.replicaMaxLagBytes)
var partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs, configs.head.replicaLagMaxMessages)
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
// let some time pass
@ -53,7 +53,7 @@ class IsrExpirationTest extends JUnit3Suite {
// now follower (broker id 1) has caught up to only 10, while the leader is at 15 AND the follower hasn't
// pulled any data for > replicaMaxLagTimeMs ms. So it is stuck
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaMaxLagTimeMs, configs.head.replicaMaxLagBytes)
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs, configs.head.replicaLagMaxMessages)
assertEquals("Replica 1 should be out of sync", Set(configs.last.brokerId), partition0OSR.map(_.brokerId))
EasyMock.verify(log)
}
@ -71,7 +71,7 @@ class IsrExpirationTest extends JUnit3Suite {
// now follower (broker id 1) has caught up to only 4, while the leader is at 15. Since the gap it larger than
// replicaMaxLagBytes, the follower is out of sync.
val partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaMaxLagTimeMs, configs.head.replicaMaxLagBytes)
val partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs, configs.head.replicaLagMaxMessages)
assertEquals("Replica 1 should be out of sync", Set(configs.last.brokerId), partition0OSR.map(_.brokerId))
EasyMock.verify(log)

View File

@ -131,7 +131,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness {
leaderAndIsr.put((topic, partitionId),
new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2))
val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, 1)).toMap
val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, staleControllerEpoch)
val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, staleControllerEpoch, 0)
controllerChannelManager.sendRequest(brokerId2, leaderAndIsrRequest, staleControllerEpochCallback)
TestUtils.waitUntilTrue(() => staleControllerEpochDetected == true, 1000)

View File

@ -198,15 +198,15 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
private def createBrokerConfig(nodeId: Int, port: Int): Properties = {
val props = new Properties
props.put("brokerid", nodeId.toString)
props.put("broker.id", nodeId.toString)
props.put("port", port.toString)
props.put("log.dir", getLogDir.getAbsolutePath)
props.put("log.flush.interval", "1")
props.put("log.flush.interval.messages", "1")
props.put("enable.zookeeper", "false")
props.put("num.partitions", "20")
props.put("log.retention.hours", "10")
props.put("log.cleanup.interval.mins", "5")
props.put("log.file.size", logSize.toString)
props.put("log.segment.bytes", logSize.toString)
props.put("zk.connect", zkConnect.toString)
props
}

View File

@ -28,9 +28,9 @@ import kafka.producer.{ProducerConfig, KeyedMessage, Producer}
class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) {
override val replicaMaxLagTimeMs = 5000L
override val replicaMaxLagBytes = 10L
override val replicaMinBytes = 20
override val replicaLagTimeMaxMs = 5000L
override val replicaLagMaxMessages = 10L
override val replicaFetchMinBytes = 20
})
val topic = "new-topic"
val partitionId = 0
@ -50,7 +50,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
val producerProps = getProducerConfig(TestUtils.getBrokerListStrFromConfigs(configs), 64*1024, 100000, 10000)
producerProps.put("key.serializer.class", classOf[IntEncoder].getName.toString)
producerProps.put("producer.request.required.acks", "-1")
producerProps.put("request.required.acks", "-1")
def testHWCheckpointNoFailuresSingleLogSegment {

View File

@ -33,8 +33,8 @@ import kafka.common.TopicAndPartition
class SimpleFetchTest extends JUnit3Suite {
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) {
override val replicaMaxLagTimeMs = 100L
override val replicaMaxLagBytes = 10L
override val replicaLagTimeMaxMs = 100L
override val replicaLagMaxMessages = 10L
})
val topic = "foo"
val partitionId = 0

View File

@ -123,8 +123,8 @@ object TestUtils extends Logging {
*/
def createBrokerConfig(nodeId: Int, port: Int): Properties = {
val props = new Properties
props.put("brokerid", nodeId.toString)
props.put("hostname", "localhost")
props.put("broker.id", nodeId.toString)
props.put("host.name", "localhost")
props.put("port", port.toString)
props.put("log.dir", TestUtils.tempDir().getAbsolutePath)
props.put("zk.connect", TestZKUtils.zookeeperConnect)
@ -139,13 +139,13 @@ object TestUtils extends Logging {
consumerTimeout: Long = -1): Properties = {
val props = new Properties
props.put("zk.connect", zkConnect)
props.put("groupid", groupId)
props.put("consumerid", consumerId)
props.put("group.id", groupId)
props.put("consumer.id", consumerId)
props.put("consumer.timeout.ms", consumerTimeout.toString)
props.put("zk.sessiontimeout.ms", "400")
props.put("zk.synctime.ms", "200")
props.put("autocommit.interval.ms", "1000")
props.put("rebalance.retries.max", "4")
props.put("zk.session.timeout.ms", "400")
props.put("zk.sync.time.ms", "200")
props.put("auto.commit.interval.ms", "1000")
props.put("rebalance.max.retries", "4")
props
}
@ -292,7 +292,7 @@ object TestUtils extends Logging {
keyEncoder: Encoder[K] = new DefaultEncoder()): Producer[K, V] = {
val props = new Properties()
props.put("broker.list", brokerList)
props.put("buffer.size", "65536")
props.put("send.buffer.bytes", "65536")
props.put("connect.timeout.ms", "100000")
props.put("reconnect.interval", "10000")
props.put("serializer.class", encoder.getClass.getCanonicalName)
@ -306,10 +306,10 @@ object TestUtils extends Logging {
props.put("producer.type", "sync")
props.put("broker.list", brokerList)
props.put("partitioner.class", "kafka.utils.FixedValuePartitioner")
props.put("buffer.size", bufferSize.toString)
props.put("send.buffer.bytes", bufferSize.toString)
props.put("connect.timeout.ms", connectTimeout.toString)
props.put("reconnect.interval", reconnectInterval.toString)
props.put("producer.request.timeout.ms", 30000.toString)
props.put("request.timeout.ms", 30000.toString)
props.put("serializer.class", classOf[StringEncoder].getName.toString)
props
}

View File

@ -43,10 +43,10 @@ public class Consumer extends Thread
{
Properties props = new Properties();
props.put("zk.connect", KafkaProperties.zkConnect);
props.put("groupid", KafkaProperties.groupId);
props.put("zk.sessiontimeout.ms", "400");
props.put("zk.synctime.ms", "200");
props.put("autocommit.interval.ms", "1000");
props.put("group.id", KafkaProperties.groupId);
props.put("zk.session.timeout.ms", "400");
props.put("zk.sync.time.ms", "200");
props.put("auto.commit.interval.ms", "1000");
return new ConsumerConfig(props);

View File

@ -74,7 +74,7 @@ object ConsumerPerformance {
if(!config.showDetailedStats) {
val totalMBRead = (totalBytesRead.get*1.0)/(1024*1024)
println(("%s, %s, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs),
config.consumerConfig.fetchSize, totalMBRead, totalMBRead/elapsedSecs, totalMessagesRead.get,
config.consumerConfig.fetchMessageMaxBytes, totalMBRead, totalMBRead/elapsedSecs, totalMessagesRead.get,
totalMessagesRead.get/elapsedSecs))
}
System.exit(0)
@ -124,10 +124,10 @@ object ConsumerPerformance {
}
val props = new Properties
props.put("groupid", options.valueOf(groupIdOpt))
props.put("socket.buffer.size", options.valueOf(socketBufferSizeOpt).toString)
props.put("fetch.size", options.valueOf(fetchSizeOpt).toString)
props.put("autooffset.reset", if(options.has(resetBeginningOffsetOpt)) "largest" else "smallest")
props.put("group.id", options.valueOf(groupIdOpt))
props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString)
props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString)
props.put("auto.offset.reset", if(options.has(resetBeginningOffsetOpt)) "largest" else "smallest")
props.put("zk.connect", options.valueOf(zkConnectOpt))
props.put("consumer.timeout.ms", "5000")
val consumerConfig = new ConsumerConfig(props)
@ -190,7 +190,7 @@ object ConsumerPerformance {
val totalMBRead = (bytesRead*1.0)/(1024*1024)
val mbRead = ((bytesRead - lastBytesRead)*1.0)/(1024*1024)
println(("%s, %d, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(endMs), id,
config.consumerConfig.fetchSize, totalMBRead,
config.consumerConfig.fetchMessageMaxBytes, totalMBRead,
1000.0*(mbRead/elapsedMs), messagesRead, ((messagesRead - lastMessagesRead)/elapsedMs)*1000.0))
}

View File

@ -191,16 +191,17 @@ object ProducerPerformance extends Logging {
props.put("broker.list", config.brokerList)
props.put("compression.codec", config.compressionCodec.codec.toString)
props.put("reconnect.interval", Integer.MAX_VALUE.toString)
props.put("buffer.size", (64*1024).toString)
props.put("send.buffer.bytes", (64*1024).toString)
if(!config.isSync) {
props.put("producer.type","async")
props.put("batch.size", config.batchSize.toString)
props.put("queue.enqueueTimeout.ms", "-1")
props.put("batch.num.messages", config.batchSize.toString)
props.put("queue.enqueue.timeout.ms", "-1")
}
props.put("producer.request.required.acks", config.producerRequestRequiredAcks.toString)
props.put("producer.request.timeout.ms", config.producerRequestTimeoutMs.toString)
props.put("producer.num.retries", config.producerNumRetries.toString)
props.put("producer.retry.backoff.ms", config.producerRetryBackoffMs.toString)
props.put("client.id", "ProducerPerformance")
props.put("request.required.acks", config.producerRequestRequiredAcks.toString)
props.put("request.timeout.ms", config.producerRequestTimeoutMs.toString)
props.put("message.send.max.retries", config.producerNumRetries.toString)
props.put("retry.backoff.ms", config.producerRetryBackoffMs.toString)
props.put("serializer.class", classOf[DefaultEncoder].getName.toString)
props.put("key.serializer.class", classOf[NullEncoder[Long]].getName.toString)

View File

@ -18,10 +18,10 @@
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
producer.type=async
# to avoid dropping events if the queue is full, wait indefinitely
queue.enqueueTimeout.ms=-1
queue.enqueue.timeout.ms=-1

View File

@ -19,10 +19,10 @@
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
producer.type=async
# to avoid dropping events if the queue is full, wait indefinitely
queue.enqueueTimeout.ms=-1
queue.enqueue.timeout.ms=-1

View File

@ -19,10 +19,10 @@
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
producer.type=async
# to avoid dropping events if the queue is full, wait indefinitely
queue.enqueueTimeout.ms=-1
queue.enqueue.timeout.ms=-1

View File

@ -19,10 +19,10 @@
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
producer.type=async
# to avoid dropping events if the queue is full, wait indefinitely
queue.enqueueTimeout.ms=-1
queue.enqueue.timeout.ms=-1

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=1
broker.id=1
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-source1-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=10000000
log.segment.bytes=10000000
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,14 +63,14 @@ enable.zookeeper=true
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=2
broker.id=2
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-source2-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=10000000
log.segment.bytes=10000000
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,14 +63,14 @@ enable.zookeeper=true
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=3
broker.id=3
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-source3-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=10000000
log.segment.size=10000000
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,14 +63,14 @@ enable.zookeeper=true
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=4
broker.id=4
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-source4-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=10000000
log.segment.bytes=10000000
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,14 +63,14 @@ enable.zookeeper=true
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=1
broker.id=1
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-target1-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=10000000
log.segment.bytes=10000000
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,16 +63,16 @@ enable.zookeeper=true
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
# topic partition count map
# topic.partition.count.map=topic1:3, topic2:4

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=2
broker.id=2
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-target2-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=10000000
log.segment.bytes=10000000
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,16 +63,16 @@ enable.zookeeper=true
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
# topic partition count map
# topic.partition.count.map=topic1:3, topic2:4

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=3
broker.id=3
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-target3-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=10000000
log.segment.bytes=10000000
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,16 +63,16 @@ enable.zookeeper=true
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
# topic partition count map
# topic.partition.count.map=topic1:3, topic2:4

View File

@ -20,10 +20,10 @@
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
#consumer group id
groupid=group1
group.id=group1
mirror.topics.whitelist=test_1,test_2
autooffset.reset=smallest
auto.offset.reset=smallest

View File

@ -72,7 +72,7 @@ kill_child_processes() {
# from the settings in config/server.properties while the brokerid and
# server port will be incremented accordingly
# 3. to generate properties files with non-default values such as
# "socket.send.buffer=2097152", simply add the property with new value
# "socket.send.buffer.bytes=2097152", simply add the property with new value
# to the array variable kafka_properties_to_replace as shown below
# =========================================================================
generate_kafka_properties_files() {
@ -103,10 +103,10 @@ generate_kafka_properties_files() {
# values. Other kafka properties can be added
# in a similar fashion.
# =============================================
# kafka_properties_to_replace[1]="socket.send.buffer=2097152"
# kafka_properties_to_replace[2]="socket.receive.buffer=2097152"
# kafka_properties_to_replace[1]="socket.send.buffer.bytes=2097152"
# kafka_properties_to_replace[2]="socket.receive.buffer.bytes=2097152"
# kafka_properties_to_replace[3]="num.partitions=3"
# kafka_properties_to_replace[4]="max.socket.request.bytes=10485760"
# kafka_properties_to_replace[4]="socket.request.max.bytes=10485760"
server_properties=`cat ${this_config_dir}/server.properties`

View File

@ -26,10 +26,10 @@ broker.list=localhost:9094,localhost:9095,localhost:9096
#zk.connect=
# zookeeper session timeout; default is 6000
#zk.sessiontimeout.ms=
#zk.session.timeout.ms=
# the max time that the client waits to establish a connection to zookeeper; default is 6000
#zk.connectiontimeout.ms
#zk.connection.timeout.ms
# name of the partitioner class for partitioning events; default partition spreads data randomly
#partitioner.class=
@ -46,35 +46,18 @@ serializer.class=kafka.serializer.DefaultEncoder
# allow topic level compression
#compressed.topics=
# max message size; messages larger than that size are discarded; default is 1000000
#max.message.size=
############################# Async Producer #############################
# maximum time, in milliseconds, for buffering data on the producer queue
#queue.time=
#queue.buffering.max.ms=
# the maximum size of the blocking queue for buffering on the producer
#queue.size=
#queue.buffering.max.messages=
# Timeout for event enqueue:
# 0: events will be enqueued immediately or dropped if the queue is full
# -ve: enqueue will block indefinitely if the queue is full
# +ve: enqueue will block up to this many milliseconds if the queue is full
#queue.enqueueTimeout.ms=
#queue.enqueue.timeout.ms=
# the number of messages batched at the producer
#batch.size=
# the callback handler for one or multiple events
#callback.handler=
# properties required to initialize the callback handler
#callback.handler.props=
# the handler for events
#event.handler=
# properties required to initialize the event handler
#event.handler.props=
#batch.num.messages=

View File

@ -17,12 +17,12 @@
############################# Server Basics #############################
# The id of the broker. This must be set to a unique integer for each broker.
brokerid=0
broker.id=0
# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned
# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost
# may not be what you want.
#hostname=
#host.name=
############################# Socket Server Settings #############################
@ -31,19 +31,19 @@ brokerid=0
port=9091
# The number of threads handling network requests
network.threads=2
num.network.threads=2
# The number of threads doing disk I/O
io.threads=2
num.io.threads=2
# The send buffer (SO_SNDBUF) used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# The receive buffer (SO_RCVBUF) used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# The maximum size of a request that the socket server will accept (protection against OOM)
max.socket.request.bytes=104857600
socket.request.max.bytes=104857600
############################# Log Basics #############################
@ -70,16 +70,16 @@ num.partitions=5
# every N messages (or both). This can be done globally and overridden on a per-topic basis.
# The number of messages to accept before forcing a flush of data to disk
log.flush.interval=10000
log.flush.interval.messages=10000
# The maximum amount of time a message can sit in a log before we force a flush
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# Per-topic overrides for log.default.flush.interval.ms
#topic.flush.intervals.ms=topic1:1000, topic2:3000
# Per-topic overrides for log.flush.interval.ms
#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
############################# Log Retention Policy #############################
@ -92,13 +92,13 @@ log.default.flush.scheduler.interval.ms=1000
log.retention.hours=168
# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
# segments don't drop below log.retention.size.
#log.retention.size=1073741824
# segments don't drop below log.retention.bytes.
#log.retention.bytes=1073741824
# The maximum size of a log segment file. When this size is reached a new log segment will be created.
#log.file.size=536870912
#log.file.size=102400
log.file.size=128
#log.segment.bytes=536870912
#log.segment.bytes=102400
log.segment.bytes=128
# The interval at which log segments are checked to see if they can be deleted according
# to the retention policies
@ -117,6 +117,6 @@ enable.zookeeper=true
zk.connect=localhost:2181
# Timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
monitoring.period.secs=1

View File

@ -20,9 +20,9 @@
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
#consumer group id
groupid=group1
shallowiterator.enable=true
group.id=group1
shallow.iterator.enable=true

View File

@ -19,12 +19,12 @@ zk.connect=localhost:2183
# broker.list=1:localhost:9094,2:localhost:9095
# timeout in ms for connecting to zookeeper
# zk.connectiontimeout.ms=1000000
# zk.connection.timeout.ms=1000000
producer.type=async
# to avoid dropping events if the queue is full, wait indefinitely
queue.enqueueTimeout.ms=-1
queue.enqueue.timeout.ms=-1
num.producers.per.broker=2

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=1
broker.id=1
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-source-1-1-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=10000000
log.segment.bytes=10000000
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,14 +63,14 @@ enable.zookeeper=true
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=2
broker.id=2
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-source-1-2-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=536870912
log.segment.bytes=536870912
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,14 +63,14 @@ enable.zookeeper=true
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=1
broker.id=1
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-source-2-1-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=536870912
log.segment.bytes=536870912
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,14 +63,14 @@ enable.zookeeper=true
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=2
broker.id=2
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-source-2-2-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=536870912
log.segment.bytes=536870912
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,14 +63,14 @@ enable.zookeeper=true
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=1
broker.id=1
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-target-1-1-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=536870912
log.segment.bytes=536870912
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,16 +63,16 @@ enable.zookeeper=true
zk.connect=localhost:2183
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
# topic partition count map
# topic.partition.count.map=topic1:3, topic2:4

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=2
broker.id=2
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-target-1-2-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=536870912
log.segment.bytes=536870912
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,16 +63,16 @@ enable.zookeeper=true
zk.connect=localhost:2183
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
# topic partition count map
# topic.partition.count.map=topic1:3, topic2:4

View File

@ -20,9 +20,9 @@
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
#consumer group id
groupid=group1
shallowiterator.enable=true
group.id=group1
shallow.iterator.enable=true

View File

@ -20,9 +20,9 @@
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
#consumer group id
groupid=group1
shallowiterator.enable=true
group.id=group1
shallow.iterator.enable=true

View File

@ -1,12 +1,12 @@
zk.connect=localhost:2108
zk.connectiontimeout.ms=1000000
groupid=mm_regtest_grp
autocommit.interval.ms=120000
autooffset.reset=smallest
#fetch.size=1048576
#rebalance.retries.max=4
zk.connection.timeout.ms=1000000
group.id=mm_regtest_grp
auto.commit.interval.ms=120000
auto.offset.reset=smallest
#fetch.message.max.bytes=1048576
#rebalance.max.retries=4
#rebalance.backoff.ms=2000
socket.buffersize=1048576
fetch.size=1048576
zk.synctime.ms=15000
shallowiterator.enable=true
socket.receive.buffer.bytes=1048576
fetch.message.max.bytes=1048576
zk.sync.time.ms=15000
shallow.iterator.enable=true

View File

@ -1,5 +1,5 @@
producer.type=async
queue.enqueueTimeout.ms=-1
queue.enqueue.timeout.ms=-1
broker.list=localhost:9094
compression.codec=0

View File

@ -17,12 +17,12 @@
############################# Server Basics #############################
# The id of the broker. This must be set to a unique integer for each broker.
brokerid=0
broker.id=0
# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned
# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost
# may not be what you want.
#hostname=
#host.name=
############################# Socket Server Settings #############################
@ -31,19 +31,19 @@ brokerid=0
port=9091
# The number of threads handling network requests
network.threads=2
num.network.threads=2
# The number of threads doing disk I/O
io.threads=2
num.io.threads=2
# The send buffer (SO_SNDBUF) used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# The receive buffer (SO_RCVBUF) used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# The maximum size of a request that the socket server will accept (protection against OOM)
max.socket.request.bytes=104857600
socket.request.max.bytes=104857600
############################# Log Basics #############################
@ -70,16 +70,16 @@ num.partitions=5
# every N messages (or both). This can be done globally and overridden on a per-topic basis.
# The number of messages to accept before forcing a flush of data to disk
log.flush.interval=10000
log.flush.interval.messages=10000
# The maximum amount of time a message can sit in a log before we force a flush
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# Per-topic overrides for log.default.flush.interval.ms
#topic.flush.intervals.ms=topic1:1000, topic2:3000
# Per-topic overrides for log.flush.interval.ms
#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
############################# Log Retention Policy #############################
@ -92,13 +92,13 @@ log.default.flush.scheduler.interval.ms=1000
log.retention.hours=168
# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
# segments don't drop below log.retention.size.
#log.retention.size=1073741824
log.retention.size=-1
# segments don't drop below log.retention.bytes.
#log.retention.bytes=1073741824
log.retention.bytes=-1
# The maximum size of a log segment file. When this size is reached a new log segment will be created.
#log.file.size=536870912
log.file.size=102400
#log.segment.size=536870912
log.segment.bytes=102400
# The interval at which log segments are checked to see if they can be deleted according
# to the retention policies
@ -117,23 +117,23 @@ enable.zookeeper=true
zk.connect=localhost:2181
# Timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
monitoring.period.secs=1
max.message.size=1000000
max.queued.requests=500
message.max.bytes=1000000
queued.max.requests=500
log.roll.hours=168
log.index.max.size=10485760
log.index.size.max.bytes=10485760
log.index.interval.bytes=4096
auto.create.topics=true
auto.create.topics.enable=true
controller.socket.timeout.ms=30000
controller.message.queue.size=10
default.replication.factor=1
replica.max.lag.time.ms=10000
replica.max.lag.bytes=4000
replica.lag.time.max.ms=10000
replica.lag.max.messages=4000
replica.socket.timeout.ms=30000
replica.socket.buffersize=65536
replica.fetch.size=1048576
replica.fetch.wait.time.ms=500
replica.socket.receive.buffer.bytes=65536
replica.fetch.max.bytes=1048576
replica.fetch.wait.max.ms=500
replica.fetch.min.bytes=4096
replica.fetchers=1
num.replica.fetchers=1

View File

@ -15,12 +15,12 @@
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=0
broker.id=0
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# host.name=
# number of logical partitions on this broker
num.partitions=1
@ -35,13 +35,13 @@ num.threads=8
log.dir=/tmp/kafka-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# the maximum size of a log segment
log.file.size=536870912
log.segment.bytes=536870912
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
@ -50,7 +50,7 @@ log.cleanup.interval.mins=1
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
log.flush.interval.messages=600
#set the following properties to use zookeeper
@ -63,16 +63,16 @@ enable.zookeeper=true
zk.connect=localhost:2181
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
#log.flush.intervals.ms.per.topic=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
# topic partition count map
# topic.partition.count.map=topic1:3, topic2:4

View File

@ -17,12 +17,12 @@
############################# Server Basics #############################
# The id of the broker. This must be set to a unique integer for each broker.
brokerid=0
broker.id=0
# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned
# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost
# may not be what you want.
#hostname=
#host.name=
############################# Socket Server Settings #############################
@ -31,19 +31,19 @@ brokerid=0
port=9091
# The number of threads handling network requests
network.threads=2
num.network.threads=2
# The number of threads doing disk I/O
io.threads=2
num.io.threads=2
# The send buffer (SO_SNDBUF) used by the socket server
socket.send.buffer=1048576
socket.send.buffer.bytes=1048576
# The receive buffer (SO_RCVBUF) used by the socket server
socket.receive.buffer=1048576
socket.receive.buffer.bytes=1048576
# The maximum size of a request that the socket server will accept (protection against OOM)
max.socket.request.bytes=104857600
socket.request.max.bytes=104857600
############################# Log Basics #############################
@ -70,16 +70,16 @@ num.partitions=5
# every N messages (or both). This can be done globally and overridden on a per-topic basis.
# The number of messages to accept before forcing a flush of data to disk
log.flush.interval=10000
log.flush.interval.messages=10000
# The maximum amount of time a message can sit in a log before we force a flush
log.default.flush.interval.ms=1000
log.flush.interval.ms=1000
# Per-topic overrides for log.default.flush.interval.ms
#topic.flush.intervals.ms=topic1:1000, topic2:3000
# Per-topic overrides for log.flush.interval.ms
#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000
# The interval (in ms) at which logs are checked to see if they need to be flushed to disk.
log.default.flush.scheduler.interval.ms=1000
log.flush.scheduler.interval.ms=1000
############################# Log Retention Policy #############################
@ -92,13 +92,13 @@ log.default.flush.scheduler.interval.ms=1000
log.retention.hours=168
# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
# segments don't drop below log.retention.size.
#log.retention.size=1073741824
log.retention.size=-1
# segments don't drop below log.retention.bytes.
#log.retention.bytes=1073741824
log.retention.bytes=-1
# The maximum size of a log segment file. When this size is reached a new log segment will be created.
#log.file.size=536870912
log.file.size=102400
#log.segment.size=536870912
log.segment.bytes=102400
# The interval at which log segments are checked to see if they can be deleted according
# to the retention policies
@ -117,23 +117,23 @@ enable.zookeeper=true
zk.connect=localhost:2181
# Timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
zk.connection.timeout.ms=1000000
monitoring.period.secs=1
max.message.size=1000000
max.queued.requests=500
message.max.bytes=1000000
queued.max.requests=500
log.roll.hours=168
log.index.max.size=10485760
log.index.size.max.bytes=10485760
log.index.interval.bytes=4096
auto.create.topics=true
auto.create.topics.enable=true
controller.socket.timeout.ms=30000
controller.message.queue.size=10
default.replication.factor=1
replica.max.lag.time.ms=10000
replica.max.lag.bytes=4000
replica.lag.time.max.ms=10000
replica.lag.max.messages=4000
replica.socket.timeout.ms=30000
replica.socket.buffersize=65536
replica.fetch.size=1048576
replica.fetch.wait.time.ms=500
replica.socket.receive.buffer.bytes=65536
replica.fetch.max.bytes=1048576
replica.fetch.wait.max.ms=500
replica.fetch.min.bytes=4096
replica.fetchers=1
num.replica.fetchers=1