mirror of https://github.com/apache/kafka.git
merge from 0.8 and resolve conflicts
This commit is contained in:
commit
9249b76d1d
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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=
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
|
||||
|
|
|
|||
|
|
@ -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")) {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -23,7 +23,6 @@ import scala.collection.Map
|
|||
import kafka.common.TopicAndPartition
|
||||
import kafka.api.ApiUtils._
|
||||
|
||||
|
||||
object ProducerRequest {
|
||||
val CurrentVersion = 0.shortValue
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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 */
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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(",")))
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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 +
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 =>{
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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)))
|
||||
|
|
|
|||
|
|
@ -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 ***********/
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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, "")
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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`
|
||||
|
||||
|
|
|
|||
|
|
@ -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=
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
producer.type=async
|
||||
queue.enqueueTimeout.ms=-1
|
||||
queue.enqueue.timeout.ms=-1
|
||||
broker.list=localhost:9094
|
||||
compression.codec=0
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
Loading…
Reference in New Issue