mirror of https://github.com/apache/kafka.git
KAFKA-631 Implement a log cleaner for Kafka. Reviewed by Neha.
This commit is contained in:
parent
92f177b309
commit
e7edb5e1e9
|
@ -36,6 +36,12 @@ log4j.appender.requestAppender.File=kafka-request.log
|
||||||
log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout
|
log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout
|
||||||
log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||||
|
|
||||||
|
log4j.appender.cleanerAppender=org.apache.log4j.DailyRollingFileAppender
|
||||||
|
log4j.appender.cleanerAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||||
|
log4j.appender.cleanerAppender.File=log-cleaner.log
|
||||||
|
log4j.appender.cleanerAppender.layout=org.apache.log4j.PatternLayout
|
||||||
|
log4j.appender.cleanerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||||
|
|
||||||
# Turn on all our debugging info
|
# Turn on all our debugging info
|
||||||
#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender
|
#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender
|
||||||
#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender
|
#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender
|
||||||
|
@ -44,13 +50,18 @@ log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||||
#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG
|
#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG
|
||||||
log4j.logger.kafka=INFO
|
log4j.logger.kafka=INFO
|
||||||
|
|
||||||
log4j.logger.kafka.network.RequestChannel$=TRACE, requestAppender
|
log4j.logger.kafka.network.RequestChannel$=WARN, requestAppender
|
||||||
log4j.additivity.kafka.network.RequestChannel$=false
|
log4j.additivity.kafka.network.RequestChannel$=false
|
||||||
|
|
||||||
#log4j.logger.kafka.server.KafkaApis=TRACE, requestAppender
|
#log4j.logger.kafka.server.KafkaApis=TRACE, requestAppender
|
||||||
#log4j.additivity.kafka.server.KafkaApis=false
|
#log4j.additivity.kafka.server.KafkaApis=false
|
||||||
log4j.logger.kafka.request.logger=TRACE, requestAppender
|
log4j.logger.kafka.request.logger=WARN, requestAppender
|
||||||
log4j.additivity.kafka.request.logger=false
|
log4j.additivity.kafka.request.logger=false
|
||||||
|
|
||||||
log4j.logger.kafka.controller=TRACE, stateChangeAppender
|
log4j.logger.kafka.controller=TRACE, stateChangeAppender
|
||||||
log4j.additivity.kafka.controller=false
|
log4j.additivity.kafka.controller=false
|
||||||
|
|
||||||
|
log4j.logger.kafka.log.LogCleaner=INFO, cleanerAppender
|
||||||
|
log4j.additivity.kafka.log.LogCleaner=false
|
||||||
|
log4j.logger.kafka.log.Cleaner=INFO, cleanerAppender
|
||||||
|
log4j.additivity.kafka.log.Cleaner=false
|
||||||
|
|
|
@ -114,3 +114,4 @@ kafka.csv.metrics.dir=/tmp/kafka_metrics
|
||||||
# Disable csv reporting by default.
|
# Disable csv reporting by default.
|
||||||
kafka.csv.metrics.reporter.enabled=false
|
kafka.csv.metrics.reporter.enabled=false
|
||||||
|
|
||||||
|
log.cleanup.policy=delete
|
||||||
|
|
|
@ -23,7 +23,7 @@ import kafka.api.LeaderAndIsr
|
||||||
import kafka.server.ReplicaManager
|
import kafka.server.ReplicaManager
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.common.ErrorMapping
|
import kafka.common._
|
||||||
import kafka.controller.{LeaderIsrAndControllerEpoch, KafkaController}
|
import kafka.controller.{LeaderIsrAndControllerEpoch, KafkaController}
|
||||||
|
|
||||||
|
|
||||||
|
@ -75,11 +75,11 @@ class Partition(val topic: String,
|
||||||
case None =>
|
case None =>
|
||||||
if (isReplicaLocal(replicaId)) {
|
if (isReplicaLocal(replicaId)) {
|
||||||
val log = logManager.getOrCreateLog(topic, partitionId)
|
val log = logManager.getOrCreateLog(topic, partitionId)
|
||||||
val offset = replicaManager.highWatermarkCheckpoints(log.dir.getParent).read(topic, partitionId).min(log.logEndOffset)
|
val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParent)
|
||||||
|
val offset = checkpoint.read.getOrElse(TopicAndPartition(topic, partitionId), 0L).min(log.logEndOffset)
|
||||||
val localReplica = new Replica(replicaId, this, time, offset, Some(log))
|
val localReplica = new Replica(replicaId, this, time, offset, Some(log))
|
||||||
addReplicaIfNotExists(localReplica)
|
addReplicaIfNotExists(localReplica)
|
||||||
}
|
} else {
|
||||||
else {
|
|
||||||
val remoteReplica = new Replica(replicaId, this, time)
|
val remoteReplica = new Replica(replicaId, this, time)
|
||||||
addReplicaIfNotExists(remoteReplica)
|
addReplicaIfNotExists(remoteReplica)
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,23 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka.common
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Thrown when an optimistic locking attempt receives concurrent modifications
|
||||||
|
*/
|
||||||
|
class OptimisticLockFailureException(message: String) extends RuntimeException(message)
|
|
@ -0,0 +1,41 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka.log
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configuration parameters for the log cleaner
|
||||||
|
*
|
||||||
|
* @param numThreads The number of cleaner threads to run
|
||||||
|
* @param dedupeBufferSize The total memory used for log deduplication
|
||||||
|
* @param dedupeBufferLoadFactor The maximum percent full for the deduplication buffer
|
||||||
|
* @param maxMessageSize The maximum size of a message that can appear in the log
|
||||||
|
* @param maxIoBytesPerSecond The maximum read and write I/O that all cleaner threads are allowed to do
|
||||||
|
* @param backOffMs The amount of time to wait before rechecking if no logs are eligible for cleaning
|
||||||
|
* @param enableCleaner Allows completely disabling the log cleaner
|
||||||
|
* @param hashAlgorithm The hash algorithm to use in key comparison.
|
||||||
|
*/
|
||||||
|
case class CleanerConfig(val numThreads: Int = 1,
|
||||||
|
val dedupeBufferSize: Int = 4*1024*1024,
|
||||||
|
val dedupeBufferLoadFactor: Double = 0.75,
|
||||||
|
val ioBufferSize: Int = 1024*1024,
|
||||||
|
val maxMessageSize: Int = 32*1024*1024,
|
||||||
|
val maxIoBytesPerSecond: Double = Double.MaxValue,
|
||||||
|
val backOffMs: Long = 60 * 1000,
|
||||||
|
val enableCleaner: Boolean = true,
|
||||||
|
val hashAlgorithm: String = "MD5") {
|
||||||
|
}
|
|
@ -38,7 +38,7 @@ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
|
||||||
* @param isSlice Should the start and end parameters be used for slicing?
|
* @param isSlice Should the start and end parameters be used for slicing?
|
||||||
*/
|
*/
|
||||||
@nonthreadsafe
|
@nonthreadsafe
|
||||||
class FileMessageSet private[kafka](val file: File,
|
class FileMessageSet private[kafka](@volatile var file: File,
|
||||||
private[log] val channel: FileChannel,
|
private[log] val channel: FileChannel,
|
||||||
private[log] val start: Int,
|
private[log] val start: Int,
|
||||||
private[log] val end: Int,
|
private[log] val end: Int,
|
||||||
|
@ -223,14 +223,36 @@ class FileMessageSet private[kafka](val file: File,
|
||||||
* Truncate this file message set to the given size in bytes. Note that this API does no checking that the
|
* Truncate this file message set to the given size in bytes. Note that this API does no checking that the
|
||||||
* given size falls on a valid message boundary.
|
* given size falls on a valid message boundary.
|
||||||
* @param targetSize The size to truncate to.
|
* @param targetSize The size to truncate to.
|
||||||
|
* @return The number of bytes truncated off
|
||||||
*/
|
*/
|
||||||
def truncateTo(targetSize: Int) = {
|
def truncateTo(targetSize: Int): Int = {
|
||||||
if(targetSize > sizeInBytes || targetSize < 0)
|
val originalSize = sizeInBytes
|
||||||
|
if(targetSize > originalSize || targetSize < 0)
|
||||||
throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
|
throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
|
||||||
" size of this log segment is " + sizeInBytes + " bytes.")
|
" size of this log segment is " + originalSize + " bytes.")
|
||||||
channel.truncate(targetSize)
|
channel.truncate(targetSize)
|
||||||
channel.position(targetSize)
|
channel.position(targetSize)
|
||||||
_size.set(targetSize)
|
_size.set(targetSize)
|
||||||
|
originalSize - targetSize
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read from the underlying file into the buffer starting at the given position
|
||||||
|
*/
|
||||||
|
def readInto(buffer: ByteBuffer, position: Int): ByteBuffer = {
|
||||||
|
channel.read(buffer, position)
|
||||||
|
buffer.flip()
|
||||||
|
buffer
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Rename the file that backs this message set
|
||||||
|
* @return true iff the rename was successful
|
||||||
|
*/
|
||||||
|
def renameTo(f: File): Boolean = {
|
||||||
|
val success = this.file.renameTo(f)
|
||||||
|
this.file = f
|
||||||
|
success
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -49,15 +49,9 @@ import com.yammer.metrics.core.Gauge
|
||||||
*/
|
*/
|
||||||
@threadsafe
|
@threadsafe
|
||||||
class Log(val dir: File,
|
class Log(val dir: File,
|
||||||
val scheduler: Scheduler,
|
val config: LogConfig,
|
||||||
val maxSegmentSize: Int,
|
|
||||||
val maxMessageSize: Int,
|
|
||||||
val flushInterval: Int = Int.MaxValue,
|
|
||||||
val rollIntervalMs: Long = Long.MaxValue,
|
|
||||||
val needsRecovery: Boolean,
|
val needsRecovery: Boolean,
|
||||||
val maxIndexSize: Int = (10*1024*1024),
|
val scheduler: Scheduler,
|
||||||
val indexIntervalBytes: Int = 4096,
|
|
||||||
val segmentDeleteDelayMs: Long = 60000,
|
|
||||||
time: Time = SystemTime) extends Logging with KafkaMetricsGroup {
|
time: Time = SystemTime) extends Logging with KafkaMetricsGroup {
|
||||||
|
|
||||||
import kafka.log.Log._
|
import kafka.log.Log._
|
||||||
|
@ -74,6 +68,9 @@ class Log(val dir: File,
|
||||||
/* the actual segments of the log */
|
/* the actual segments of the log */
|
||||||
private val segments: ConcurrentNavigableMap[Long,LogSegment] = loadSegments()
|
private val segments: ConcurrentNavigableMap[Long,LogSegment] = loadSegments()
|
||||||
|
|
||||||
|
/* The number of times the log has been truncated */
|
||||||
|
private val truncates = new AtomicInteger(0)
|
||||||
|
|
||||||
/* Calculate the offset of the next message */
|
/* Calculate the offset of the next message */
|
||||||
private val nextOffset: AtomicLong = new AtomicLong(activeSegment.nextOffset())
|
private val nextOffset: AtomicLong = new AtomicLong(activeSegment.nextOffset())
|
||||||
|
|
||||||
|
@ -90,58 +87,82 @@ class Log(val dir: File,
|
||||||
private def loadSegments(): ConcurrentNavigableMap[Long, LogSegment] = {
|
private def loadSegments(): ConcurrentNavigableMap[Long, LogSegment] = {
|
||||||
// open all the segments read-only
|
// open all the segments read-only
|
||||||
val logSegments = new ConcurrentSkipListMap[Long, LogSegment]
|
val logSegments = new ConcurrentSkipListMap[Long, LogSegment]
|
||||||
val ls = dir.listFiles()
|
|
||||||
if(ls != null) {
|
// create the log directory if it doesn't exist
|
||||||
for(file <- ls if file.isFile) {
|
dir.mkdirs()
|
||||||
if(!file.canRead)
|
|
||||||
throw new IOException("Could not read file " + file)
|
// first do a pass through the files in the log directory and remove any temporary files
|
||||||
val filename = file.getName
|
// and complete any interrupted swap operations
|
||||||
if(filename.endsWith(DeletedFileSuffix)) {
|
for(file <- dir.listFiles if file.isFile) {
|
||||||
// if the file ends in .deleted, delete it
|
if(!file.canRead)
|
||||||
val deleted = file.delete()
|
throw new IOException("Could not read file " + file)
|
||||||
if(!deleted)
|
val filename = file.getName
|
||||||
warn("Attempt to delete defunct segment file %s failed.".format(filename))
|
if(filename.endsWith(DeletedFileSuffix) || filename.endsWith(CleanedFileSuffix)) {
|
||||||
} else if(filename.endsWith(IndexFileSuffix)) {
|
// if the file ends in .deleted or .cleaned, delete it
|
||||||
// if it is an index file, make sure it has a corresponding .log file
|
file.delete()
|
||||||
val logFile = new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix))
|
} else if(filename.endsWith(SwapFileSuffix)) {
|
||||||
if(!logFile.exists) {
|
// we crashed in the middle of a swap operation, to recover:
|
||||||
warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath))
|
// if a log, swap it in and delete the .index file
|
||||||
file.delete()
|
// if an index just delete it, it will be rebuilt
|
||||||
}
|
val baseName = new File(Utils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
|
||||||
} else if(filename.endsWith(LogFileSuffix)) {
|
if(baseName.getPath.endsWith(IndexFileSuffix)) {
|
||||||
// if its a log file, load the corresponding log segment
|
file.delete()
|
||||||
val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong
|
} else if(baseName.getPath.endsWith(LogFileSuffix)){
|
||||||
val hasIndex = Log.indexFilename(dir, start).exists
|
// delete the index
|
||||||
val segment = new LogSegment(dir = dir,
|
val index = new File(Utils.replaceSuffix(baseName.getPath, LogFileSuffix, IndexFileSuffix))
|
||||||
startOffset = start,
|
index.delete()
|
||||||
indexIntervalBytes = indexIntervalBytes,
|
// complete the swap operation
|
||||||
maxIndexSize = maxIndexSize)
|
val renamed = file.renameTo(baseName)
|
||||||
if(!hasIndex) {
|
if(renamed)
|
||||||
// this can only happen if someone manually deletes the index file
|
info("Found log file %s from interrupted swap operation, repairing.".format(file.getPath))
|
||||||
error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath))
|
else
|
||||||
segment.recover(maxMessageSize)
|
throw new KafkaException("Failed to rename file %s.".format(file.getPath))
|
||||||
}
|
|
||||||
logSegments.put(start, segment)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// now do a second pass and load all the .log and .index files
|
||||||
|
for(file <- dir.listFiles if file.isFile) {
|
||||||
|
val filename = file.getName
|
||||||
|
if(filename.endsWith(IndexFileSuffix)) {
|
||||||
|
// if it is an index file, make sure it has a corresponding .log file
|
||||||
|
val logFile = new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix))
|
||||||
|
if(!logFile.exists) {
|
||||||
|
warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath))
|
||||||
|
file.delete()
|
||||||
|
}
|
||||||
|
} else if(filename.endsWith(LogFileSuffix)) {
|
||||||
|
// if its a log file, load the corresponding log segment
|
||||||
|
val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong
|
||||||
|
val hasIndex = Log.indexFilename(dir, start).exists
|
||||||
|
val segment = new LogSegment(dir = dir,
|
||||||
|
startOffset = start,
|
||||||
|
indexIntervalBytes = config.indexInterval,
|
||||||
|
maxIndexSize = config.maxIndexSize)
|
||||||
|
if(!hasIndex) {
|
||||||
|
error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath))
|
||||||
|
segment.recover(config.maxMessageSize)
|
||||||
|
}
|
||||||
|
logSegments.put(start, segment)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
if(logSegments.size == 0) {
|
if(logSegments.size == 0) {
|
||||||
// no existing segments, create a new mutable segment beginning at offset 0
|
// no existing segments, create a new mutable segment beginning at offset 0
|
||||||
logSegments.put(0,
|
logSegments.put(0,
|
||||||
new LogSegment(dir = dir,
|
new LogSegment(dir = dir,
|
||||||
startOffset = 0,
|
startOffset = 0,
|
||||||
indexIntervalBytes = indexIntervalBytes,
|
indexIntervalBytes = config.indexInterval,
|
||||||
maxIndexSize = maxIndexSize))
|
maxIndexSize = config.maxIndexSize))
|
||||||
} else {
|
} else {
|
||||||
// reset the index size of the currently active log segment to allow more entries
|
// reset the index size of the currently active log segment to allow more entries
|
||||||
val active = logSegments.lastEntry.getValue
|
val active = logSegments.lastEntry.getValue
|
||||||
active.index.resize(maxIndexSize)
|
active.index.resize(config.maxIndexSize)
|
||||||
|
|
||||||
// run recovery on the active segment if necessary
|
// run recovery on the active segment if necessary
|
||||||
if(needsRecovery) {
|
if(needsRecovery) {
|
||||||
info("Recovering active segment of %s.".format(name))
|
info("Recovering active segment of %s.".format(name))
|
||||||
active.recover(maxMessageSize)
|
active.recover(config.maxMessageSize)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
logSegments
|
logSegments
|
||||||
|
@ -153,6 +174,11 @@ class Log(val dir: File,
|
||||||
*/
|
*/
|
||||||
def numberOfSegments: Int = segments.size
|
def numberOfSegments: Int = segments.size
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The number of truncates that have occurred since the log was opened.
|
||||||
|
*/
|
||||||
|
def numberOfTruncates: Int = truncates.get
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Close this log
|
* Close this log
|
||||||
*/
|
*/
|
||||||
|
@ -194,20 +220,22 @@ class Log(val dir: File,
|
||||||
val segment = maybeRoll()
|
val segment = maybeRoll()
|
||||||
|
|
||||||
if(assignOffsets) {
|
if(assignOffsets) {
|
||||||
// assign offsets to the messageset
|
// assign offsets to the messageset
|
||||||
appendInfo.firstOffset = nextOffset.get
|
appendInfo.firstOffset = nextOffset.get
|
||||||
val offsetCounter = new AtomicLong(nextOffset.get)
|
val offset = new AtomicLong(nextOffset.get)
|
||||||
validMessages = validMessages.assignOffsets(offsetCounter, appendInfo.codec)
|
validMessages = validMessages.assignOffsets(offset, appendInfo.codec)
|
||||||
appendInfo.lastOffset = offsetCounter.get - 1
|
appendInfo.lastOffset = offset.get - 1
|
||||||
} else {
|
} else {
|
||||||
// we are taking the offsets we are given
|
// we are taking the offsets we are given
|
||||||
if(!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffset.get)
|
if(!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffset.get)
|
||||||
throw new IllegalArgumentException("Out of order offsets found in " + messages)
|
throw new IllegalArgumentException("Out of order offsets found in " + messages)
|
||||||
}
|
}
|
||||||
|
|
||||||
// now append to the log
|
// now append to the log
|
||||||
trace("Appending message set to %s with offsets %d to %d.".format(name, appendInfo.firstOffset, appendInfo.lastOffset))
|
trace("Appending message set to %s with offsets %d to %d.".format(name, appendInfo.firstOffset, appendInfo.lastOffset))
|
||||||
segment.append(appendInfo.firstOffset, validMessages)
|
segment.append(appendInfo.firstOffset, validMessages)
|
||||||
|
|
||||||
|
// increment the log end offset
|
||||||
nextOffset.set(appendInfo.lastOffset + 1)
|
nextOffset.set(appendInfo.lastOffset + 1)
|
||||||
|
|
||||||
// maybe flush the log and index
|
// maybe flush the log and index
|
||||||
|
@ -263,8 +291,8 @@ class Log(val dir: File,
|
||||||
// check the validity of the message by checking CRC and message size
|
// check the validity of the message by checking CRC and message size
|
||||||
val m = messageAndOffset.message
|
val m = messageAndOffset.message
|
||||||
m.ensureValid()
|
m.ensureValid()
|
||||||
if(MessageSet.entrySize(m) > maxMessageSize)
|
if(MessageSet.entrySize(m) > config.maxMessageSize)
|
||||||
throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d.".format(MessageSet.entrySize(m), maxMessageSize))
|
throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d.".format(MessageSet.entrySize(m), config.maxMessageSize))
|
||||||
|
|
||||||
messageCount += 1;
|
messageCount += 1;
|
||||||
|
|
||||||
|
@ -372,18 +400,21 @@ class Log(val dir: File,
|
||||||
*/
|
*/
|
||||||
private def maybeRoll(): LogSegment = {
|
private def maybeRoll(): LogSegment = {
|
||||||
val segment = activeSegment
|
val segment = activeSegment
|
||||||
if (segment.size > maxSegmentSize) {
|
if (segment.size > config.segmentSize ||
|
||||||
info("Rolling %s due to full data log".format(name))
|
segment.size > 0 && time.milliseconds - segment.created > config.segmentMs ||
|
||||||
|
segment.index.isFull) {
|
||||||
|
debug("Rolling new log segment in %s (log_size = %d/%d, index_size = %d/%d, age_ms = %d/%d)."
|
||||||
|
.format(name,
|
||||||
|
segment.size,
|
||||||
|
config.segmentSize,
|
||||||
|
segment.index.entries,
|
||||||
|
segment.index.maxEntries,
|
||||||
|
time.milliseconds - segment.created,
|
||||||
|
config.segmentMs))
|
||||||
roll()
|
roll()
|
||||||
} else if (segment.size > 0 && time.milliseconds - segment.created > rollIntervalMs) {
|
} else {
|
||||||
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
|
segment
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -412,11 +443,11 @@ class Log(val dir: File,
|
||||||
}
|
}
|
||||||
val segment = new LogSegment(dir,
|
val segment = new LogSegment(dir,
|
||||||
startOffset = newOffset,
|
startOffset = newOffset,
|
||||||
indexIntervalBytes = indexIntervalBytes,
|
indexIntervalBytes = config.indexInterval,
|
||||||
maxIndexSize = maxIndexSize)
|
maxIndexSize = config.maxIndexSize)
|
||||||
val prev = segments.put(segment.baseOffset, segment)
|
val prev = addSegment(segment)
|
||||||
if(prev != null)
|
if(prev != null)
|
||||||
throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists.".format(dir.getName, newOffset))
|
throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists.".format(name, newOffset))
|
||||||
segment
|
segment
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -426,12 +457,12 @@ class Log(val dir: File,
|
||||||
* @param numberOfMessages The number of messages that are being appended
|
* @param numberOfMessages The number of messages that are being appended
|
||||||
*/
|
*/
|
||||||
private def maybeFlush(numberOfMessages : Int) {
|
private def maybeFlush(numberOfMessages : Int) {
|
||||||
if(unflushed.addAndGet(numberOfMessages) >= flushInterval)
|
if(unflushed.addAndGet(numberOfMessages) >= config.flushInterval)
|
||||||
flush()
|
flush()
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Flush this log file and assoicated index to the physical disk
|
* Flush this log file and associated index to the physical disk
|
||||||
*/
|
*/
|
||||||
def flush() : Unit = {
|
def flush() : Unit = {
|
||||||
if (unflushed.get == 0)
|
if (unflushed.get == 0)
|
||||||
|
@ -475,6 +506,7 @@ class Log(val dir: File,
|
||||||
activeSegment.truncateTo(targetOffset)
|
activeSegment.truncateTo(targetOffset)
|
||||||
this.nextOffset.set(targetOffset)
|
this.nextOffset.set(targetOffset)
|
||||||
}
|
}
|
||||||
|
truncates.getAndIncrement
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -487,12 +519,12 @@ class Log(val dir: File,
|
||||||
lock synchronized {
|
lock synchronized {
|
||||||
val segmentsToDelete = logSegments.toList
|
val segmentsToDelete = logSegments.toList
|
||||||
segmentsToDelete.foreach(deleteSegment(_))
|
segmentsToDelete.foreach(deleteSegment(_))
|
||||||
segments.put(newOffset,
|
addSegment(new LogSegment(dir,
|
||||||
new LogSegment(dir,
|
newOffset,
|
||||||
newOffset,
|
indexIntervalBytes = config.indexInterval,
|
||||||
indexIntervalBytes = indexIntervalBytes,
|
maxIndexSize = config.maxIndexSize))
|
||||||
maxIndexSize = maxIndexSize))
|
|
||||||
this.nextOffset.set(newOffset)
|
this.nextOffset.set(newOffset)
|
||||||
|
truncates.getAndIncrement
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -511,7 +543,13 @@ class Log(val dir: File,
|
||||||
*/
|
*/
|
||||||
def logSegments: Iterable[LogSegment] = asIterable(segments.values)
|
def logSegments: Iterable[LogSegment] = asIterable(segments.values)
|
||||||
|
|
||||||
override def toString() = "Log(" + this.dir + ")"
|
/**
|
||||||
|
* Get all segments beginning with the segment that includes "from" and ending with the segment
|
||||||
|
* that includes up to "to-1" or the end of the log (if to > logEndOffset)
|
||||||
|
*/
|
||||||
|
def logSegments(from: Long, to: Long) = asIterable(segments.subMap(from, true, to, false).values)
|
||||||
|
|
||||||
|
override def toString() = "Log(" + dir + ")"
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method performs an asynchronous log segment delete by doing the following:
|
* This method performs an asynchronous log segment delete by doing the following:
|
||||||
|
@ -526,38 +564,83 @@ class Log(val dir: File,
|
||||||
* @param segment The log segment to schedule for deletion
|
* @param segment The log segment to schedule for deletion
|
||||||
*/
|
*/
|
||||||
private def deleteSegment(segment: LogSegment) {
|
private def deleteSegment(segment: LogSegment) {
|
||||||
info("Scheduling log segment %d for log %s for deletion.".format(segment.baseOffset, dir.getName))
|
info("Scheduling log segment %d for log %s for deletion.".format(segment.baseOffset, name))
|
||||||
lock synchronized {
|
lock synchronized {
|
||||||
segments.remove(segment.baseOffset)
|
segments.remove(segment.baseOffset)
|
||||||
val deletedLog = new File(segment.log.file.getPath + Log.DeletedFileSuffix)
|
asyncDeleteSegment(segment)
|
||||||
val deletedIndex = new File(segment.index.file.getPath + Log.DeletedFileSuffix)
|
|
||||||
val renamedLog = segment.log.file.renameTo(deletedLog)
|
|
||||||
val renamedIndex = segment.index.file.renameTo(deletedIndex)
|
|
||||||
if(!renamedLog && segment.log.file.exists)
|
|
||||||
throw new KafkaStorageException("Failed to rename file %s to %s for log %s.".format(segment.log.file.getPath, deletedLog.getPath, name))
|
|
||||||
if(!renamedIndex && segment.index.file.exists)
|
|
||||||
throw new KafkaStorageException("Failed to rename file %s to %s for log %s.".format(segment.index.file.getPath, deletedIndex.getPath, name))
|
|
||||||
def asyncDeleteFiles() {
|
|
||||||
info("Deleting log segment %s for log %s.".format(segment.baseOffset, name))
|
|
||||||
if(!deletedLog.delete())
|
|
||||||
warn("Failed to delete log segment file %s for log %s.".format(deletedLog.getPath, name))
|
|
||||||
if(!deletedIndex.delete())
|
|
||||||
warn("Failed to delete index segment file %s for log %s.".format(deletedLog.getPath, name))
|
|
||||||
}
|
|
||||||
scheduler.schedule("delete-log-segment", asyncDeleteFiles, delay = segmentDeleteDelayMs)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Perform an asynchronous delete on the given file if it exists (otherwise do nothing)
|
||||||
|
* @throws KafkaStorageException if the file can't be renamed and still exists
|
||||||
|
*/
|
||||||
|
private def asyncDeleteSegment(segment: LogSegment) {
|
||||||
|
segment.changeFileSuffixes("", Log.DeletedFileSuffix)
|
||||||
|
def deleteSeg() {
|
||||||
|
info("Deleting segment %d from log %s.".format(segment.baseOffset, name))
|
||||||
|
segment.delete()
|
||||||
|
}
|
||||||
|
scheduler.schedule("delete-file", deleteSeg, delay = config.fileDeleteDelayMs)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Swap a new segment in place and delete one or more existing segments in a crash-safe manner. The old segments will
|
||||||
|
* be asynchronously deleted.
|
||||||
|
*
|
||||||
|
* @param newSegment The new log segment to add to the log
|
||||||
|
* @param oldSegments The old log segments to delete from the log
|
||||||
|
*/
|
||||||
|
private[log] def replaceSegments(newSegment: LogSegment, oldSegments: Seq[LogSegment], expectedTruncates: Int) {
|
||||||
|
lock synchronized {
|
||||||
|
if(expectedTruncates != numberOfTruncates)
|
||||||
|
throw new OptimisticLockFailureException("The log has been truncated, expected %d but found %d.".format(expectedTruncates, numberOfTruncates))
|
||||||
|
// need to do this in two phases to be crash safe AND do the delete asynchronously
|
||||||
|
// if we crash in the middle of this we complete the swap in loadSegments()
|
||||||
|
newSegment.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix)
|
||||||
|
addSegment(newSegment)
|
||||||
|
|
||||||
|
// delete the old files
|
||||||
|
for(seg <- oldSegments) {
|
||||||
|
// remove the index entry
|
||||||
|
if(seg.baseOffset != newSegment.baseOffset)
|
||||||
|
segments.remove(seg.baseOffset)
|
||||||
|
// delete segment
|
||||||
|
asyncDeleteSegment(seg)
|
||||||
|
}
|
||||||
|
// okay we are safe now, remove the swap suffix
|
||||||
|
newSegment.changeFileSuffixes(Log.SwapFileSuffix, "")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
|
||||||
|
* @param segment The segment to add
|
||||||
|
*/
|
||||||
|
def addSegment(segment: LogSegment) = this.segments.put(segment.baseOffset, segment)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helper functions for logs
|
* Helper functions for logs
|
||||||
*/
|
*/
|
||||||
object Log {
|
object Log {
|
||||||
|
|
||||||
|
/** a log file */
|
||||||
val LogFileSuffix = ".log"
|
val LogFileSuffix = ".log"
|
||||||
|
|
||||||
|
/** an index file */
|
||||||
val IndexFileSuffix = ".index"
|
val IndexFileSuffix = ".index"
|
||||||
|
|
||||||
|
/** a file that is scheduled to be deleted */
|
||||||
val DeletedFileSuffix = ".deleted"
|
val DeletedFileSuffix = ".deleted"
|
||||||
|
|
||||||
|
/** A temporary file that is being used for log cleaning */
|
||||||
|
val CleanedFileSuffix = ".cleaned"
|
||||||
|
|
||||||
|
/** A temporary file used when swapping files into the log */
|
||||||
|
val SwapFileSuffix = ".swap"
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
|
* Make log segment file name from offset bytes. All this does is pad out the offset number with zeros
|
||||||
* so that ls sorts the files numerically.
|
* so that ls sorts the files numerically.
|
||||||
|
|
|
@ -0,0 +1,557 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka.log
|
||||||
|
|
||||||
|
import scala.collection._
|
||||||
|
import scala.math
|
||||||
|
import java.nio._
|
||||||
|
import java.util.concurrent.Semaphore
|
||||||
|
import java.util.concurrent.TimeUnit
|
||||||
|
import java.util.concurrent.atomic._
|
||||||
|
import java.io.File
|
||||||
|
import kafka.common._
|
||||||
|
import kafka.message._
|
||||||
|
import kafka.server.OffsetCheckpoint
|
||||||
|
import kafka.utils._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy.
|
||||||
|
* A message with key K and offset O is obsolete if there exists a message with key K and offset O' such that O < O'.
|
||||||
|
*
|
||||||
|
* Each log can be thought of being split into two sections of segments: a "clean" section which has previously been cleaned followed by a
|
||||||
|
* "dirty" section that has not yet been cleaned. The active log segment is always excluded from cleaning.
|
||||||
|
*
|
||||||
|
* The cleaning is carried out by a pool of background threads. Each thread chooses the dirtiest log that has the "dedupe" retention policy
|
||||||
|
* and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log.
|
||||||
|
*
|
||||||
|
* To clean a log the cleaner first builds a mapping of key=>last_offset for the dirty section of the log. For memory efficiency this mapping
|
||||||
|
* is approximate. That is allowed to lose some key=>offset pairs, but never to return a wrong answer. See kafka.log.OffsetMap for details of
|
||||||
|
* the implementation of the mapping.
|
||||||
|
*
|
||||||
|
* Once the key=>offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a
|
||||||
|
* higher offset than what is found in the segment (i.e. messages with a key that appears in the dirty section of the log).
|
||||||
|
*
|
||||||
|
* To avoid segments shrinking to very small sizes with repeated cleanings we implement a rule by which if we will merge successive segments when
|
||||||
|
* doing a cleaning if their log and index size are less than the maximum log and index size prior to the clean beginning.
|
||||||
|
*
|
||||||
|
* Cleaned segments are swapped into the log as they become available.
|
||||||
|
*
|
||||||
|
* One nuance that the cleaner must handle is log truncation. If a log is truncated while it is being cleaned the cleaning of that log is aborted.
|
||||||
|
*
|
||||||
|
* @param config Configuration parameters for the cleaner
|
||||||
|
* @param logDirs The directories where offset checkpoints reside
|
||||||
|
* @param logs The pool of logs
|
||||||
|
* @param time A way to control the passage of time
|
||||||
|
*/
|
||||||
|
class LogCleaner(val config: CleanerConfig,
|
||||||
|
val logDirs: Array[File],
|
||||||
|
val logs: Pool[TopicAndPartition, Log],
|
||||||
|
time: Time = SystemTime) extends Logging {
|
||||||
|
|
||||||
|
/* the offset checkpoints holding the last cleaned point for each log */
|
||||||
|
private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, "cleaner-offset-checkpoint")))).toMap
|
||||||
|
|
||||||
|
/* the set of logs currently being cleaned */
|
||||||
|
private val inProgress = mutable.HashSet[TopicAndPartition]()
|
||||||
|
|
||||||
|
/* a global lock used to control all access to the in-progress set and the offset checkpoints */
|
||||||
|
private val lock = new Object
|
||||||
|
|
||||||
|
/* a counter for creating unique thread names*/
|
||||||
|
private val threadId = new AtomicInteger(0)
|
||||||
|
|
||||||
|
/* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */
|
||||||
|
private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond,
|
||||||
|
checkIntervalMs = 300,
|
||||||
|
throttleDown = true,
|
||||||
|
time = time)
|
||||||
|
|
||||||
|
/* the threads */
|
||||||
|
private val cleaners = (0 until config.numThreads).map(_ => new CleanerThread())
|
||||||
|
|
||||||
|
/* a hook for testing to synchronize on log cleaning completions */
|
||||||
|
private val cleaned = new Semaphore(0)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Start the background cleaning
|
||||||
|
*/
|
||||||
|
def startup() {
|
||||||
|
info("Starting the log cleaner")
|
||||||
|
cleaners.foreach(_.start())
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stop the background cleaning
|
||||||
|
*/
|
||||||
|
def shutdown() {
|
||||||
|
info("Shutting down the log cleaner.")
|
||||||
|
cleaners.foreach(_.interrupt())
|
||||||
|
cleaners.foreach(_.join())
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* For testing, a way to know when work has completed. This method blocks until the
|
||||||
|
* cleaner has processed up to the given offset on the specified topic/partition
|
||||||
|
*/
|
||||||
|
def awaitCleaned(topic: String, part: Int, offset: Long, timeout: Long = 30000L): Unit = {
|
||||||
|
while(!allCleanerCheckpoints.contains(TopicAndPartition(topic, part)))
|
||||||
|
cleaned.tryAcquire(timeout, TimeUnit.MILLISECONDS)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the position processed for all logs.
|
||||||
|
*/
|
||||||
|
def allCleanerCheckpoints(): Map[TopicAndPartition, Long] =
|
||||||
|
checkpoints.values.flatMap(_.read()).toMap
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Choose the log to clean next and add it to the in-progress set. We recompute this
|
||||||
|
* every time off the full set of logs to allow logs to be dynamically added to the pool of logs
|
||||||
|
* the log manager maintains.
|
||||||
|
*/
|
||||||
|
private def grabFilthiestLog(): Option[LogToClean] = {
|
||||||
|
lock synchronized {
|
||||||
|
val lastClean = allCleanerCheckpoints()
|
||||||
|
val cleanableLogs = logs.filter(l => l._2.config.dedupe) // skip any logs marked for delete rather than dedupe
|
||||||
|
.filterNot(l => inProgress.contains(l._1)) // skip any logs already in-progress
|
||||||
|
.map(l => LogToClean(l._1, l._2, lastClean.getOrElse(l._1, 0))) // create a LogToClean instance for each
|
||||||
|
val dirtyLogs = cleanableLogs.filter(l => l.totalBytes > 0) // must have some bytes
|
||||||
|
.filter(l => l.cleanableRatio > l.log.config.minCleanableRatio) // and must meet the minimum threshold for dirty byte ratio
|
||||||
|
if(dirtyLogs.isEmpty) {
|
||||||
|
None
|
||||||
|
} else {
|
||||||
|
val filthiest = dirtyLogs.max
|
||||||
|
inProgress += filthiest.topicPartition
|
||||||
|
Some(filthiest)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Save out the endOffset and remove the given log from the in-progress set.
|
||||||
|
*/
|
||||||
|
private def doneCleaning(topicAndPartition: TopicAndPartition, dataDir: File, endOffset: Long) {
|
||||||
|
lock synchronized {
|
||||||
|
val checkpoint = checkpoints(dataDir)
|
||||||
|
val offsets = checkpoint.read() + ((topicAndPartition, endOffset))
|
||||||
|
checkpoint.write(offsets)
|
||||||
|
inProgress -= topicAndPartition
|
||||||
|
}
|
||||||
|
cleaned.release()
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The cleaner threads do the actual log cleaning. Each thread processes does its cleaning repeatedly by
|
||||||
|
* choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments.
|
||||||
|
*/
|
||||||
|
private class CleanerThread extends Thread {
|
||||||
|
val cleaner = new Cleaner(id = threadId.getAndIncrement(),
|
||||||
|
offsetMap = new SkimpyOffsetMap(memory = config.dedupeBufferSize / config.numThreads,
|
||||||
|
maxLoadFactor = config.dedupeBufferLoadFactor,
|
||||||
|
hashAlgorithm = config.hashAlgorithm),
|
||||||
|
ioBufferSize = config.ioBufferSize / config.numThreads / 2,
|
||||||
|
maxIoBufferSize = config.maxMessageSize,
|
||||||
|
throttler = throttler,
|
||||||
|
time = time)
|
||||||
|
|
||||||
|
setName("kafka-log-cleaner-thread-" + cleaner.id)
|
||||||
|
setDaemon(false)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The main loop for the cleaner thread
|
||||||
|
*/
|
||||||
|
override def run() {
|
||||||
|
info("Starting cleaner thread %d...".format(cleaner.id))
|
||||||
|
try {
|
||||||
|
while(!isInterrupted) {
|
||||||
|
cleanOrSleep()
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
case e: InterruptedException => // all done
|
||||||
|
case e: Exception =>
|
||||||
|
error("Error in cleaner thread %d:".format(cleaner.id), e)
|
||||||
|
}
|
||||||
|
info("Shutting down cleaner thread %d.".format(cleaner.id))
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clean a log if there is a dirty log available, otherwise sleep for a bit
|
||||||
|
*/
|
||||||
|
private def cleanOrSleep() {
|
||||||
|
grabFilthiestLog() match {
|
||||||
|
case None =>
|
||||||
|
// there are no cleanable logs, sleep a while
|
||||||
|
time.sleep(config.backOffMs)
|
||||||
|
case Some(cleanable) =>
|
||||||
|
// there's a log, clean it
|
||||||
|
var endOffset = cleanable.firstDirtyOffset
|
||||||
|
try {
|
||||||
|
endOffset = cleaner.clean(cleanable)
|
||||||
|
logStats(cleaner.id, cleanable.log.name, cleanable.firstDirtyOffset, endOffset, cleaner.stats)
|
||||||
|
} catch {
|
||||||
|
case e: OptimisticLockFailureException =>
|
||||||
|
info("Cleaning of log was aborted due to colliding truncate operation.")
|
||||||
|
} finally {
|
||||||
|
doneCleaning(cleanable.topicPartition, cleanable.log.dir.getParentFile, endOffset)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Log out statistics on a single run of the cleaner.
|
||||||
|
*/
|
||||||
|
def logStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats) {
|
||||||
|
def mb(bytes: Double) = bytes / (1024*1024)
|
||||||
|
val message =
|
||||||
|
"%n\tLog cleaner %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) +
|
||||||
|
"\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead),
|
||||||
|
stats.elapsedSecs,
|
||||||
|
mb(stats.bytesRead/stats.elapsedSecs)) +
|
||||||
|
"\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead),
|
||||||
|
stats.elapsedIndexSecs,
|
||||||
|
mb(stats.mapBytesRead)/stats.elapsedIndexSecs,
|
||||||
|
100 * stats.elapsedIndexSecs.toDouble/stats.elapsedSecs) +
|
||||||
|
"\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead),
|
||||||
|
stats.elapsedSecs - stats.elapsedIndexSecs,
|
||||||
|
mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) +
|
||||||
|
"\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead), stats.messagesRead) +
|
||||||
|
"\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) +
|
||||||
|
"\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead),
|
||||||
|
100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead))
|
||||||
|
info(message)
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class holds the actual logic for cleaning a log
|
||||||
|
* @param id An identifier used for logging
|
||||||
|
* @param offsetMap The map used for deduplication
|
||||||
|
* @param bufferSize The size of the buffers to use. Memory usage will be 2x this number as there is a read and write buffer.
|
||||||
|
* @param throttler The throttler instance to use for limiting I/O rate.
|
||||||
|
* @param time The time instance
|
||||||
|
*/
|
||||||
|
private[log] class Cleaner(val id: Int,
|
||||||
|
offsetMap: OffsetMap,
|
||||||
|
ioBufferSize: Int,
|
||||||
|
maxIoBufferSize: Int,
|
||||||
|
throttler: Throttler,
|
||||||
|
time: Time) extends Logging {
|
||||||
|
|
||||||
|
this.logIdent = "Cleaner " + id + ":"
|
||||||
|
val stats = new CleanerStats(time)
|
||||||
|
private var readBuffer = ByteBuffer.allocate(ioBufferSize) // buffer for disk read I/O
|
||||||
|
private var writeBuffer = ByteBuffer.allocate(ioBufferSize) // buffer for disk write I/O
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clean the given log
|
||||||
|
*
|
||||||
|
* @param cleanable The log to be cleaned
|
||||||
|
*
|
||||||
|
* @return The first offset not cleaned
|
||||||
|
*/
|
||||||
|
private[log] def clean(cleanable: LogToClean): Long = {
|
||||||
|
stats.clear()
|
||||||
|
val topic = cleanable.topicPartition.topic
|
||||||
|
val part = cleanable.topicPartition.partition
|
||||||
|
info("Beginning cleaning of %s-%d.".format(topic, part))
|
||||||
|
val log = cleanable.log
|
||||||
|
val truncateCount = log.numberOfTruncates
|
||||||
|
|
||||||
|
// build the offset map
|
||||||
|
val upperBoundOffset = math.min(log.activeSegment.baseOffset, cleanable.firstDirtyOffset + offsetMap.capacity)
|
||||||
|
val endOffset = buildOffsetMap(log, cleanable.firstDirtyOffset, upperBoundOffset, offsetMap) + 1
|
||||||
|
stats.indexDone()
|
||||||
|
|
||||||
|
// group the segments and clean the groups
|
||||||
|
for (group <- groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, log.config.maxIndexSize)) {
|
||||||
|
info("Cleaning segments %s for log %s...".format(group.map(_.baseOffset).mkString(","), log.name))
|
||||||
|
cleanSegments(log, group, offsetMap, truncateCount)
|
||||||
|
}
|
||||||
|
stats.allDone()
|
||||||
|
endOffset
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clean a group of segments into a single replacement segment
|
||||||
|
*
|
||||||
|
* @param log The log being cleaned
|
||||||
|
* @param segments The group of segments being cleaned
|
||||||
|
* @param map The offset map to use for cleaning segments
|
||||||
|
* @param expectedTruncateCount A count used to check if the log is being truncated and rewritten under our feet
|
||||||
|
*/
|
||||||
|
private[log] def cleanSegments(log: Log, segments: Seq[LogSegment], map: OffsetMap, expectedTruncateCount: Int) {
|
||||||
|
// create a new segment with the suffix .cleaned appended to both the log and index name
|
||||||
|
val logFile = new File(segments.head.log.file.getPath + Log.CleanedFileSuffix)
|
||||||
|
logFile.delete()
|
||||||
|
val indexFile = new File(segments.head.index.file.getPath + Log.CleanedFileSuffix)
|
||||||
|
indexFile.delete()
|
||||||
|
val messages = new FileMessageSet(logFile)
|
||||||
|
val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize)
|
||||||
|
val cleaned = new LogSegment(messages, index, segments.head.baseOffset, segments.head.indexIntervalBytes, time)
|
||||||
|
|
||||||
|
// clean segments into the new destination segment
|
||||||
|
for (old <- segments)
|
||||||
|
cleanInto(old, cleaned, map)
|
||||||
|
|
||||||
|
// trim excess index
|
||||||
|
index.trimToValidSize()
|
||||||
|
|
||||||
|
// flush new segment to disk before swap
|
||||||
|
cleaned.flush()
|
||||||
|
|
||||||
|
// swap in new segment
|
||||||
|
info("Swapping in cleaned segment %d for %s in log %s.".format(cleaned.baseOffset, segments.map(_.baseOffset).mkString(","), log.name))
|
||||||
|
try {
|
||||||
|
log.replaceSegments(cleaned, segments, expectedTruncateCount)
|
||||||
|
} catch {
|
||||||
|
case e: OptimisticLockFailureException =>
|
||||||
|
cleaned.delete()
|
||||||
|
throw e
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Clean the given source log segment into the destination segment using the key=>offset mapping
|
||||||
|
* provided
|
||||||
|
*
|
||||||
|
* @param source The dirty log segment
|
||||||
|
* @param dest The cleaned log segment
|
||||||
|
* @param map The key=>offset mapping
|
||||||
|
*
|
||||||
|
* TODO: Implement proper compression support
|
||||||
|
*/
|
||||||
|
private[log] def cleanInto(source: LogSegment, dest: LogSegment, map: OffsetMap) {
|
||||||
|
var position = 0
|
||||||
|
while (position < source.log.sizeInBytes) {
|
||||||
|
checkDone()
|
||||||
|
// read a chunk of messages and copy any that are to be retained to the write buffer to be written out
|
||||||
|
readBuffer.clear()
|
||||||
|
writeBuffer.clear()
|
||||||
|
val messages = new ByteBufferMessageSet(source.log.readInto(readBuffer, position))
|
||||||
|
throttler.maybeThrottle(messages.sizeInBytes)
|
||||||
|
// check each message to see if it is to be retained
|
||||||
|
var messagesRead = 0
|
||||||
|
for (entry <- messages) {
|
||||||
|
messagesRead += 1
|
||||||
|
val size = MessageSet.entrySize(entry.message)
|
||||||
|
position += size
|
||||||
|
stats.readMessage(size)
|
||||||
|
val key = entry.message.key
|
||||||
|
require(key != null, "Found null key in log segment %s which is marked as dedupe.".format(source.log.file.getAbsolutePath))
|
||||||
|
val lastOffset = map.get(key)
|
||||||
|
/* retain the record if it isn't present in the map OR it is present but this offset is the highest (and it's not a delete) */
|
||||||
|
val retainRecord = lastOffset < 0 || (entry.offset >= lastOffset && entry.message.payload != null)
|
||||||
|
if (retainRecord) {
|
||||||
|
ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset)
|
||||||
|
stats.recopyMessage(size)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// if any messages are to be retained, write them out
|
||||||
|
if(writeBuffer.position > 0) {
|
||||||
|
writeBuffer.flip()
|
||||||
|
val retained = new ByteBufferMessageSet(writeBuffer)
|
||||||
|
dest.append(retained.head.offset, retained)
|
||||||
|
throttler.maybeThrottle(writeBuffer.limit)
|
||||||
|
}
|
||||||
|
|
||||||
|
// if we read bytes but didn't get even one complete message, our I/O buffer is too small, grow it and try again
|
||||||
|
if(readBuffer.limit > 0 && messagesRead == 0)
|
||||||
|
growBuffers()
|
||||||
|
}
|
||||||
|
restoreBuffers()
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Double the I/O buffer capacity
|
||||||
|
*/
|
||||||
|
def growBuffers() {
|
||||||
|
if(readBuffer.capacity >= maxIoBufferSize || writeBuffer.capacity >= maxIoBufferSize)
|
||||||
|
throw new IllegalStateException("This log contains a message larger than maximum allowable size of %s.".format(maxIoBufferSize))
|
||||||
|
val newSize = math.min(this.readBuffer.capacity * 2, maxIoBufferSize)
|
||||||
|
info("Growing cleaner I/O buffers from " + readBuffer.capacity + "bytes to " + newSize + " bytes.")
|
||||||
|
this.readBuffer = ByteBuffer.allocate(newSize)
|
||||||
|
this.writeBuffer = ByteBuffer.allocate(newSize)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Restore the I/O buffer capacity to its original size
|
||||||
|
*/
|
||||||
|
def restoreBuffers() {
|
||||||
|
if(this.readBuffer.capacity > this.ioBufferSize)
|
||||||
|
this.readBuffer = ByteBuffer.allocate(this.ioBufferSize)
|
||||||
|
if(this.writeBuffer.capacity > this.ioBufferSize)
|
||||||
|
this.writeBuffer = ByteBuffer.allocate(this.ioBufferSize)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Group the segments in a log into groups totaling less than a given size. the size is enforced separately for the log data and the index data.
|
||||||
|
* We collect a group of such segments together into a single
|
||||||
|
* destination segment. This prevents segment sizes from shrinking too much.
|
||||||
|
*
|
||||||
|
* @param segments The log segments to group
|
||||||
|
* @param maxSize the maximum size in bytes for the total of all log data in a group
|
||||||
|
* @param maxIndexSize the maximum size in bytes for the total of all index data in a group
|
||||||
|
*
|
||||||
|
* @return A list of grouped segments
|
||||||
|
*/
|
||||||
|
private[log] def groupSegmentsBySize(segments: Iterable[LogSegment], maxSize: Int, maxIndexSize: Int): List[Seq[LogSegment]] = {
|
||||||
|
var grouped = List[List[LogSegment]]()
|
||||||
|
var segs = segments.toList
|
||||||
|
while(!segs.isEmpty) {
|
||||||
|
var group = List(segs.head)
|
||||||
|
var logSize = segs.head.size
|
||||||
|
var indexSize = segs.head.index.sizeInBytes
|
||||||
|
segs = segs.tail
|
||||||
|
while(!segs.isEmpty &&
|
||||||
|
logSize + segs.head.size < maxSize &&
|
||||||
|
indexSize + segs.head.index.sizeInBytes < maxIndexSize) {
|
||||||
|
group = segs.head :: group
|
||||||
|
logSize += segs.head.size
|
||||||
|
indexSize += segs.head.index.sizeInBytes
|
||||||
|
segs = segs.tail
|
||||||
|
}
|
||||||
|
grouped ::= group.reverse
|
||||||
|
}
|
||||||
|
grouped.reverse
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a map of key_hash => offset for the keys in the dirty portion of the log to use in cleaning.
|
||||||
|
* @param log The log to use
|
||||||
|
* @param start The offset at which dirty messages begin
|
||||||
|
* @param end The ending offset for the map that is being built
|
||||||
|
* @param map The map in which to store the mappings
|
||||||
|
*
|
||||||
|
* @return The final offset the map covers
|
||||||
|
*/
|
||||||
|
private[log] def buildOffsetMap(log: Log, start: Long, end: Long, map: OffsetMap): Long = {
|
||||||
|
map.clear()
|
||||||
|
val segments = log.logSegments(start, end)
|
||||||
|
info("Building offset map for log %s for %d segments in offset range [%d, %d).".format(log.name, segments.size, start, end))
|
||||||
|
var offset = segments.head.baseOffset
|
||||||
|
require(offset == start, "Last clean offset is %d but segment base offset is %d for log %s.".format(start, offset, log.name))
|
||||||
|
for (segment <- segments) {
|
||||||
|
checkDone()
|
||||||
|
offset = buildOffsetMap(segment, map)
|
||||||
|
}
|
||||||
|
info("Offset map for log %s complete.".format(log.name))
|
||||||
|
offset
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add the messages in the given segment to the offset map
|
||||||
|
*
|
||||||
|
* @param segment The segment to index
|
||||||
|
* @param map The map in which to store the key=>offset mapping
|
||||||
|
*
|
||||||
|
* @return The final offset covered by the map
|
||||||
|
*/
|
||||||
|
private def buildOffsetMap(segment: LogSegment, map: OffsetMap): Long = {
|
||||||
|
var position = 0
|
||||||
|
var offset = segment.baseOffset
|
||||||
|
while (position < segment.log.sizeInBytes) {
|
||||||
|
checkDone()
|
||||||
|
readBuffer.clear()
|
||||||
|
val messages = new ByteBufferMessageSet(segment.log.readInto(readBuffer, position))
|
||||||
|
throttler.maybeThrottle(messages.sizeInBytes)
|
||||||
|
val startPosition = position
|
||||||
|
for (entry <- messages) {
|
||||||
|
val message = entry.message
|
||||||
|
require(message.hasKey)
|
||||||
|
val size = MessageSet.entrySize(message)
|
||||||
|
position += size
|
||||||
|
map.put(message.key, entry.offset)
|
||||||
|
offset = entry.offset
|
||||||
|
stats.indexMessage(size)
|
||||||
|
}
|
||||||
|
// if we didn't read even one complete message, our read buffer may be too small
|
||||||
|
if(position == startPosition)
|
||||||
|
growBuffers()
|
||||||
|
}
|
||||||
|
restoreBuffers()
|
||||||
|
offset
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If we aren't running any more throw an AllDoneException
|
||||||
|
*/
|
||||||
|
private def checkDone() {
|
||||||
|
if (Thread.currentThread.isInterrupted)
|
||||||
|
throw new InterruptedException
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A simple struct for collecting stats about log cleaning
|
||||||
|
*/
|
||||||
|
private case class CleanerStats(time: Time = SystemTime) {
|
||||||
|
var startTime, mapCompleteTime, endTime, bytesRead, bytesWritten, mapBytesRead, mapMessagesRead, messagesRead, messagesWritten = 0L
|
||||||
|
clear()
|
||||||
|
|
||||||
|
def readMessage(size: Int) {
|
||||||
|
messagesRead += 1
|
||||||
|
bytesRead += size
|
||||||
|
}
|
||||||
|
|
||||||
|
def recopyMessage(size: Int) {
|
||||||
|
messagesWritten += 1
|
||||||
|
bytesWritten += size
|
||||||
|
}
|
||||||
|
|
||||||
|
def indexMessage(size: Int) {
|
||||||
|
mapMessagesRead += 1
|
||||||
|
mapBytesRead += size
|
||||||
|
}
|
||||||
|
|
||||||
|
def indexDone() {
|
||||||
|
mapCompleteTime = time.milliseconds
|
||||||
|
}
|
||||||
|
|
||||||
|
def allDone() {
|
||||||
|
endTime = time.milliseconds
|
||||||
|
}
|
||||||
|
|
||||||
|
def elapsedSecs = (endTime - startTime)/1000.0
|
||||||
|
|
||||||
|
def elapsedIndexSecs = (mapCompleteTime - startTime)/1000.0
|
||||||
|
|
||||||
|
def clear() {
|
||||||
|
startTime = time.milliseconds
|
||||||
|
mapCompleteTime = -1L
|
||||||
|
endTime = -1L
|
||||||
|
bytesRead = 0L
|
||||||
|
bytesWritten = 0L
|
||||||
|
mapBytesRead = 0L
|
||||||
|
mapMessagesRead = 0L
|
||||||
|
messagesRead = 0L
|
||||||
|
messagesWritten = 0L
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper class for a log, its topic/partition, and the last clean position
|
||||||
|
*/
|
||||||
|
private case class LogToClean(topicPartition: TopicAndPartition, log: Log, firstDirtyOffset: Long) extends Ordered[LogToClean] {
|
||||||
|
val cleanBytes = log.logSegments(-1, firstDirtyOffset-1).map(_.size).sum
|
||||||
|
val dirtyBytes = log.logSegments(firstDirtyOffset, math.max(firstDirtyOffset, log.activeSegment.baseOffset)).map(_.size).sum
|
||||||
|
val cleanableRatio = dirtyBytes / totalBytes.toDouble
|
||||||
|
def totalBytes = cleanBytes + dirtyBytes
|
||||||
|
override def compare(that: LogToClean): Int = math.signum(this.cleanableRatio - that.cleanableRatio).toInt
|
||||||
|
}
|
|
@ -0,0 +1,51 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka.log
|
||||||
|
|
||||||
|
import java.io.File
|
||||||
|
import scala.collection._
|
||||||
|
import kafka.common._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configuration settings for a log
|
||||||
|
* @param segmentSize The soft maximum for the size of a segment file in the log
|
||||||
|
* @param segmentMs The soft maximum on the amount of time before a new log segment is rolled
|
||||||
|
* @param flushInterval The number of messages that can be written to the log before a flush is forced
|
||||||
|
* @param flushMs The amount of time the log can have dirty data before a flush is forced
|
||||||
|
* @param retentionSize The approximate total number of bytes this log can use
|
||||||
|
* @param retentionMs The age approximate maximum age of the last segment that is retained
|
||||||
|
* @param maxIndexSize The maximum size of an index file
|
||||||
|
* @param indexInterval The approximate number of bytes between index entries
|
||||||
|
* @param fileDeleteDelayMs The time to wait before deleting a file from the filesystem
|
||||||
|
* @param minCleanableRatio The ratio of bytes that are available for cleaning to the bytes already cleaned
|
||||||
|
* @param dedupe Should old segments in this log be deleted or deduplicated?
|
||||||
|
*/
|
||||||
|
case class LogConfig(val segmentSize: Int = 1024*1024,
|
||||||
|
val segmentMs: Long = Long.MaxValue,
|
||||||
|
val flushInterval: Long = Long.MaxValue,
|
||||||
|
val flushMs: Long = Long.MaxValue,
|
||||||
|
val retentionSize: Long = Long.MaxValue,
|
||||||
|
val retentionMs: Long = Long.MaxValue,
|
||||||
|
val maxMessageSize: Int = Int.MaxValue,
|
||||||
|
val maxIndexSize: Int = 1024*1024,
|
||||||
|
val indexInterval: Int = 4096,
|
||||||
|
val fileDeleteDelayMs: Long = 60*1000,
|
||||||
|
val minCleanableRatio: Double = 0.5,
|
||||||
|
val dedupe: Boolean = false)
|
||||||
|
|
||||||
|
|
|
@ -36,32 +36,31 @@ import kafka.server.KafkaConfig
|
||||||
* A background thread handles log retention by periodically truncating excess log segments.
|
* A background thread handles log retention by periodically truncating excess log segments.
|
||||||
*/
|
*/
|
||||||
@threadsafe
|
@threadsafe
|
||||||
class LogManager(val config: KafkaConfig,
|
class LogManager(val logDirs: Array[File],
|
||||||
|
val topicConfigs: Map[String, LogConfig],
|
||||||
|
val defaultConfig: LogConfig,
|
||||||
|
val cleanerConfig: CleanerConfig,
|
||||||
|
val flushCheckMs: Long,
|
||||||
|
val retentionCheckMs: Long,
|
||||||
scheduler: Scheduler,
|
scheduler: Scheduler,
|
||||||
private val time: Time) extends Logging {
|
private val time: Time) extends Logging {
|
||||||
|
|
||||||
val CleanShutdownFile = ".kafka_cleanshutdown"
|
val CleanShutdownFile = ".kafka_cleanshutdown"
|
||||||
val LockFile = ".lock"
|
val LockFile = ".lock"
|
||||||
val InitialTaskDelayMs = 30*1000
|
val InitialTaskDelayMs = 30*1000
|
||||||
val logDirs: Array[File] = config.logDirs.map(new File(_)).toArray
|
|
||||||
private val logFileSizeMap = config.logSegmentBytesPerTopicMap
|
|
||||||
private val logFlushInterval = config.logFlushIntervalMessages
|
|
||||||
private val logFlushIntervals = config.logFlushIntervalMsPerTopicMap
|
|
||||||
private val logCreationLock = new Object
|
private val logCreationLock = new Object
|
||||||
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.logCleanupIntervalMins
|
|
||||||
private val logCleanupDefaultAgeMs = 1000L * 60 * 60 * config.logRetentionHours
|
|
||||||
|
|
||||||
this.logIdent = "[Log Manager on Broker " + config.brokerId + "] "
|
|
||||||
private val logs = new Pool[TopicAndPartition, Log]()
|
private val logs = new Pool[TopicAndPartition, Log]()
|
||||||
|
|
||||||
createAndValidateLogDirs(logDirs)
|
createAndValidateLogDirs(logDirs)
|
||||||
private var dirLocks = lockLogDirs(logDirs)
|
private var dirLocks = lockLogDirs(logDirs)
|
||||||
loadLogs(logDirs)
|
loadLogs(logDirs)
|
||||||
|
|
||||||
|
private val cleaner: LogCleaner =
|
||||||
|
if(cleanerConfig.enableCleaner)
|
||||||
|
new LogCleaner(cleanerConfig, logDirs, logs, time = time)
|
||||||
|
else
|
||||||
|
null
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create and check validity of the given directories, specifically:
|
* Create and check validity of the given directories, specifically:
|
||||||
* <ol>
|
* <ol>
|
||||||
|
@ -114,18 +113,11 @@ class LogManager(val config: KafkaConfig,
|
||||||
if(dir.isDirectory){
|
if(dir.isDirectory){
|
||||||
info("Loading log '" + dir.getName + "'")
|
info("Loading log '" + dir.getName + "'")
|
||||||
val topicPartition = parseTopicPartitionName(dir.getName)
|
val topicPartition = parseTopicPartitionName(dir.getName)
|
||||||
val rollIntervalMs = logRollMsMap.get(topicPartition.topic).getOrElse(this.logRollDefaultIntervalMs)
|
val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
|
||||||
val maxLogFileSize = logFileSizeMap.get(topicPartition.topic).getOrElse(config.logSegmentBytes)
|
|
||||||
val log = new Log(dir,
|
val log = new Log(dir,
|
||||||
scheduler,
|
config,
|
||||||
maxLogFileSize,
|
|
||||||
config.messageMaxBytes,
|
|
||||||
logFlushInterval,
|
|
||||||
rollIntervalMs,
|
|
||||||
needsRecovery,
|
needsRecovery,
|
||||||
config.logIndexSizeMaxBytes,
|
scheduler,
|
||||||
config.logIndexIntervalBytes,
|
|
||||||
config.logDeleteDelayMs,
|
|
||||||
time)
|
time)
|
||||||
val previous = this.logs.put(topicPartition, log)
|
val previous = this.logs.put(topicPartition, log)
|
||||||
if(previous != null)
|
if(previous != null)
|
||||||
|
@ -142,20 +134,41 @@ class LogManager(val config: KafkaConfig,
|
||||||
def startup() {
|
def startup() {
|
||||||
/* Schedule the cleanup task to delete old logs */
|
/* Schedule the cleanup task to delete old logs */
|
||||||
if(scheduler != null) {
|
if(scheduler != null) {
|
||||||
info("Starting log cleanup with a period of %d ms.".format(logCleanupIntervalMs))
|
info("Starting log cleanup with a period of %d ms.".format(retentionCheckMs))
|
||||||
scheduler.schedule("kafka-log-cleaner",
|
scheduler.schedule("kafka-log-retention",
|
||||||
cleanupLogs,
|
cleanupLogs,
|
||||||
delay = InitialTaskDelayMs,
|
delay = InitialTaskDelayMs,
|
||||||
period = logCleanupIntervalMs,
|
period = retentionCheckMs,
|
||||||
TimeUnit.MILLISECONDS)
|
TimeUnit.MILLISECONDS)
|
||||||
info("Starting log flusher with a default period of %d ms with the following overrides: %s."
|
info("Starting log flusher with a default period of %d ms.".format(flushCheckMs))
|
||||||
.format(config.logFlushIntervalMs, logFlushIntervals.map(e => e._1.toString + "=" + e._2.toString).mkString(", ")))
|
|
||||||
scheduler.schedule("kafka-log-flusher",
|
scheduler.schedule("kafka-log-flusher",
|
||||||
flushDirtyLogs,
|
flushDirtyLogs,
|
||||||
delay = InitialTaskDelayMs,
|
delay = InitialTaskDelayMs,
|
||||||
period = config.logFlushSchedulerIntervalMs,
|
period = flushCheckMs,
|
||||||
TimeUnit.MILLISECONDS)
|
TimeUnit.MILLISECONDS)
|
||||||
}
|
}
|
||||||
|
if(cleanerConfig.enableCleaner)
|
||||||
|
cleaner.startup()
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Close all the logs
|
||||||
|
*/
|
||||||
|
def shutdown() {
|
||||||
|
debug("Shutting down.")
|
||||||
|
try {
|
||||||
|
// stop the cleaner first
|
||||||
|
if(cleaner != null)
|
||||||
|
Utils.swallow(cleaner.shutdown())
|
||||||
|
// close the logs
|
||||||
|
allLogs.foreach(_.close())
|
||||||
|
// mark that the shutdown was clean by creating the clean shutdown marker file
|
||||||
|
logDirs.foreach(dir => Utils.swallow(new File(dir, CleanShutdownFile).createNewFile()))
|
||||||
|
} finally {
|
||||||
|
// regardless of whether the close succeeded, we need to unlock the data directories
|
||||||
|
dirLocks.foreach(_.destroy())
|
||||||
|
}
|
||||||
|
debug("Shutdown complete.")
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -197,18 +210,10 @@ class LogManager(val config: KafkaConfig,
|
||||||
val dataDir = nextLogDir()
|
val dataDir = nextLogDir()
|
||||||
val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition)
|
val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition)
|
||||||
dir.mkdirs()
|
dir.mkdirs()
|
||||||
val rollIntervalMs = logRollMsMap.get(topicAndPartition.topic).getOrElse(this.logRollDefaultIntervalMs)
|
|
||||||
val maxLogFileSize = logFileSizeMap.get(topicAndPartition.topic).getOrElse(config.logSegmentBytes)
|
|
||||||
log = new Log(dir,
|
log = new Log(dir,
|
||||||
scheduler,
|
defaultConfig,
|
||||||
maxLogFileSize,
|
|
||||||
config.messageMaxBytes,
|
|
||||||
logFlushInterval,
|
|
||||||
rollIntervalMs,
|
|
||||||
needsRecovery = false,
|
needsRecovery = false,
|
||||||
config.logIndexSizeMaxBytes,
|
scheduler,
|
||||||
config.logIndexIntervalBytes,
|
|
||||||
config.logDeleteDelayMs,
|
|
||||||
time)
|
time)
|
||||||
info("Created log for topic %s partition %d in %s.".format(topicAndPartition.topic, topicAndPartition.partition, dataDir.getAbsolutePath))
|
info("Created log for topic %s partition %d in %s.".format(topicAndPartition.topic, topicAndPartition.partition, dataDir.getAbsolutePath))
|
||||||
logs.put(topicAndPartition, log)
|
logs.put(topicAndPartition, log)
|
||||||
|
@ -242,8 +247,7 @@ class LogManager(val config: KafkaConfig,
|
||||||
private def cleanupExpiredSegments(log: Log): Int = {
|
private def cleanupExpiredSegments(log: Log): Int = {
|
||||||
val startMs = time.milliseconds
|
val startMs = time.milliseconds
|
||||||
val topic = parseTopicPartitionName(log.name).topic
|
val topic = parseTopicPartitionName(log.name).topic
|
||||||
val logCleanupThresholdMs = logRetentionMsMap.get(topic).getOrElse(this.logCleanupDefaultAgeMs)
|
log.deleteOldSegments(startMs - _.lastModified > log.config.retentionMs)
|
||||||
log.deleteOldSegments(startMs - _.lastModified > logCleanupThresholdMs)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -252,10 +256,9 @@ class LogManager(val config: KafkaConfig,
|
||||||
*/
|
*/
|
||||||
private def cleanupSegmentsToMaintainSize(log: Log): Int = {
|
private def cleanupSegmentsToMaintainSize(log: Log): Int = {
|
||||||
val topic = parseTopicPartitionName(log.dir.getName).topic
|
val topic = parseTopicPartitionName(log.dir.getName).topic
|
||||||
val maxLogRetentionSize = logRetentionSizeMap.get(topic).getOrElse(config.logRetentionBytes)
|
if(log.config.retentionSize < 0 || log.size < log.config.retentionSize)
|
||||||
if(maxLogRetentionSize < 0 || log.size < maxLogRetentionSize)
|
|
||||||
return 0
|
return 0
|
||||||
var diff = log.size - maxLogRetentionSize
|
var diff = log.size - log.config.retentionSize
|
||||||
def shouldDelete(segment: LogSegment) = {
|
def shouldDelete(segment: LogSegment) = {
|
||||||
if(diff - segment.size >= 0) {
|
if(diff - segment.size >= 0) {
|
||||||
diff -= segment.size
|
diff -= segment.size
|
||||||
|
@ -274,7 +277,7 @@ class LogManager(val config: KafkaConfig,
|
||||||
debug("Beginning log cleanup...")
|
debug("Beginning log cleanup...")
|
||||||
var total = 0
|
var total = 0
|
||||||
val startMs = time.milliseconds
|
val startMs = time.milliseconds
|
||||||
for(log <- allLogs) {
|
for(log <- allLogs; if !log.config.dedupe) {
|
||||||
debug("Garbage collecting '" + log.name + "'")
|
debug("Garbage collecting '" + log.name + "'")
|
||||||
total += cleanupExpiredSegments(log) + cleanupSegmentsToMaintainSize(log)
|
total += cleanupExpiredSegments(log) + cleanupSegmentsToMaintainSize(log)
|
||||||
}
|
}
|
||||||
|
@ -282,23 +285,6 @@ class LogManager(val config: KafkaConfig,
|
||||||
(time.milliseconds - startMs) / 1000 + " seconds")
|
(time.milliseconds - startMs) / 1000 + " seconds")
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Close all the logs
|
|
||||||
*/
|
|
||||||
def shutdown() {
|
|
||||||
debug("Shutting down.")
|
|
||||||
try {
|
|
||||||
// close the logs
|
|
||||||
allLogs.foreach(_.close())
|
|
||||||
// mark that the shutdown was clean by creating the clean shutdown marker file
|
|
||||||
logDirs.foreach(dir => Utils.swallow(new File(dir, CleanShutdownFile).createNewFile()))
|
|
||||||
} finally {
|
|
||||||
// regardless of whether the close succeeded, we need to unlock the data directories
|
|
||||||
dirLocks.foreach(_.destroy())
|
|
||||||
}
|
|
||||||
debug("Shutdown complete.")
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get all the partition logs
|
* Get all the partition logs
|
||||||
*/
|
*/
|
||||||
|
@ -312,13 +298,9 @@ class LogManager(val config: KafkaConfig,
|
||||||
for ((topicAndPartition, log) <- logs) {
|
for ((topicAndPartition, log) <- logs) {
|
||||||
try {
|
try {
|
||||||
val timeSinceLastFlush = time.milliseconds - log.lastFlushTime
|
val timeSinceLastFlush = time.milliseconds - log.lastFlushTime
|
||||||
|
debug("Checking if flush is needed on " + topicAndPartition.topic + " flush interval " + log.config.flushMs +
|
||||||
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 +
|
|
||||||
" last flushed " + log.lastFlushTime + " time since last flush: " + timeSinceLastFlush)
|
" last flushed " + log.lastFlushTime + " time since last flush: " + timeSinceLastFlush)
|
||||||
if(timeSinceLastFlush >= logFlushInterval)
|
if(timeSinceLastFlush >= log.config.flushMs)
|
||||||
log.flush
|
log.flush
|
||||||
} catch {
|
} catch {
|
||||||
case e =>
|
case e =>
|
||||||
|
|
|
@ -79,7 +79,7 @@ class LogSegment(val log: FileMessageSet,
|
||||||
* @return The position in the log storing the message with the least offset >= the requested offset or null if no message meets this criteria.
|
* @return The position in the log storing the message with the least offset >= the requested offset or null if no message meets this criteria.
|
||||||
*/
|
*/
|
||||||
@threadsafe
|
@threadsafe
|
||||||
private def translateOffset(offset: Long, startingFilePosition: Int = 0): OffsetPosition = {
|
private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): OffsetPosition = {
|
||||||
val mapping = index.lookup(offset)
|
val mapping = index.lookup(offset)
|
||||||
log.searchFor(offset, max(mapping.position, startingFilePosition))
|
log.searchFor(offset, max(mapping.position, startingFilePosition))
|
||||||
}
|
}
|
||||||
|
@ -168,18 +168,20 @@ class LogSegment(val log: FileMessageSet,
|
||||||
* Truncate off all index and log entries with offsets >= the given offset.
|
* Truncate off all index and log entries with offsets >= the given offset.
|
||||||
* If the given offset is larger than the largest message in this segment, do nothing.
|
* If the given offset is larger than the largest message in this segment, do nothing.
|
||||||
* @param offset The offset to truncate to
|
* @param offset The offset to truncate to
|
||||||
|
* @return The number of log bytes truncated
|
||||||
*/
|
*/
|
||||||
@nonthreadsafe
|
@nonthreadsafe
|
||||||
def truncateTo(offset: Long) {
|
def truncateTo(offset: Long): Int = {
|
||||||
val mapping = translateOffset(offset)
|
val mapping = translateOffset(offset)
|
||||||
if(mapping == null)
|
if(mapping == null)
|
||||||
return
|
return 0
|
||||||
index.truncateTo(offset)
|
index.truncateTo(offset)
|
||||||
// after truncation, reset and allocate more space for the (new currently active) index
|
// after truncation, reset and allocate more space for the (new currently active) index
|
||||||
index.resize(index.maxIndexSize)
|
index.resize(index.maxIndexSize)
|
||||||
log.truncateTo(mapping.position)
|
val bytesTruncated = log.truncateTo(mapping.position)
|
||||||
if (log.sizeInBytes == 0)
|
if(log.sizeInBytes == 0)
|
||||||
created = time.milliseconds
|
created = time.milliseconds
|
||||||
|
bytesTruncated
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -210,6 +212,18 @@ class LogSegment(val log: FileMessageSet,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Change the suffix for the index and log file for this log segment
|
||||||
|
*/
|
||||||
|
def changeFileSuffixes(oldSuffix: String, newSuffix: String) {
|
||||||
|
val logRenamed = log.renameTo(new File(Utils.replaceSuffix(log.file.getPath, oldSuffix, newSuffix)))
|
||||||
|
if(!logRenamed)
|
||||||
|
throw new KafkaStorageException("Failed to change the log file suffix from %s to %s for log segment %d".format(oldSuffix, newSuffix, baseOffset))
|
||||||
|
val indexRenamed = index.renameTo(new File(Utils.replaceSuffix(index.file.getPath, oldSuffix, newSuffix)))
|
||||||
|
if(!indexRenamed)
|
||||||
|
throw new KafkaStorageException("Failed to change the index file suffix from %s to %s for log segment %d".format(oldSuffix, newSuffix, baseOffset))
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Close this log segment
|
* Close this log segment
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -49,7 +49,7 @@ import kafka.utils._
|
||||||
* All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal
|
* All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal
|
||||||
* storage format.
|
* storage format.
|
||||||
*/
|
*/
|
||||||
class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = -1) extends Logging {
|
class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSize: Int = -1) extends Logging {
|
||||||
|
|
||||||
/* initialize the memory mapping for this index */
|
/* initialize the memory mapping for this index */
|
||||||
private var mmap: MappedByteBuffer =
|
private var mmap: MappedByteBuffer =
|
||||||
|
@ -84,20 +84,15 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
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
|
|
||||||
*/
|
|
||||||
def maxEntries = mmap.limit / 8
|
|
||||||
|
|
||||||
/* the number of eight-byte entries currently in the index */
|
/* the number of eight-byte entries currently in the index */
|
||||||
private var size = new AtomicInteger(mmap.position / 8)
|
private var size = new AtomicInteger(mmap.position / 8)
|
||||||
|
|
||||||
/* the last offset in the index */
|
/* the last offset in the index */
|
||||||
var lastOffset = readLastOffset()
|
var lastOffset = readLastOffset()
|
||||||
|
|
||||||
|
info("Created index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d"
|
||||||
|
.format(file.getAbsolutePath, maxEntries, maxIndexSize, entries(), lastOffset))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The last offset written to the index
|
* The last offset written to the index
|
||||||
*/
|
*/
|
||||||
|
@ -110,6 +105,11 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
|
||||||
baseOffset + offset
|
baseOffset + offset
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The maximum number of eight-byte entries this index can hold
|
||||||
|
*/
|
||||||
|
def maxEntries = mmap.limit / 8
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Find the largest offset less than or equal to the given targetOffset
|
* Find the largest offset less than or equal to the given targetOffset
|
||||||
* and return a pair holding this offset and it's corresponding physical file position.
|
* and return a pair holding this offset and it's corresponding physical file position.
|
||||||
|
@ -284,11 +284,26 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
|
||||||
/** The number of entries in this index */
|
/** The number of entries in this index */
|
||||||
def entries() = size.get
|
def entries() = size.get
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The number of bytes actually used by this index
|
||||||
|
*/
|
||||||
|
def sizeInBytes() = 8 * entries
|
||||||
|
|
||||||
/** Close the index */
|
/** Close the index */
|
||||||
def close() {
|
def close() {
|
||||||
trimToValidSize()
|
trimToValidSize()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Rename the file that backs this offset index
|
||||||
|
* @return true iff the rename was successful
|
||||||
|
*/
|
||||||
|
def renameTo(f: File): Boolean = {
|
||||||
|
val success = this.file.renameTo(f)
|
||||||
|
this.file = f
|
||||||
|
success
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Round a number to the greatest exact multiple of the given factor less than the given number.
|
* Round a number to the greatest exact multiple of the given factor less than the given number.
|
||||||
* E.g. roundToExactMultiple(67, 8) == 64
|
* E.g. roundToExactMultiple(67, 8) == 64
|
||||||
|
|
|
@ -0,0 +1,136 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka.log
|
||||||
|
|
||||||
|
import java.util.Arrays
|
||||||
|
import java.security.MessageDigest
|
||||||
|
import java.nio.ByteBuffer
|
||||||
|
import kafka.utils._
|
||||||
|
|
||||||
|
trait OffsetMap {
|
||||||
|
def capacity: Int
|
||||||
|
def put(key: ByteBuffer, offset: Long)
|
||||||
|
def get(key: ByteBuffer): Long
|
||||||
|
def clear()
|
||||||
|
def size: Int
|
||||||
|
def utilization: Double = size.toDouble / capacity
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An approximate map used for deduplicating the log.
|
||||||
|
* @param memory The amount of memory this map can use
|
||||||
|
* @param maxLoadFactor The maximum percent full this offset map can be
|
||||||
|
* @param hashAlgorithm The hash algorithm instance to use: MD2, MD5, SHA-1, SHA-256, SHA-384, SHA-512
|
||||||
|
*/
|
||||||
|
@nonthreadsafe
|
||||||
|
class SkimpyOffsetMap(val memory: Int, val maxLoadFactor: Double, val hashAlgorithm: String = "MD5") extends OffsetMap {
|
||||||
|
private val bytes = ByteBuffer.allocate(memory)
|
||||||
|
|
||||||
|
/* the hash algorithm instance to use, defualt is MD5 */
|
||||||
|
private val digest = MessageDigest.getInstance(hashAlgorithm)
|
||||||
|
|
||||||
|
/* the number of bytes for this hash algorithm */
|
||||||
|
private val hashSize = digest.getDigestLength
|
||||||
|
|
||||||
|
/* create some hash buffers to avoid reallocating each time */
|
||||||
|
private val hash1 = new Array[Byte](hashSize)
|
||||||
|
private val hash2 = new Array[Byte](hashSize)
|
||||||
|
|
||||||
|
/* number of entries put into the map */
|
||||||
|
private var entries = 0
|
||||||
|
|
||||||
|
/* a byte added as a prefix to all keys to make collisions non-static in repeated uses. Changed in clear(). */
|
||||||
|
private var salt: Byte = 0
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The number of bytes of space each entry uses (the number of bytes in the hash plus an 8 byte offset)
|
||||||
|
*/
|
||||||
|
val bytesPerEntry = hashSize + 8
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The maximum number of entries this map can contain before it exceeds the max load factor
|
||||||
|
*/
|
||||||
|
override val capacity: Int = (maxLoadFactor * memory / bytesPerEntry).toInt
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Associate a offset with a key.
|
||||||
|
* @param key The key
|
||||||
|
* @param offset The offset
|
||||||
|
*/
|
||||||
|
override def put(key: ByteBuffer, offset: Long) {
|
||||||
|
if(size + 1 > capacity)
|
||||||
|
throw new IllegalStateException("Attempt to add to a full offset map with a maximum capacity of %d.".format(capacity))
|
||||||
|
hash(key, hash1)
|
||||||
|
bytes.position(offsetFor(hash1))
|
||||||
|
bytes.put(hash1)
|
||||||
|
bytes.putLong(offset)
|
||||||
|
entries += 1
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the offset associated with this key. This method is approximate,
|
||||||
|
* it may not find an offset previously stored, but cannot give a wrong offset.
|
||||||
|
* @param key The key
|
||||||
|
* @return The offset associated with this key or -1 if the key is not found
|
||||||
|
*/
|
||||||
|
override def get(key: ByteBuffer): Long = {
|
||||||
|
hash(key, hash1)
|
||||||
|
bytes.position(offsetFor(hash1))
|
||||||
|
bytes.get(hash2)
|
||||||
|
// if the computed hash equals the stored hash return the stored offset
|
||||||
|
if(Arrays.equals(hash1, hash2))
|
||||||
|
bytes.getLong()
|
||||||
|
else
|
||||||
|
-1L
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Change the salt used for key hashing making all existing keys unfindable.
|
||||||
|
* Doesn't actually zero out the array.
|
||||||
|
*/
|
||||||
|
override def clear() {
|
||||||
|
this.entries = 0
|
||||||
|
this.salt = (this.salt + 1).toByte
|
||||||
|
Arrays.fill(bytes.array, bytes.arrayOffset, bytes.arrayOffset + bytes.limit, 0.toByte)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The number of entries put into the map (note that not all may remain)
|
||||||
|
*/
|
||||||
|
override def size: Int = entries
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Choose a slot in the array for this hash
|
||||||
|
*/
|
||||||
|
private def offsetFor(hash: Array[Byte]): Int =
|
||||||
|
bytesPerEntry * (Utils.abs(Utils.readInt(hash, 0)) % capacity)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The offset at which we have stored the given key
|
||||||
|
* @param key The key to hash
|
||||||
|
* @param buffer The buffer to store the hash into
|
||||||
|
*/
|
||||||
|
private def hash(key: ByteBuffer, buffer: Array[Byte]) {
|
||||||
|
key.mark()
|
||||||
|
digest.update(salt)
|
||||||
|
digest.update(key)
|
||||||
|
key.reset()
|
||||||
|
digest.digest(buffer, 0, hashSize)
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -73,7 +73,7 @@ object ByteBufferMessageSet {
|
||||||
new ByteBufferMessageSet(outputBuffer)
|
new ByteBufferMessageSet(outputBuffer)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) {
|
private[kafka] def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) {
|
||||||
buffer.putLong(offset)
|
buffer.putLong(offset)
|
||||||
buffer.putInt(message.size)
|
buffer.putInt(message.size)
|
||||||
buffer.put(message.buffer)
|
buffer.put(message.buffer)
|
||||||
|
@ -150,7 +150,7 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message
|
||||||
return allDone()
|
return allDone()
|
||||||
val offset = topIter.getLong()
|
val offset = topIter.getLong()
|
||||||
val size = topIter.getInt()
|
val size = topIter.getInt()
|
||||||
if(size < 0)
|
if(size < 0 || size < Message.MinHeaderSize)
|
||||||
throw new InvalidMessageException("Message found with corrupt size (" + size + ")")
|
throw new InvalidMessageException("Message found with corrupt size (" + size + ")")
|
||||||
|
|
||||||
// we have an incomplete message
|
// we have an incomplete message
|
||||||
|
|
|
@ -1,118 +0,0 @@
|
||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package kafka.server
|
|
||||||
|
|
||||||
import kafka.utils.Logging
|
|
||||||
import kafka.common._
|
|
||||||
import java.util.concurrent.locks.ReentrantLock
|
|
||||||
import java.io._
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This class handles the read/write to the highwaterMark checkpoint file. The file stores the high watermark value for
|
|
||||||
* all topics and partitions that this broker hosts. The format of this file is as follows -
|
|
||||||
* version
|
|
||||||
* number of entries
|
|
||||||
* topic partition highwatermark
|
|
||||||
*/
|
|
||||||
|
|
||||||
object HighwaterMarkCheckpoint {
|
|
||||||
val highWatermarkFileName = "replication-offset-checkpoint"
|
|
||||||
val currentHighwaterMarkFileVersion = 0
|
|
||||||
}
|
|
||||||
|
|
||||||
class HighwaterMarkCheckpoint(val path: String) extends Logging {
|
|
||||||
/* create the highwatermark file handle for all partitions */
|
|
||||||
val name = path + "/" + HighwaterMarkCheckpoint.highWatermarkFileName
|
|
||||||
private val hwFile = new File(name)
|
|
||||||
private val hwFileLock = new ReentrantLock()
|
|
||||||
// recover from previous tmp file, if required
|
|
||||||
|
|
||||||
def write(highwaterMarksPerPartition: Map[TopicAndPartition, Long]) {
|
|
||||||
hwFileLock.lock()
|
|
||||||
try {
|
|
||||||
// write to temp file and then swap with the highwatermark file
|
|
||||||
val tempHwFile = new File(hwFile + ".tmp")
|
|
||||||
|
|
||||||
val hwFileWriter = new BufferedWriter(new FileWriter(tempHwFile))
|
|
||||||
// checkpoint highwatermark for all partitions
|
|
||||||
// write the current version
|
|
||||||
hwFileWriter.write(HighwaterMarkCheckpoint.currentHighwaterMarkFileVersion.toString)
|
|
||||||
hwFileWriter.newLine()
|
|
||||||
// write the number of entries in the highwatermark file
|
|
||||||
hwFileWriter.write(highwaterMarksPerPartition.size.toString)
|
|
||||||
hwFileWriter.newLine()
|
|
||||||
|
|
||||||
highwaterMarksPerPartition.foreach { partitionAndHw =>
|
|
||||||
hwFileWriter.write("%s %s %s".format(partitionAndHw._1.topic, partitionAndHw._1.partition, partitionAndHw._2))
|
|
||||||
hwFileWriter.newLine()
|
|
||||||
}
|
|
||||||
hwFileWriter.flush()
|
|
||||||
hwFileWriter.close()
|
|
||||||
// swap new high watermark file with previous one
|
|
||||||
if(!tempHwFile.renameTo(hwFile)) {
|
|
||||||
fatal("Attempt to swap the new high watermark file with the old one failed")
|
|
||||||
System.exit(1)
|
|
||||||
}
|
|
||||||
}finally {
|
|
||||||
hwFileLock.unlock()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
def read(topic: String, partition: Int): Long = {
|
|
||||||
hwFileLock.lock()
|
|
||||||
try {
|
|
||||||
hwFile.length() match {
|
|
||||||
case 0 =>
|
|
||||||
warn("No highwatermark file is found. Returning 0 as the highwatermark for topic %s partition %d."
|
|
||||||
.format(topic, partition))
|
|
||||||
0L
|
|
||||||
case _ =>
|
|
||||||
val hwFileReader = new BufferedReader(new FileReader(hwFile))
|
|
||||||
val version = hwFileReader.readLine().toShort
|
|
||||||
version match {
|
|
||||||
case HighwaterMarkCheckpoint.currentHighwaterMarkFileVersion =>
|
|
||||||
val numberOfHighWatermarks = hwFileReader.readLine().toInt
|
|
||||||
val partitionHighWatermarks =
|
|
||||||
for(i <- 0 until numberOfHighWatermarks) yield {
|
|
||||||
val nextHwEntry = hwFileReader.readLine()
|
|
||||||
val partitionHwInfo = nextHwEntry.split(" ")
|
|
||||||
val topic = partitionHwInfo(0)
|
|
||||||
val partitionId = partitionHwInfo(1).toInt
|
|
||||||
val highWatermark = partitionHwInfo(2).toLong
|
|
||||||
(TopicAndPartition(topic, partitionId) -> highWatermark)
|
|
||||||
}
|
|
||||||
hwFileReader.close()
|
|
||||||
val hwOpt = partitionHighWatermarks.toMap.get(TopicAndPartition(topic, partition))
|
|
||||||
hwOpt match {
|
|
||||||
case Some(hw) =>
|
|
||||||
debug("Read hw %d for topic %s partition %d from highwatermark checkpoint file".format(hw, topic, partition))
|
|
||||||
hw
|
|
||||||
case None =>
|
|
||||||
warn("No previously checkpointed highwatermark value found for topic %s ".format(topic) +
|
|
||||||
"partition %d. Returning 0 as the highwatermark".format(partition))
|
|
||||||
0L
|
|
||||||
}
|
|
||||||
case _ => fatal("Unrecognized version of the highwatermark checkpoint file " + version)
|
|
||||||
System.exit(1)
|
|
||||||
-1L
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}finally {
|
|
||||||
hwFileLock.unlock()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -105,7 +105,35 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
|
||||||
val logRetentionBytesPerTopicMap = props.getMap("log.retention.bytes.per.topic", _.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 */
|
/* the frequency in minutes that the log cleaner checks whether any log is eligible for deletion */
|
||||||
val logCleanupIntervalMins = props.getIntInRange("log.cleanup.interval.mins", 10, (1, Int.MaxValue))
|
val logCleanupIntervalMs = props.getLongInRange("log.retention.check.interval.ms", 5*60*1000, (1, Long.MaxValue))
|
||||||
|
|
||||||
|
/* the default cleanup policy for segments beyond the retention window, must be either "delete" or "dedupe" */
|
||||||
|
val logCleanupPolicy = props.getString("log.cleanup.policy", "delete")
|
||||||
|
|
||||||
|
/* a per-topic override for the cleanup policy for segments beyond the retention window */
|
||||||
|
val logCleanupPolicyMap = props.getMap("topic.log.cleanup.policy")
|
||||||
|
|
||||||
|
/* the number of background threads to use for log cleaning */
|
||||||
|
val logCleanerThreads = props.getIntInRange("log.cleaner.threads", 1, (0, Int.MaxValue))
|
||||||
|
|
||||||
|
/* the log cleaner will be throttled so that the sum of its read and write i/o will be less than this value on average */
|
||||||
|
val logCleanerIoMaxBytesPerSecond = props.getDouble("log.cleaner.io.max.bytes.per.second", Double.MaxValue)
|
||||||
|
|
||||||
|
/* the total memory used for log deduplication across all cleaner threads */
|
||||||
|
val logCleanerDedupeBufferSize = props.getIntInRange("log.cleaner.dedupe.buffer.size", 500*1024*1024, (0, Int.MaxValue))
|
||||||
|
require(logCleanerDedupeBufferSize / logCleanerThreads > 1024*1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.")
|
||||||
|
|
||||||
|
/* the total memory used for log cleaner I/O buffers across all cleaner threads */
|
||||||
|
val logCleanerIoBufferSize = props.getIntInRange("log.cleaner.io.buffer.size", 4*1024*1024, (0, Int.MaxValue))
|
||||||
|
|
||||||
|
/* the amount of time to sleep when there are no logs to clean */
|
||||||
|
val logCleanerBackoffMs = props.getLongInRange("log.cleaner.backoff.ms", 30*1000, (0L, Long.MaxValue))
|
||||||
|
|
||||||
|
/* the minimum ratio of dirty log to total log for a log to eligible for cleaning */
|
||||||
|
val logCleanerMinCleanRatio = props.getDouble("log.cleaner.min.cleanable.ratio", 0.5)
|
||||||
|
|
||||||
|
/* should we enable log cleaning? */
|
||||||
|
val logCleanerEnable = props.getBoolean("log.cleaner.enable", false)
|
||||||
|
|
||||||
/* the maximum size in bytes of the offset index */
|
/* the maximum size in bytes of the offset index */
|
||||||
val logIndexSizeMaxBytes = props.getIntInRange("log.index.size.max.bytes", 10*1024*1024, (4, Int.MaxValue))
|
val logIndexSizeMaxBytes = props.getIntInRange("log.index.size.max.bytes", 10*1024*1024, (4, Int.MaxValue))
|
||||||
|
@ -116,6 +144,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
|
||||||
/* the number of messages accumulated on a log partition before messages are flushed to disk */
|
/* the number of messages accumulated on a log partition before messages are flushed to disk */
|
||||||
val logFlushIntervalMessages = props.getIntInRange("log.flush.interval.messages", 500, (1, Int.MaxValue))
|
val logFlushIntervalMessages = props.getIntInRange("log.flush.interval.messages", 500, (1, Int.MaxValue))
|
||||||
|
|
||||||
|
/* the amount of time to wait before deleting a file from the filesystem */
|
||||||
val logDeleteDelayMs = props.getLongInRange("log.segment.delete.delay.ms", 60000, (0, Long.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 */
|
/* 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 */
|
||||||
|
|
|
@ -18,9 +18,12 @@
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import kafka.network.SocketServer
|
import kafka.network.SocketServer
|
||||||
|
import kafka.log.LogConfig
|
||||||
|
import kafka.log.CleanerConfig
|
||||||
import kafka.log.LogManager
|
import kafka.log.LogManager
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import java.util.concurrent._
|
import java.util.concurrent._
|
||||||
|
import java.io.File
|
||||||
import atomic.AtomicBoolean
|
import atomic.AtomicBoolean
|
||||||
import org.I0Itec.zkclient.ZkClient
|
import org.I0Itec.zkclient.ZkClient
|
||||||
import kafka.controller.{ControllerStats, KafkaController}
|
import kafka.controller.{ControllerStats, KafkaController}
|
||||||
|
@ -56,9 +59,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
||||||
kafkaScheduler.startup()
|
kafkaScheduler.startup()
|
||||||
|
|
||||||
/* start log manager */
|
/* start log manager */
|
||||||
logManager = new LogManager(config,
|
logManager = createLogManager(config)
|
||||||
kafkaScheduler,
|
|
||||||
time)
|
|
||||||
logManager.startup()
|
logManager.startup()
|
||||||
|
|
||||||
socketServer = new SocketServer(config.brokerId,
|
socketServer = new SocketServer(config.brokerId,
|
||||||
|
@ -138,6 +139,50 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
||||||
def awaitShutdown(): Unit = shutdownLatch.await()
|
def awaitShutdown(): Unit = shutdownLatch.await()
|
||||||
|
|
||||||
def getLogManager(): LogManager = logManager
|
def getLogManager(): LogManager = logManager
|
||||||
|
|
||||||
|
private def createLogManager(config: KafkaConfig): LogManager = {
|
||||||
|
val topics = config.logCleanupPolicyMap.keys ++
|
||||||
|
config.logSegmentBytesPerTopicMap.keys ++
|
||||||
|
config.logFlushIntervalMsPerTopicMap.keys ++
|
||||||
|
config.logRollHoursPerTopicMap.keys ++
|
||||||
|
config.logRetentionBytesPerTopicMap.keys ++
|
||||||
|
config.logRetentionHoursPerTopicMap.keys
|
||||||
|
val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes,
|
||||||
|
segmentMs = 60 * 60 * 1000 * config.logRollHours,
|
||||||
|
flushInterval = config.logFlushIntervalMessages,
|
||||||
|
flushMs = config.logFlushIntervalMs.toLong,
|
||||||
|
retentionSize = config.logRetentionBytes,
|
||||||
|
retentionMs = 60 * 60 * 1000 * config.logRetentionHours,
|
||||||
|
maxMessageSize = config.messageMaxBytes,
|
||||||
|
maxIndexSize = config.logIndexSizeMaxBytes,
|
||||||
|
indexInterval = config.logIndexIntervalBytes,
|
||||||
|
fileDeleteDelayMs = config.logDeleteDelayMs,
|
||||||
|
minCleanableRatio = config.logCleanerMinCleanRatio,
|
||||||
|
dedupe = config.logCleanupPolicy.trim.toLowerCase == "dedupe")
|
||||||
|
val logConfigs = for(topic <- topics) yield
|
||||||
|
topic -> defaultLogConfig.copy(segmentSize = config.logSegmentBytesPerTopicMap.getOrElse(topic, config.logSegmentBytes),
|
||||||
|
segmentMs = 60 * 60 * 1000 * config.logRollHoursPerTopicMap.getOrElse(topic, config.logRollHours),
|
||||||
|
flushMs = config.logFlushIntervalMsPerTopicMap.getOrElse(topic, config.logFlushIntervalMs).toLong,
|
||||||
|
retentionSize = config.logRetentionBytesPerTopicMap.getOrElse(topic, config.logRetentionBytes),
|
||||||
|
retentionMs = 60 * 60 * 1000 * config.logRetentionHoursPerTopicMap.getOrElse(topic, config.logRetentionHours),
|
||||||
|
dedupe = config.logCleanupPolicyMap.getOrElse(topic, config.logCleanupPolicy).trim.toLowerCase == "dedupe")
|
||||||
|
val cleanerConfig = CleanerConfig(numThreads = config.logCleanerThreads,
|
||||||
|
dedupeBufferSize = config.logCleanerDedupeBufferSize,
|
||||||
|
ioBufferSize = config.logCleanerIoBufferSize,
|
||||||
|
maxMessageSize = config.messageMaxBytes,
|
||||||
|
maxIoBytesPerSecond = config.logCleanerIoMaxBytesPerSecond,
|
||||||
|
backOffMs = config.logCleanerBackoffMs,
|
||||||
|
enableCleaner = config.logCleanerEnable)
|
||||||
|
new LogManager(logDirs = config.logDirs.map(new File(_)).toArray,
|
||||||
|
topicConfigs = logConfigs.toMap,
|
||||||
|
defaultConfig = defaultLogConfig,
|
||||||
|
cleanerConfig = cleanerConfig,
|
||||||
|
flushCheckMs = config.logFlushSchedulerIntervalMs,
|
||||||
|
retentionCheckMs = config.logCleanupIntervalMs,
|
||||||
|
scheduler = kafkaScheduler,
|
||||||
|
time = time)
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,103 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package kafka.server
|
||||||
|
|
||||||
|
import scala.collection._
|
||||||
|
import kafka.utils.Logging
|
||||||
|
import kafka.common._
|
||||||
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
|
import java.io._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class saves out a map of topic/partition=>offsets to a file
|
||||||
|
*/
|
||||||
|
class OffsetCheckpoint(val file: File) extends Logging {
|
||||||
|
private val lock = new Object()
|
||||||
|
new File(file + ".tmp").delete() // try to delete any existing temp files for cleanliness
|
||||||
|
file.createNewFile() // in case the file doesn't exist
|
||||||
|
|
||||||
|
def write(offsets: Map[TopicAndPartition, Long]) {
|
||||||
|
lock synchronized {
|
||||||
|
// write to temp file and then swap with the existing file
|
||||||
|
val temp = new File(file.getAbsolutePath + ".tmp")
|
||||||
|
|
||||||
|
val writer = new BufferedWriter(new FileWriter(temp))
|
||||||
|
try {
|
||||||
|
// write the current version
|
||||||
|
writer.write(0.toString)
|
||||||
|
writer.newLine()
|
||||||
|
|
||||||
|
// write the number of entries
|
||||||
|
writer.write(offsets.size.toString)
|
||||||
|
writer.newLine()
|
||||||
|
|
||||||
|
// write the entries
|
||||||
|
offsets.foreach { case (topicPart, offset) =>
|
||||||
|
writer.write("%s %d %d".format(topicPart.topic, topicPart.partition, offset))
|
||||||
|
writer.newLine()
|
||||||
|
}
|
||||||
|
|
||||||
|
// flush and overwrite old file
|
||||||
|
writer.flush()
|
||||||
|
if(!temp.renameTo(file))
|
||||||
|
throw new IOException("File rename from %s to %s failed.".format(temp.getAbsolutePath, file.getAbsolutePath))
|
||||||
|
} finally {
|
||||||
|
writer.close()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def read(): Map[TopicAndPartition, Long] = {
|
||||||
|
lock synchronized {
|
||||||
|
val reader = new BufferedReader(new FileReader(file))
|
||||||
|
try {
|
||||||
|
var line = reader.readLine()
|
||||||
|
if(line == null)
|
||||||
|
return Map.empty
|
||||||
|
val version = line.toInt
|
||||||
|
version match {
|
||||||
|
case 0 =>
|
||||||
|
line = reader.readLine()
|
||||||
|
if(line == null)
|
||||||
|
return Map.empty
|
||||||
|
val expectedSize = line.toInt
|
||||||
|
var offsets = Map[TopicAndPartition, Long]()
|
||||||
|
line = reader.readLine()
|
||||||
|
while(line != null) {
|
||||||
|
val pieces = line.split("\\s+")
|
||||||
|
if(pieces.length != 3)
|
||||||
|
throw new IOException("Malformed line in offset checkpoint file: '%s'.".format(line))
|
||||||
|
|
||||||
|
val topic = pieces(0)
|
||||||
|
val partition = pieces(1).toInt
|
||||||
|
val offset = pieces(2).toLong
|
||||||
|
offsets += (TopicAndPartition(pieces(0), partition) -> offset)
|
||||||
|
line = reader.readLine()
|
||||||
|
}
|
||||||
|
if(offsets.size != expectedSize)
|
||||||
|
throw new IOException("Expected %d entries but found only %d".format(expectedSize, offsets.size))
|
||||||
|
offsets
|
||||||
|
case _ =>
|
||||||
|
throw new IOException("Unrecognized version of the highwatermark checkpoint file: " + version)
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
reader.close()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -20,6 +20,7 @@ import kafka.cluster.{Broker, Partition, Replica}
|
||||||
import collection._
|
import collection._
|
||||||
import mutable.HashMap
|
import mutable.HashMap
|
||||||
import org.I0Itec.zkclient.ZkClient
|
import org.I0Itec.zkclient.ZkClient
|
||||||
|
import java.io.{File, IOException}
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import kafka.log.LogManager
|
import kafka.log.LogManager
|
||||||
|
@ -33,6 +34,7 @@ import kafka.controller.KafkaController
|
||||||
|
|
||||||
object ReplicaManager {
|
object ReplicaManager {
|
||||||
val UnknownLogEndOffset = -1L
|
val UnknownLogEndOffset = -1L
|
||||||
|
val HighWatermarkFilename = "replication-offset-checkpoint"
|
||||||
}
|
}
|
||||||
|
|
||||||
class ReplicaManager(val config: KafkaConfig,
|
class ReplicaManager(val config: KafkaConfig,
|
||||||
|
@ -48,7 +50,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
val replicaFetcherManager = new ReplicaFetcherManager(config, this)
|
val replicaFetcherManager = new ReplicaFetcherManager(config, this)
|
||||||
this.logIdent = "Replica Manager on Broker " + config.brokerId + ": "
|
this.logIdent = "Replica Manager on Broker " + config.brokerId + ": "
|
||||||
private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false)
|
private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false)
|
||||||
val highWatermarkCheckpoints = config.logDirs.map(dir => (dir, new HighwaterMarkCheckpoint(dir))).toMap
|
val highWatermarkCheckpoints = config.logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap
|
||||||
|
|
||||||
newGauge(
|
newGauge(
|
||||||
"LeaderCount",
|
"LeaderCount",
|
||||||
|
@ -67,7 +69,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
val isrExpandRate = newMeter("IsrExpandsPerSec", "expands", TimeUnit.SECONDS)
|
val isrExpandRate = newMeter("IsrExpandsPerSec", "expands", TimeUnit.SECONDS)
|
||||||
val isrShrinkRate = newMeter("ISRShrinksPerSec", "shrinks", TimeUnit.SECONDS)
|
val isrShrinkRate = newMeter("IsrShrinksPerSec", "shrinks", TimeUnit.SECONDS)
|
||||||
|
|
||||||
|
|
||||||
def startHighWaterMarksCheckPointThread() = {
|
def startHighWaterMarksCheckPointThread() = {
|
||||||
|
@ -265,7 +267,13 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParent)
|
val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParent)
|
||||||
for((dir, reps) <- replicasByDir) {
|
for((dir, reps) <- replicasByDir) {
|
||||||
val hwms = reps.map(r => (TopicAndPartition(r.topic, r.partitionId) -> r.highWatermark)).toMap
|
val hwms = reps.map(r => (TopicAndPartition(r.topic, r.partitionId) -> r.highWatermark)).toMap
|
||||||
highWatermarkCheckpoints(dir).write(hwms)
|
try {
|
||||||
|
highWatermarkCheckpoints(dir).write(hwms)
|
||||||
|
} catch {
|
||||||
|
case e: IOException =>
|
||||||
|
fatal("Error writing to highwatermark file: ", e)
|
||||||
|
Runtime.getRuntime().halt(1)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -9,56 +9,56 @@ import java.nio.channels._
|
||||||
* The given path will be created and opened if it doesn't exist.
|
* The given path will be created and opened if it doesn't exist.
|
||||||
*/
|
*/
|
||||||
class FileLock(val file: File) extends Logging {
|
class FileLock(val file: File) extends Logging {
|
||||||
file.createNewFile()
|
file.createNewFile() // create the file if it doesn't exist
|
||||||
private val channel = new RandomAccessFile(file, "rw").getChannel()
|
private val channel = new RandomAccessFile(file, "rw").getChannel()
|
||||||
private var flock: java.nio.channels.FileLock = null
|
private var flock: java.nio.channels.FileLock = null
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Lock the file or throw an exception if the lock is already held
|
* Lock the file or throw an exception if the lock is already held
|
||||||
*/
|
*/
|
||||||
def lock() {
|
def lock() {
|
||||||
this synchronized {
|
this synchronized {
|
||||||
trace("Acquiring lock on " + file.getAbsolutePath)
|
trace("Acquiring lock on " + file.getAbsolutePath)
|
||||||
flock = channel.lock()
|
flock = channel.lock()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Try to lock the file and return true if the locking succeeds
|
||||||
|
*/
|
||||||
|
def tryLock(): Boolean = {
|
||||||
|
this synchronized {
|
||||||
|
trace("Acquiring lock on " + file.getAbsolutePath)
|
||||||
|
try {
|
||||||
|
// weirdly this method will return null if the lock is held by another
|
||||||
|
// process, but will throw an exception if the lock is held by this process
|
||||||
|
// so we have to handle both cases
|
||||||
|
flock = channel.tryLock()
|
||||||
|
flock != null
|
||||||
|
} catch {
|
||||||
|
case e: OverlappingFileLockException => false
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Try to lock the file and return true if the locking succeeds
|
* Unlock the lock if it is held
|
||||||
*/
|
*/
|
||||||
def tryLock(): Boolean = {
|
def unlock() {
|
||||||
this synchronized {
|
this synchronized {
|
||||||
trace("Acquiring lock on " + file.getAbsolutePath)
|
trace("Releasing lock on " + file.getAbsolutePath)
|
||||||
try {
|
if(flock != null)
|
||||||
// weirdly this method will return null if the lock is held by another
|
flock.release()
|
||||||
// process, but will throw an exception if the lock is held by this process
|
|
||||||
// so we have to handle both cases
|
|
||||||
flock = channel.tryLock()
|
|
||||||
flock != null
|
|
||||||
} catch {
|
|
||||||
case e: OverlappingFileLockException => false
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Unlock the lock if it is held
|
* Destroy this lock, closing the associated FileChannel
|
||||||
*/
|
*/
|
||||||
def unlock() {
|
def destroy() = {
|
||||||
this synchronized {
|
this synchronized {
|
||||||
trace("Releasing lock on " + file.getAbsolutePath)
|
unlock()
|
||||||
if(flock != null)
|
channel.close()
|
||||||
flock.release()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Destroy this lock, closing the associated FileChannel
|
|
||||||
*/
|
|
||||||
def destroy() = {
|
|
||||||
this synchronized {
|
|
||||||
unlock()
|
|
||||||
channel.close()
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -23,12 +23,13 @@ trait Logging {
|
||||||
val loggerName = this.getClass.getName
|
val loggerName = this.getClass.getName
|
||||||
lazy val logger = Logger.getLogger(loggerName)
|
lazy val logger = Logger.getLogger(loggerName)
|
||||||
|
|
||||||
protected var logIdent = ""
|
protected var logIdent: String = null
|
||||||
|
|
||||||
// Force initialization to register Log4jControllerMBean
|
// Force initialization to register Log4jControllerMBean
|
||||||
private val log4jController = Log4jController
|
private val log4jController = Log4jController
|
||||||
|
|
||||||
private def msgWithLogIdent(msg: String) = logIdent + msg
|
private def msgWithLogIdent(msg: String) =
|
||||||
|
if(logIdent == null) msg else logIdent + msg
|
||||||
|
|
||||||
def trace(msg: => String): Unit = {
|
def trace(msg: => String): Unit = {
|
||||||
if (logger.isTraceEnabled())
|
if (logger.isTraceEnabled())
|
||||||
|
|
|
@ -32,20 +32,14 @@ import scala.math._
|
||||||
*/
|
*/
|
||||||
@threadsafe
|
@threadsafe
|
||||||
class Throttler(val desiredRatePerSec: Double,
|
class Throttler(val desiredRatePerSec: Double,
|
||||||
val checkIntervalMs: Long,
|
val checkIntervalMs: Long = 100L,
|
||||||
val throttleDown: Boolean,
|
val throttleDown: Boolean = true,
|
||||||
val time: Time) extends Logging {
|
val time: Time = SystemTime) extends Logging {
|
||||||
|
|
||||||
private val lock = new Object
|
private val lock = new Object
|
||||||
private var periodStartNs: Long = time.nanoseconds
|
private var periodStartNs: Long = time.nanoseconds
|
||||||
private var observedSoFar: Double = 0.0
|
private var observedSoFar: Double = 0.0
|
||||||
|
|
||||||
def this(desiredRatePerSec: Double, throttleDown: Boolean) =
|
|
||||||
this(desiredRatePerSec, Throttler.DefaultCheckIntervalMs, throttleDown, SystemTime)
|
|
||||||
|
|
||||||
def this(desiredRatePerSec: Double) =
|
|
||||||
this(desiredRatePerSec, Throttler.DefaultCheckIntervalMs, true, SystemTime)
|
|
||||||
|
|
||||||
def maybeThrottle(observed: Double) {
|
def maybeThrottle(observed: Double) {
|
||||||
lock synchronized {
|
lock synchronized {
|
||||||
observedSoFar += observed
|
observedSoFar += observed
|
||||||
|
@ -58,11 +52,11 @@ class Throttler(val desiredRatePerSec: Double,
|
||||||
val needAdjustment = !(throttleDown ^ (rateInSecs > desiredRatePerSec))
|
val needAdjustment = !(throttleDown ^ (rateInSecs > desiredRatePerSec))
|
||||||
if(needAdjustment) {
|
if(needAdjustment) {
|
||||||
// solve for the amount of time to sleep to make us hit the desired rate
|
// solve for the amount of time to sleep to make us hit the desired rate
|
||||||
val desiredRateMs = desiredRatePerSec / Time.MsPerSec.asInstanceOf[Double]
|
val desiredRateMs = desiredRatePerSec / Time.MsPerSec.toDouble
|
||||||
val elapsedMs = elapsedNs / Time.NsPerMs
|
val elapsedMs = elapsedNs / Time.NsPerMs
|
||||||
val sleepTime = round(observedSoFar / desiredRateMs - elapsedMs)
|
val sleepTime = round(observedSoFar / desiredRateMs - elapsedMs)
|
||||||
if(sleepTime > 0) {
|
if(sleepTime > 0) {
|
||||||
println("Natural rate is %f per second but desired rate is %f, sleeping for %d ms to compensate.".format(rateInSecs, desiredRatePerSec, sleepTime))
|
trace("Natural rate is %f per second but desired rate is %f, sleeping for %d ms to compensate.".format(rateInSecs, desiredRatePerSec, sleepTime))
|
||||||
time.sleep(sleepTime)
|
time.sleep(sleepTime)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -76,20 +70,20 @@ class Throttler(val desiredRatePerSec: Double,
|
||||||
|
|
||||||
object Throttler {
|
object Throttler {
|
||||||
|
|
||||||
val DefaultCheckIntervalMs = 100L
|
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val rand = new Random()
|
val rand = new Random()
|
||||||
val throttler = new Throttler(1000000, 100, true, SystemTime)
|
val throttler = new Throttler(100000, 100, true, SystemTime)
|
||||||
|
val interval = 30000
|
||||||
var start = System.currentTimeMillis
|
var start = System.currentTimeMillis
|
||||||
var total = 0
|
var total = 0
|
||||||
while(true) {
|
while(true) {
|
||||||
val value = rand.nextInt(1000)
|
val value = rand.nextInt(1000)
|
||||||
|
Thread.sleep(1)
|
||||||
throttler.maybeThrottle(value)
|
throttler.maybeThrottle(value)
|
||||||
total += value
|
total += value
|
||||||
val now = System.currentTimeMillis
|
val now = System.currentTimeMillis
|
||||||
if(now - start >= 1000) {
|
if(now - start >= interval) {
|
||||||
println(total)
|
println(total / (interval/1000.0))
|
||||||
start = now
|
start = now
|
||||||
total = 0
|
total = 0
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,7 @@ import scala.collection._
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import kafka.common.KafkaException
|
import kafka.common.KafkaException
|
||||||
|
import kafka.common.KafkaStorageException
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -159,7 +160,7 @@ object Utils extends Logging {
|
||||||
* @param log The log method to use for logging. E.g. logger.warn
|
* @param log The log method to use for logging. E.g. logger.warn
|
||||||
* @param action The action to execute
|
* @param action The action to execute
|
||||||
*/
|
*/
|
||||||
def swallow(log: (Object, Throwable) => Unit, action: => Unit) = {
|
def swallow(log: (Object, Throwable) => Unit, action: => Unit) {
|
||||||
try {
|
try {
|
||||||
action
|
action
|
||||||
} catch {
|
} catch {
|
||||||
|
@ -528,4 +529,37 @@ object Utils extends Logging {
|
||||||
*/
|
*/
|
||||||
def abs(n: Int) = n & 0x7fffffff
|
def abs(n: Int) = n & 0x7fffffff
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Replace the given string suffix with the new suffix. If the string doesn't end with the given suffix throw an exception.
|
||||||
|
*/
|
||||||
|
def replaceSuffix(s: String, oldSuffix: String, newSuffix: String): String = {
|
||||||
|
if(!s.endsWith(oldSuffix))
|
||||||
|
throw new IllegalArgumentException("Expected string to end with '%s' but string is '%s'".format(oldSuffix, s))
|
||||||
|
s.substring(0, s.length - oldSuffix.length) + newSuffix
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a file with the given path
|
||||||
|
* @param path The path to create
|
||||||
|
* @throw KafkaStorageException If the file create fails
|
||||||
|
* @return The created file
|
||||||
|
*/
|
||||||
|
def createFile(path: String): File = {
|
||||||
|
val f = new File(path)
|
||||||
|
val created = f.createNewFile()
|
||||||
|
if(!created)
|
||||||
|
throw new KafkaStorageException("Failed to create file %s.".format(path))
|
||||||
|
f
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read a big-endian integer from a byte array
|
||||||
|
*/
|
||||||
|
def readInt(bytes: Array[Byte], offset: Int): Int = {
|
||||||
|
((bytes(offset) & 0xFF) << 24) |
|
||||||
|
((bytes(offset + 1) & 0xFF) << 16) |
|
||||||
|
((bytes(offset + 2) & 0xFF) << 8) |
|
||||||
|
(bytes(offset + 3) & 0xFF)
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,10 +38,7 @@ class VerifiableProperties(val props: Properties) extends Logging {
|
||||||
/**
|
/**
|
||||||
* Read a required integer property value or throw an exception if no such property is found
|
* Read a required integer property value or throw an exception if no such property is found
|
||||||
*/
|
*/
|
||||||
def getInt(name: String): Int = {
|
def getInt(name: String): Int = getString(name).toInt
|
||||||
require(containsKey(name), "Missing required property '" + name + "'")
|
|
||||||
return getInt(name, -1)
|
|
||||||
}
|
|
||||||
|
|
||||||
def getIntInRange(name: String, range: (Int, Int)): Int = {
|
def getIntInRange(name: String, range: (Int, Int)): Int = {
|
||||||
require(containsKey(name), "Missing required property '" + name + "'")
|
require(containsKey(name), "Missing required property '" + name + "'")
|
||||||
|
@ -92,10 +89,7 @@ class VerifiableProperties(val props: Properties) extends Logging {
|
||||||
/**
|
/**
|
||||||
* Read a required long property value or throw an exception if no such property is found
|
* Read a required long property value or throw an exception if no such property is found
|
||||||
*/
|
*/
|
||||||
def getLong(name: String): Long = {
|
def getLong(name: String): Long = getString(name).toLong
|
||||||
require(containsKey(name), "Missing required property '" + name + "'")
|
|
||||||
return getLong(name, -1)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Read an long from the properties instance
|
* Read an long from the properties instance
|
||||||
|
@ -125,6 +119,26 @@ class VerifiableProperties(val props: Properties) extends Logging {
|
||||||
v
|
v
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a required argument as a double
|
||||||
|
* @param name The property name
|
||||||
|
* @return the value
|
||||||
|
* @throw IllegalArgumentException If the given property is not present
|
||||||
|
*/
|
||||||
|
def getDouble(name: String): Double = getString(name).toDouble
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an optional argument as a double
|
||||||
|
* @param name The property name
|
||||||
|
* @default The default value for the property if not present
|
||||||
|
*/
|
||||||
|
def getDouble(name: String, default: Double): Double = {
|
||||||
|
if(containsKey(name))
|
||||||
|
getDouble(name)
|
||||||
|
else
|
||||||
|
default
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Read a boolean value from the properties instance
|
* Read a boolean value from the properties instance
|
||||||
* @param name The property name
|
* @param name The property name
|
||||||
|
@ -141,6 +155,8 @@ class VerifiableProperties(val props: Properties) extends Logging {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def getBoolean(name: String) = getString(name).toBoolean
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a string property, or, if no such property is defined, return the given default value
|
* Get a string property, or, if no such property is defined, return the given default value
|
||||||
*/
|
*/
|
||||||
|
@ -162,7 +178,7 @@ class VerifiableProperties(val props: Properties) extends Logging {
|
||||||
/**
|
/**
|
||||||
* Get a Map[String, String] from a property list in the form k1:v2, k2:v2, ...
|
* Get a Map[String, String] from a property list in the form k1:v2, k2:v2, ...
|
||||||
*/
|
*/
|
||||||
def getMap(name: String, valid: String => Boolean): Map[String, String] = {
|
def getMap(name: String, valid: String => Boolean = s => true): Map[String, String] = {
|
||||||
try {
|
try {
|
||||||
val m = Utils.parseCsvMap(getString(name, ""))
|
val m = Utils.parseCsvMap(getString(name, ""))
|
||||||
m.foreach {
|
m.foreach {
|
||||||
|
@ -189,4 +205,5 @@ class VerifiableProperties(val props: Properties) extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
override def toString(): String = props.toString
|
override def toString(): String = props.toString
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,15 +34,11 @@ object StressTestLog {
|
||||||
val dir = TestUtils.tempDir()
|
val dir = TestUtils.tempDir()
|
||||||
val time = new MockTime
|
val time = new MockTime
|
||||||
val log = new Log(dir = dir,
|
val log = new Log(dir = dir,
|
||||||
scheduler = time.scheduler,
|
config = LogConfig(segmentSize = 64*1024*1024,
|
||||||
maxSegmentSize = 64*1024*1024,
|
maxMessageSize = Int.MaxValue,
|
||||||
maxMessageSize = Int.MaxValue,
|
maxIndexSize = 1024*1024),
|
||||||
flushInterval = Int.MaxValue,
|
|
||||||
rollIntervalMs = Long.MaxValue,
|
|
||||||
needsRecovery = false,
|
needsRecovery = false,
|
||||||
maxIndexSize = 1024*1024,
|
scheduler = time.scheduler,
|
||||||
indexIntervalBytes = 4096,
|
|
||||||
segmentDeleteDelayMs = 60000,
|
|
||||||
time = time)
|
time = time)
|
||||||
val writer = new WriterThread(log)
|
val writer = new WriterThread(log)
|
||||||
writer.start()
|
writer.start()
|
||||||
|
|
|
@ -0,0 +1,216 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka
|
||||||
|
|
||||||
|
import joptsimple.OptionParser
|
||||||
|
import java.util.Properties
|
||||||
|
import java.util.Random
|
||||||
|
import java.io._
|
||||||
|
import scala.io.Source
|
||||||
|
import scala.io.BufferedSource
|
||||||
|
import kafka.producer._
|
||||||
|
import kafka.consumer._
|
||||||
|
import kafka.serializer._
|
||||||
|
import kafka.utils._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is a torture test that runs against an existing broker. Here is how it works:
|
||||||
|
*
|
||||||
|
* It produces a series of specially formatted messages to one or more partitions. Each message it produces
|
||||||
|
* it logs out to a text file. The messages have a limited set of keys, so there is duplication in the key space.
|
||||||
|
*
|
||||||
|
* The broker will clean its log as the test runs.
|
||||||
|
*
|
||||||
|
* When the specified number of messages have been produced we create a consumer and consume all the messages in the topic
|
||||||
|
* and write that out to another text file.
|
||||||
|
*
|
||||||
|
* Using a stable unix sort we sort both the producer log of what was sent and the consumer log of what was retrieved by the message key.
|
||||||
|
* Then we compare the final message in both logs for each key. If this final message is not the same for all keys we
|
||||||
|
* print an error and exit with exit code 1, otherwise we print the size reduction and exit with exit code 0.
|
||||||
|
*/
|
||||||
|
object TestLogCleaning {
|
||||||
|
|
||||||
|
def main(args: Array[String]) {
|
||||||
|
val parser = new OptionParser
|
||||||
|
val numMessagesOpt = parser.accepts("messages", "The number of messages to send or consume.")
|
||||||
|
.withRequiredArg
|
||||||
|
.describedAs("count")
|
||||||
|
.ofType(classOf[java.lang.Long])
|
||||||
|
.defaultsTo(Long.MaxValue)
|
||||||
|
val numDupsOpt = parser.accepts("duplicates", "The number of duplicates for each key.")
|
||||||
|
.withRequiredArg
|
||||||
|
.describedAs("count")
|
||||||
|
.ofType(classOf[java.lang.Integer])
|
||||||
|
.defaultsTo(5)
|
||||||
|
val brokerOpt = parser.accepts("broker", "Url to connect to.")
|
||||||
|
.withRequiredArg
|
||||||
|
.describedAs("url")
|
||||||
|
.ofType(classOf[String])
|
||||||
|
val topicsOpt = parser.accepts("topics", "The number of topics to test.")
|
||||||
|
.withRequiredArg
|
||||||
|
.describedAs("count")
|
||||||
|
.ofType(classOf[java.lang.Integer])
|
||||||
|
.defaultsTo(1)
|
||||||
|
val zkConnectOpt = parser.accepts("zk", "Zk url.")
|
||||||
|
.withRequiredArg
|
||||||
|
.describedAs("url")
|
||||||
|
.ofType(classOf[String])
|
||||||
|
val sleepSecsOpt = parser.accepts("sleep", "Time to sleep between production and consumption.")
|
||||||
|
.withRequiredArg
|
||||||
|
.describedAs("ms")
|
||||||
|
.ofType(classOf[java.lang.Integer])
|
||||||
|
.defaultsTo(0)
|
||||||
|
val cleanupOpt = parser.accepts("cleanup", "Delete temp files when done.")
|
||||||
|
|
||||||
|
val options = parser.parse(args:_*)
|
||||||
|
|
||||||
|
if(!options.has(brokerOpt) || !options.has(zkConnectOpt) || !options.has(numMessagesOpt)) {
|
||||||
|
parser.printHelpOn(System.err)
|
||||||
|
System.exit(1)
|
||||||
|
}
|
||||||
|
|
||||||
|
// parse options
|
||||||
|
val messages = options.valueOf(numMessagesOpt).longValue
|
||||||
|
val dups = options.valueOf(numDupsOpt).intValue
|
||||||
|
val brokerUrl = options.valueOf(brokerOpt)
|
||||||
|
val topicCount = options.valueOf(topicsOpt).intValue
|
||||||
|
val zkUrl = options.valueOf(zkConnectOpt)
|
||||||
|
val sleepSecs = options.valueOf(sleepSecsOpt).intValue
|
||||||
|
val cleanup = options.has(cleanupOpt)
|
||||||
|
|
||||||
|
val testId = new Random().nextInt(Int.MaxValue)
|
||||||
|
val topics = (0 until topicCount).map("log-cleaner-test-" + testId + "-" + _).toArray
|
||||||
|
|
||||||
|
println("Producing %d messages...".format(messages))
|
||||||
|
val producedDataFile = produceMessages(brokerUrl, topics, messages, dups, cleanup)
|
||||||
|
println("Sleeping for %d seconds...".format(sleepSecs))
|
||||||
|
Thread.sleep(sleepSecs * 1000)
|
||||||
|
println("Consuming messages...")
|
||||||
|
val consumedDataFile = consumeMessages(zkUrl, topics, cleanup)
|
||||||
|
|
||||||
|
val producedLines = lineCount(producedDataFile)
|
||||||
|
val consumedLines = lineCount(consumedDataFile)
|
||||||
|
val reduction = 1.0 - consumedLines.toDouble/producedLines.toDouble
|
||||||
|
println("%d rows of data produced, %d rows of data consumed (%.1f%% reduction).".format(producedLines, consumedLines, 100 * reduction))
|
||||||
|
|
||||||
|
println("Validating output files...")
|
||||||
|
validateOutput(externalSort(producedDataFile), externalSort(consumedDataFile))
|
||||||
|
println("All done.")
|
||||||
|
}
|
||||||
|
|
||||||
|
def lineCount(file: File): Int = io.Source.fromFile(file).getLines.size
|
||||||
|
|
||||||
|
def validateOutput(produced: BufferedReader, consumed: BufferedReader) {
|
||||||
|
while(true) {
|
||||||
|
val prod = readFinalValue(produced)
|
||||||
|
val cons = readFinalValue(consumed)
|
||||||
|
if(prod == null && cons == null) {
|
||||||
|
return
|
||||||
|
} else if(prod != cons) {
|
||||||
|
System.err.println("Validation failed prod = %s, cons = %s!".format(prod, cons))
|
||||||
|
System.exit(1)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def readFinalValue(reader: BufferedReader): (String, Int, Int) = {
|
||||||
|
def readTuple() = {
|
||||||
|
val line = reader.readLine
|
||||||
|
if(line == null)
|
||||||
|
null
|
||||||
|
else
|
||||||
|
line.split("\t")
|
||||||
|
}
|
||||||
|
var prev = readTuple()
|
||||||
|
if(prev == null)
|
||||||
|
return null
|
||||||
|
while(true) {
|
||||||
|
reader.mark(1024)
|
||||||
|
val curr = readTuple()
|
||||||
|
if(curr == null || curr(0) != prev(0) || curr(1) != prev(1)) {
|
||||||
|
reader.reset()
|
||||||
|
return (prev(0), prev(1).toInt, prev(2).toInt)
|
||||||
|
} else {
|
||||||
|
prev = curr
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null
|
||||||
|
}
|
||||||
|
|
||||||
|
def externalSort(file: File): BufferedReader = {
|
||||||
|
val builder = new ProcessBuilder("sort", "--key=1,2", "--stable", "--buffer-size=20%", file.getAbsolutePath)
|
||||||
|
val process = builder.start()
|
||||||
|
new BufferedReader(new InputStreamReader(process.getInputStream()))
|
||||||
|
}
|
||||||
|
|
||||||
|
def produceMessages(brokerUrl: String,
|
||||||
|
topics: Array[String],
|
||||||
|
messages: Long,
|
||||||
|
dups: Int,
|
||||||
|
cleanup: Boolean): File = {
|
||||||
|
val producerProps = new Properties
|
||||||
|
producerProps.setProperty("producer.type", "async")
|
||||||
|
producerProps.setProperty("broker.list", brokerUrl)
|
||||||
|
producerProps.setProperty("serializer.class", classOf[StringEncoder].getName)
|
||||||
|
producerProps.setProperty("key.serializer.class", classOf[StringEncoder].getName)
|
||||||
|
producerProps.setProperty("queue.enqueue.timeout.ms", "-1")
|
||||||
|
producerProps.setProperty("batch.size", 1000.toString)
|
||||||
|
val producer = new Producer[String, String](new ProducerConfig(producerProps))
|
||||||
|
val rand = new Random(1)
|
||||||
|
val keyCount = (messages / dups).toInt
|
||||||
|
val producedFile = File.createTempFile("kafka-log-cleaner-produced-", ".txt")
|
||||||
|
if(cleanup)
|
||||||
|
producedFile.deleteOnExit()
|
||||||
|
val producedWriter = new BufferedWriter(new FileWriter(producedFile), 1024*1024)
|
||||||
|
for(i <- 0L until (messages * topics.length)) {
|
||||||
|
val topic = topics((i % topics.length).toInt)
|
||||||
|
val key = rand.nextInt(keyCount)
|
||||||
|
producer.send(KeyedMessage(topic = topic, key = key.toString, message = i.toString))
|
||||||
|
producedWriter.write("%s\t%s\t%s\n".format(topic, key, i))
|
||||||
|
}
|
||||||
|
producedWriter.close()
|
||||||
|
producer.close()
|
||||||
|
producedFile
|
||||||
|
}
|
||||||
|
|
||||||
|
def consumeMessages(zkUrl: String, topics: Array[String], cleanup: Boolean): File = {
|
||||||
|
val consumerProps = new Properties
|
||||||
|
consumerProps.setProperty("group.id", "log-cleaner-test-" + new Random().nextInt(Int.MaxValue))
|
||||||
|
consumerProps.setProperty("zk.connect", zkUrl)
|
||||||
|
consumerProps.setProperty("consumer.timeout.ms", (5*1000).toString)
|
||||||
|
val connector = new ZookeeperConsumerConnector(new ConsumerConfig(consumerProps))
|
||||||
|
val streams = connector.createMessageStreams(topics.map(topic => (topic, 1)).toMap, new StringDecoder, new StringDecoder)
|
||||||
|
val consumedFile = File.createTempFile("kafka-log-cleaner-consumed-", ".txt")
|
||||||
|
if(cleanup)
|
||||||
|
consumedFile.deleteOnExit()
|
||||||
|
val consumedWriter = new BufferedWriter(new FileWriter(consumedFile))
|
||||||
|
for(topic <- topics) {
|
||||||
|
val stream = streams(topic).head
|
||||||
|
try {
|
||||||
|
for(item <- stream)
|
||||||
|
consumedWriter.write("%s\t%s\t%s\n".format(topic, item.key, item.message))
|
||||||
|
} catch {
|
||||||
|
case e: ConsumerTimeoutException =>
|
||||||
|
}
|
||||||
|
}
|
||||||
|
consumedWriter.close()
|
||||||
|
connector.shutdown()
|
||||||
|
consumedFile
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -34,7 +34,8 @@ object TestLogPerformance {
|
||||||
val config = new KafkaConfig(props)
|
val config = new KafkaConfig(props)
|
||||||
val dir = TestUtils.tempDir()
|
val dir = TestUtils.tempDir()
|
||||||
val scheduler = new KafkaScheduler(1)
|
val scheduler = new KafkaScheduler(1)
|
||||||
val log = new Log(dir, scheduler, 50*1024*1024, config.messageMaxBytes, 5000000, config.logRollHours*60*60*1000L, needsRecovery = false, segmentDeleteDelayMs = 0, time = SystemTime)
|
val logConfig = LogConfig()
|
||||||
|
val log = new Log(dir, logConfig, needsRecovery = false, scheduler = scheduler, time = SystemTime)
|
||||||
val bytes = new Array[Byte](messageSize)
|
val bytes = new Array[Byte](messageSize)
|
||||||
new java.util.Random().nextBytes(bytes)
|
new java.util.Random().nextBytes(bytes)
|
||||||
val message = new Message(bytes)
|
val message = new Message(bytes)
|
||||||
|
|
|
@ -0,0 +1,227 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka.log
|
||||||
|
|
||||||
|
import junit.framework.Assert._
|
||||||
|
import org.scalatest.junit.JUnitSuite
|
||||||
|
import org.junit.{After, Before, Test}
|
||||||
|
import java.nio._
|
||||||
|
import java.io.File
|
||||||
|
import scala.collection._
|
||||||
|
import kafka.common._
|
||||||
|
import kafka.utils._
|
||||||
|
import kafka.message._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unit tests for the log cleaning logic
|
||||||
|
*/
|
||||||
|
class CleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
|
val dir = TestUtils.tempDir()
|
||||||
|
val logConfig = LogConfig(segmentSize=1024, maxIndexSize=1024, dedupe=true)
|
||||||
|
val time = new MockTime()
|
||||||
|
val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time)
|
||||||
|
|
||||||
|
@After
|
||||||
|
def teardown() {
|
||||||
|
Utils.rm(dir)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test simple log cleaning
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
def testCleanSegments() {
|
||||||
|
val cleaner = makeCleaner(Int.MaxValue)
|
||||||
|
val log = makeLog(config = logConfig.copy(segmentSize = 1024))
|
||||||
|
|
||||||
|
// append messages to the log until we have four segments
|
||||||
|
while(log.numberOfSegments < 4)
|
||||||
|
log.append(messages(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
|
val keysFound = keysInLog(log)
|
||||||
|
assertEquals((0L until log.logEndOffset), keysFound)
|
||||||
|
|
||||||
|
// pretend we have the following keys
|
||||||
|
val keys = immutable.ListSet(1, 3, 5, 7, 9)
|
||||||
|
val map = new FakeOffsetMap(Int.MaxValue)
|
||||||
|
keys.foreach(k => map.put(key(k), Long.MaxValue))
|
||||||
|
|
||||||
|
// clean the log
|
||||||
|
cleaner.cleanSegments(log, log.logSegments.take(3).toSeq, map, 0)
|
||||||
|
val shouldRemain = keysInLog(log).filter(!keys.contains(_))
|
||||||
|
assertEquals(shouldRemain, keysInLog(log))
|
||||||
|
}
|
||||||
|
|
||||||
|
/* extract all the keys from a log */
|
||||||
|
def keysInLog(log: Log): Iterable[Int] =
|
||||||
|
log.logSegments.flatMap(s => s.log.map(m => Utils.readString(m.message.key).toInt))
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that a truncation during cleaning throws an OptimisticLockFailureException
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
def testCleanSegmentsWithTruncation() {
|
||||||
|
val cleaner = makeCleaner(Int.MaxValue)
|
||||||
|
val log = makeLog(config = logConfig.copy(segmentSize = 1024))
|
||||||
|
|
||||||
|
// append messages to the log until we have four segments
|
||||||
|
while(log.numberOfSegments < 2)
|
||||||
|
log.append(messages(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
|
|
||||||
|
log.truncateTo(log.logEndOffset-2)
|
||||||
|
val keys = keysInLog(log)
|
||||||
|
val map = new FakeOffsetMap(Int.MaxValue)
|
||||||
|
keys.foreach(k => map.put(key(k), Long.MaxValue))
|
||||||
|
intercept[OptimisticLockFailureException] {
|
||||||
|
cleaner.cleanSegments(log, log.logSegments.take(3).toSeq, map, 0)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Validate the logic for grouping log segments together for cleaning
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
def testSegmentGrouping() {
|
||||||
|
val cleaner = makeCleaner(Int.MaxValue)
|
||||||
|
val log = makeLog(config = logConfig.copy(segmentSize = 300, indexInterval = 1))
|
||||||
|
|
||||||
|
// append some messages to the log
|
||||||
|
var i = 0
|
||||||
|
while(log.numberOfSegments < 10) {
|
||||||
|
log.append(TestUtils.singleMessageSet("hello".getBytes))
|
||||||
|
i += 1
|
||||||
|
}
|
||||||
|
|
||||||
|
// grouping by very large values should result in a single group with all the segments in it
|
||||||
|
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
||||||
|
assertEquals(1, groups.size)
|
||||||
|
assertEquals(log.numberOfSegments, groups(0).size)
|
||||||
|
checkSegmentOrder(groups)
|
||||||
|
|
||||||
|
// grouping by very small values should result in all groups having one entry
|
||||||
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue)
|
||||||
|
assertEquals(log.numberOfSegments, groups.size)
|
||||||
|
assertTrue("All groups should be singletons.", groups.forall(_.size == 1))
|
||||||
|
checkSegmentOrder(groups)
|
||||||
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = 1)
|
||||||
|
assertEquals(log.numberOfSegments, groups.size)
|
||||||
|
assertTrue("All groups should be singletons.", groups.forall(_.size == 1))
|
||||||
|
checkSegmentOrder(groups)
|
||||||
|
|
||||||
|
val groupSize = 3
|
||||||
|
|
||||||
|
// check grouping by log size
|
||||||
|
val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1
|
||||||
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue)
|
||||||
|
checkSegmentOrder(groups)
|
||||||
|
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
|
||||||
|
|
||||||
|
// check grouping by index size
|
||||||
|
val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes()).sum + 1
|
||||||
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize)
|
||||||
|
checkSegmentOrder(groups)
|
||||||
|
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]) {
|
||||||
|
val offsets = groups.flatMap(_.map(_.baseOffset))
|
||||||
|
assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test building an offset map off the log
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
def testBuildOffsetMap() {
|
||||||
|
val map = new FakeOffsetMap(1000)
|
||||||
|
val log = makeLog()
|
||||||
|
val cleaner = makeCleaner(Int.MaxValue)
|
||||||
|
val start = 0
|
||||||
|
val end = 500
|
||||||
|
val offsets = writeToLog(log, (start until end) zip (start until end))
|
||||||
|
def checkRange(map: FakeOffsetMap, start: Int, end: Int) {
|
||||||
|
val endOffset = cleaner.buildOffsetMap(log, start, end, map) + 1
|
||||||
|
assertEquals("Last offset should be the end offset.", end, endOffset)
|
||||||
|
assertEquals("Should have the expected number of messages in the map.", end-start, map.size)
|
||||||
|
for(i <- start until end)
|
||||||
|
assertEquals("Should find all the keys", i.toLong, map.get(key(i)))
|
||||||
|
assertEquals("Should not find a value too small", -1L, map.get(key(start-1)))
|
||||||
|
assertEquals("Should not find a value too large", -1L, map.get(key(end)))
|
||||||
|
}
|
||||||
|
val segments = log.logSegments.toSeq
|
||||||
|
checkRange(map, 0, segments(1).baseOffset.toInt)
|
||||||
|
checkRange(map, segments(1).baseOffset.toInt, segments(3).baseOffset.toInt)
|
||||||
|
checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that we don't exceed the maximum capacity of the offset map, that is that an offset map
|
||||||
|
* with a max size of 1000 will only clean 1000 new entries even if more than that are available.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
def testBuildOffsetMapOverCapacity() {
|
||||||
|
val map = new FakeOffsetMap(1000)
|
||||||
|
val log = makeLog()
|
||||||
|
val cleaner = makeCleaner(Int.MaxValue)
|
||||||
|
val vals = 0 until 1001
|
||||||
|
val offsets = writeToLog(log, vals zip vals)
|
||||||
|
val lastOffset = cleaner.buildOffsetMap(log, vals.start, vals.end, map)
|
||||||
|
assertEquals("Shouldn't go beyond the capacity of the offset map.", 1000, lastOffset)
|
||||||
|
}
|
||||||
|
|
||||||
|
def makeLog(dir: File = dir, config: LogConfig = logConfig) =
|
||||||
|
new Log(dir = dir, config = config, needsRecovery = false, scheduler = time.scheduler, time = time)
|
||||||
|
|
||||||
|
def makeCleaner(capacity: Int) =
|
||||||
|
new Cleaner(id = 0, new FakeOffsetMap(capacity), ioBufferSize = 64*1024, maxIoBufferSize = 64*1024, throttler = throttler, time = time)
|
||||||
|
|
||||||
|
def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = {
|
||||||
|
for((key, value) <- seq)
|
||||||
|
yield log.append(messages(key, value)).firstOffset
|
||||||
|
}
|
||||||
|
|
||||||
|
def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes)
|
||||||
|
|
||||||
|
def messages(key: Int, value: Int) =
|
||||||
|
new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=value.toString.getBytes))
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
class FakeOffsetMap(val capacity: Int) extends OffsetMap {
|
||||||
|
val map = new java.util.HashMap[String, Long]()
|
||||||
|
|
||||||
|
private def keyFor(key: ByteBuffer) =
|
||||||
|
new String(Utils.readBytes(key.duplicate), "UTF-8")
|
||||||
|
|
||||||
|
def put(key: ByteBuffer, offset: Long): Unit =
|
||||||
|
map.put(keyFor(key), offset)
|
||||||
|
|
||||||
|
def get(key: ByteBuffer): Long = {
|
||||||
|
val k = keyFor(key)
|
||||||
|
if(map.containsKey(k))
|
||||||
|
map.get(k)
|
||||||
|
else
|
||||||
|
-1L
|
||||||
|
}
|
||||||
|
|
||||||
|
def clear() = map.clear()
|
||||||
|
|
||||||
|
def size: Int = map.size
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,117 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka.log
|
||||||
|
|
||||||
|
import java.io.File
|
||||||
|
import scala.collection._
|
||||||
|
import org.junit._
|
||||||
|
import kafka.common.TopicAndPartition
|
||||||
|
import kafka.utils._
|
||||||
|
import kafka.message._
|
||||||
|
import org.scalatest.junit.JUnitSuite
|
||||||
|
import junit.framework.Assert._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is an integration test that tests the fully integrated log cleaner
|
||||||
|
*/
|
||||||
|
class LogCleanerIntegrationTest extends JUnitSuite {
|
||||||
|
|
||||||
|
val time = new MockTime()
|
||||||
|
val segmentSize = 100
|
||||||
|
val deleteDelay = 1000
|
||||||
|
val logName = "log"
|
||||||
|
val logDir = TestUtils.tempDir()
|
||||||
|
var counter = 0
|
||||||
|
val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2))
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def cleanerTest() {
|
||||||
|
val cleaner = makeCleaner(parts = 3)
|
||||||
|
val log = cleaner.logs.get(topics(0))
|
||||||
|
|
||||||
|
val appends = writeDups(numKeys = 100, numDups = 3, log)
|
||||||
|
val startSize = log.size
|
||||||
|
cleaner.startup()
|
||||||
|
|
||||||
|
val lastCleaned = log.activeSegment.baseOffset
|
||||||
|
// wait until we clean up to base_offset of active segment - minDirtyMessages
|
||||||
|
cleaner.awaitCleaned("log", 0, lastCleaned)
|
||||||
|
|
||||||
|
val read = readFromLog(log)
|
||||||
|
assertEquals("Contents of the map shouldn't change.", appends.toMap, read.toMap)
|
||||||
|
assertTrue(startSize > log.size)
|
||||||
|
|
||||||
|
// write some more stuff and validate again
|
||||||
|
val appends2 = appends ++ writeDups(numKeys = 100, numDups = 3, log)
|
||||||
|
val lastCleaned2 = log.activeSegment.baseOffset
|
||||||
|
cleaner.awaitCleaned("log", 0, lastCleaned2)
|
||||||
|
val read2 = readFromLog(log)
|
||||||
|
assertEquals("Contents of the map shouldn't change.", appends2.toMap, read2.toMap)
|
||||||
|
|
||||||
|
cleaner.shutdown()
|
||||||
|
}
|
||||||
|
|
||||||
|
def readFromLog(log: Log): Iterable[(Int, Int)] = {
|
||||||
|
for(segment <- log.logSegments; message <- segment.log) yield {
|
||||||
|
val key = Utils.readString(message.message.key).toInt
|
||||||
|
val value = Utils.readString(message.message.payload).toInt
|
||||||
|
key -> value
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def writeDups(numKeys: Int, numDups: Int, log: Log): Seq[(Int, Int)] = {
|
||||||
|
for(dup <- 0 until numDups; key <- 0 until numKeys) yield {
|
||||||
|
val count = counter
|
||||||
|
val appendInfo = log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true)
|
||||||
|
counter += 1
|
||||||
|
(key, count)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
def teardown() {
|
||||||
|
Utils.rm(logDir)
|
||||||
|
}
|
||||||
|
|
||||||
|
/* create a cleaner instance and logs with the given parameters */
|
||||||
|
def makeCleaner(parts: Int,
|
||||||
|
minDirtyMessages: Int = 0,
|
||||||
|
numThreads: Int = 1,
|
||||||
|
defaultPolicy: String = "dedupe",
|
||||||
|
policyOverrides: Map[String, String] = Map()): LogCleaner = {
|
||||||
|
|
||||||
|
// create partitions and add them to the pool
|
||||||
|
val logs = new Pool[TopicAndPartition, Log]()
|
||||||
|
for(i <- 0 until parts) {
|
||||||
|
val dir = new File(logDir, "log-" + i)
|
||||||
|
dir.mkdirs()
|
||||||
|
val log = new Log(dir = dir,
|
||||||
|
LogConfig(segmentSize = segmentSize, maxIndexSize = 100*1024, fileDeleteDelayMs = deleteDelay, dedupe = true),
|
||||||
|
needsRecovery = false,
|
||||||
|
scheduler = time.scheduler,
|
||||||
|
time = time)
|
||||||
|
logs.put(TopicAndPartition("log", i), log)
|
||||||
|
}
|
||||||
|
|
||||||
|
new LogCleaner(CleanerConfig(numThreads = numThreads),
|
||||||
|
logDirs = Array(logDir),
|
||||||
|
logs = logs,
|
||||||
|
time = time)
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -29,21 +29,18 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
|
|
||||||
val time: MockTime = new MockTime()
|
val time: MockTime = new MockTime()
|
||||||
val maxRollInterval = 100
|
val maxRollInterval = 100
|
||||||
val maxLogAgeHours = 10
|
val maxLogAgeMs = 10*60*60*1000
|
||||||
|
val logConfig = LogConfig(segmentSize = 1024, maxIndexSize = 4096, retentionMs = maxLogAgeMs)
|
||||||
var logDir: File = null
|
var logDir: File = null
|
||||||
var logManager: LogManager = null
|
var logManager: LogManager = null
|
||||||
var config: KafkaConfig = null
|
|
||||||
val name = "kafka"
|
val name = "kafka"
|
||||||
val veryLargeLogFlushInterval = 10000000L
|
val veryLargeLogFlushInterval = 10000000L
|
||||||
|
val cleanerConfig = CleanerConfig(enableCleaner = false)
|
||||||
|
|
||||||
override def setUp() {
|
override def setUp() {
|
||||||
super.setUp()
|
super.setUp()
|
||||||
config = new KafkaConfig(TestUtils.createBrokerConfig(0, -1)) {
|
logDir = TestUtils.tempDir()
|
||||||
override val logSegmentBytes = 1024
|
logManager = new LogManager(Array(logDir), Map(), logConfig, cleanerConfig, 1000L, 1000L, time.scheduler, time)
|
||||||
override val logFlushIntervalMessages = 10000
|
|
||||||
override val logRetentionHours = maxLogAgeHours
|
|
||||||
}
|
|
||||||
logManager = new LogManager(config, time.scheduler, time)
|
|
||||||
logManager.startup
|
logManager.startup
|
||||||
logDir = logManager.logDirs(0)
|
logDir = logManager.logDirs(0)
|
||||||
}
|
}
|
||||||
|
@ -62,7 +59,7 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
@Test
|
@Test
|
||||||
def testCreateLog() {
|
def testCreateLog() {
|
||||||
val log = logManager.getOrCreateLog(name, 0)
|
val log = logManager.getOrCreateLog(name, 0)
|
||||||
val logFile = new File(config.logDirs(0), name + "-0")
|
val logFile = new File(logDir, name + "-0")
|
||||||
assertTrue(logFile.exists)
|
assertTrue(logFile.exists)
|
||||||
log.append(TestUtils.singleMessageSet("test".getBytes()))
|
log.append(TestUtils.singleMessageSet("test".getBytes()))
|
||||||
}
|
}
|
||||||
|
@ -74,7 +71,7 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
def testGetNonExistentLog() {
|
def testGetNonExistentLog() {
|
||||||
val log = logManager.getLog(name, 0)
|
val log = logManager.getLog(name, 0)
|
||||||
assertEquals("No log should be found.", None, log)
|
assertEquals("No log should be found.", None, log)
|
||||||
val logFile = new File(config.logDirs(0), name + "-0")
|
val logFile = new File(logDir, name + "-0")
|
||||||
assertTrue(!logFile.exists)
|
assertTrue(!logFile.exists)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -94,9 +91,9 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
|
|
||||||
log.logSegments.foreach(_.log.file.setLastModified(time.milliseconds))
|
log.logSegments.foreach(_.log.file.setLastModified(time.milliseconds))
|
||||||
|
|
||||||
time.sleep(maxLogAgeHours*60*60*1000 + 1)
|
time.sleep(maxLogAgeMs + 1)
|
||||||
assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
|
assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
|
||||||
time.sleep(log.segmentDeleteDelayMs + 1)
|
time.sleep(log.config.fileDeleteDelayMs + 1)
|
||||||
assertEquals("Files should have been deleted", log.numberOfSegments * 2, log.dir.list.length)
|
assertEquals("Files should have been deleted", log.numberOfSegments * 2, log.dir.list.length)
|
||||||
assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).sizeInBytes)
|
assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).sizeInBytes)
|
||||||
|
|
||||||
|
@ -116,14 +113,10 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
@Test
|
@Test
|
||||||
def testCleanupSegmentsToMaintainSize() {
|
def testCleanupSegmentsToMaintainSize() {
|
||||||
val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes
|
val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes
|
||||||
val props = TestUtils.createBrokerConfig(0, -1)
|
|
||||||
logManager.shutdown()
|
logManager.shutdown()
|
||||||
config = new KafkaConfig(props) {
|
|
||||||
override val logSegmentBytes = (10 * (setSize - 1)) // each segment will be 10 messages
|
val config = logConfig.copy(segmentSize = 10 * (setSize - 1), retentionSize = 5L * 10L * setSize + 10L)
|
||||||
override val logRetentionBytes = (5 * 10 * setSize + 10).asInstanceOf[Long]
|
logManager = new LogManager(Array(logDir), Map(), config, cleanerConfig, 1000L, 1000L, time.scheduler, time)
|
||||||
override val logRollHours = maxRollInterval
|
|
||||||
}
|
|
||||||
logManager = new LogManager(config, time.scheduler, time)
|
|
||||||
logManager.startup
|
logManager.startup
|
||||||
|
|
||||||
// create a log
|
// create a log
|
||||||
|
@ -138,13 +131,12 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
offset = info.firstOffset
|
offset = info.firstOffset
|
||||||
}
|
}
|
||||||
|
|
||||||
// should be exactly 100 full segments + 1 new empty one
|
assertEquals("Check we have the expected number of segments.", numMessages * setSize / config.segmentSize, 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
|
// this cleanup shouldn't find any expired segments but should delete some to reduce size
|
||||||
time.sleep(logManager.InitialTaskDelayMs)
|
time.sleep(logManager.InitialTaskDelayMs)
|
||||||
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
|
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
|
||||||
time.sleep(log.segmentDeleteDelayMs + 1)
|
time.sleep(log.config.fileDeleteDelayMs + 1)
|
||||||
assertEquals("Files should have been deleted", log.numberOfSegments * 2, log.dir.list.length)
|
assertEquals("Files should have been deleted", log.numberOfSegments * 2, log.dir.list.length)
|
||||||
assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).sizeInBytes)
|
assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).sizeInBytes)
|
||||||
try {
|
try {
|
||||||
|
@ -162,14 +154,9 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testTimeBasedFlush() {
|
def testTimeBasedFlush() {
|
||||||
val props = TestUtils.createBrokerConfig(0, -1)
|
|
||||||
logManager.shutdown()
|
logManager.shutdown()
|
||||||
config = new KafkaConfig(props) {
|
val config = logConfig.copy(flushMs = 1000)
|
||||||
override val logFlushSchedulerIntervalMs = 1000
|
logManager = new LogManager(Array(logDir), Map(), config, cleanerConfig, 1000L, 1000L, time.scheduler, time)
|
||||||
override val logFlushIntervalMs = 1000
|
|
||||||
override val logFlushIntervalMessages = Int.MaxValue
|
|
||||||
}
|
|
||||||
logManager = new LogManager(config, time.scheduler, time)
|
|
||||||
logManager.startup
|
logManager.startup
|
||||||
val log = logManager.getOrCreateLog(name, 0)
|
val log = logManager.getOrCreateLog(name, 0)
|
||||||
val lastFlush = log.lastFlushTime
|
val lastFlush = log.lastFlushTime
|
||||||
|
@ -187,13 +174,11 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
@Test
|
@Test
|
||||||
def testLeastLoadedAssignment() {
|
def testLeastLoadedAssignment() {
|
||||||
// create a log manager with multiple data directories
|
// create a log manager with multiple data directories
|
||||||
val props = TestUtils.createBrokerConfig(0, -1)
|
val dirs = Array(TestUtils.tempDir(),
|
||||||
val dirs = Seq(TestUtils.tempDir().getAbsolutePath,
|
TestUtils.tempDir(),
|
||||||
TestUtils.tempDir().getAbsolutePath,
|
TestUtils.tempDir())
|
||||||
TestUtils.tempDir().getAbsolutePath)
|
|
||||||
props.put("log.dirs", dirs.mkString(","))
|
|
||||||
logManager.shutdown()
|
logManager.shutdown()
|
||||||
logManager = new LogManager(new KafkaConfig(props), time.scheduler, time)
|
logManager = new LogManager(dirs, Map(), logConfig, cleanerConfig, 1000L, 1000L, time.scheduler, time)
|
||||||
|
|
||||||
// verify that logs are always assigned to the least loaded partition
|
// verify that logs are always assigned to the least loaded partition
|
||||||
for(partition <- 0 until 20) {
|
for(partition <- 0 until 20) {
|
||||||
|
@ -209,7 +194,7 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
*/
|
*/
|
||||||
def testTwoLogManagersUsingSameDirFails() {
|
def testTwoLogManagersUsingSameDirFails() {
|
||||||
try {
|
try {
|
||||||
new LogManager(logManager.config, time.scheduler, time)
|
new LogManager(Array(logDir), Map(), logConfig, cleanerConfig, 1000L, 1000L, time.scheduler, time)
|
||||||
fail("Should not be able to create a second log manager instance with the same data directory")
|
fail("Should not be able to create a second log manager instance with the same data directory")
|
||||||
} catch {
|
} catch {
|
||||||
case e: KafkaException => // this is good
|
case e: KafkaException => // this is good
|
||||||
|
|
|
@ -160,6 +160,21 @@ class LogSegmentTest extends JUnit3Suite {
|
||||||
assertEquals(53, seg.nextOffset())
|
assertEquals(53, seg.nextOffset())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that we can change the file suffixes for the log and index files
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
def testChangeFileSuffixes() {
|
||||||
|
val seg = createSegment(40)
|
||||||
|
val logFile = seg.log.file
|
||||||
|
val indexFile = seg.index.file
|
||||||
|
seg.changeFileSuffixes("", ".deleted")
|
||||||
|
assertEquals(logFile.getAbsolutePath + ".deleted", seg.log.file.getAbsolutePath)
|
||||||
|
assertEquals(indexFile.getAbsolutePath + ".deleted", seg.index.file.getAbsolutePath)
|
||||||
|
assertTrue(seg.log.file.exists)
|
||||||
|
assertTrue(seg.index.file.exists)
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a segment with some data and an index. Then corrupt the index,
|
* Create a segment with some data and an index. Then corrupt the index,
|
||||||
* and recover the segment, the entries should all be readable.
|
* and recover the segment, the entries should all be readable.
|
||||||
|
|
|
@ -34,6 +34,7 @@ class LogTest extends JUnitSuite {
|
||||||
var logDir: File = null
|
var logDir: File = null
|
||||||
val time = new MockTime
|
val time = new MockTime
|
||||||
var config: KafkaConfig = null
|
var config: KafkaConfig = null
|
||||||
|
val logConfig = LogConfig()
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
def setUp() {
|
def setUp() {
|
||||||
|
@ -61,12 +62,15 @@ class LogTest extends JUnitSuite {
|
||||||
@Test
|
@Test
|
||||||
def testTimeBasedLogRoll() {
|
def testTimeBasedLogRoll() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singleMessageSet("test".getBytes())
|
||||||
val rollMs = 1 * 60 * 60L
|
|
||||||
val time: MockTime = new MockTime()
|
val time: MockTime = new MockTime()
|
||||||
|
|
||||||
// create a log
|
// create a log
|
||||||
val log = new Log(logDir, time.scheduler, 1000, config.messageMaxBytes, 1000, rollMs, needsRecovery = false, time = time)
|
val log = new Log(logDir,
|
||||||
time.sleep(rollMs + 1)
|
logConfig.copy(segmentMs = 1 * 60 * 60L),
|
||||||
|
needsRecovery = false,
|
||||||
|
scheduler = time.scheduler,
|
||||||
|
time = time)
|
||||||
|
time.sleep(log.config.segmentMs + 1)
|
||||||
|
|
||||||
// segment age is less than its limit
|
// segment age is less than its limit
|
||||||
log.append(set)
|
log.append(set)
|
||||||
|
@ -76,13 +80,13 @@ class LogTest extends JUnitSuite {
|
||||||
assertEquals("There should still be exactly one segment.", 1, log.numberOfSegments)
|
assertEquals("There should still be exactly one segment.", 1, log.numberOfSegments)
|
||||||
|
|
||||||
for(numSegments <- 2 until 4) {
|
for(numSegments <- 2 until 4) {
|
||||||
time.sleep(rollMs + 1)
|
time.sleep(log.config.segmentMs + 1)
|
||||||
log.append(set)
|
log.append(set)
|
||||||
assertEquals("Changing time beyond rollMs and appending should create a new segment.", numSegments, log.numberOfSegments)
|
assertEquals("Changing time beyond rollMs and appending should create a new segment.", numSegments, log.numberOfSegments)
|
||||||
}
|
}
|
||||||
|
|
||||||
val numSegments = log.numberOfSegments
|
val numSegments = log.numberOfSegments
|
||||||
time.sleep(rollMs + 1)
|
time.sleep(log.config.segmentMs + 1)
|
||||||
log.append(new ByteBufferMessageSet())
|
log.append(new ByteBufferMessageSet())
|
||||||
assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments)
|
assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments)
|
||||||
}
|
}
|
||||||
|
@ -95,10 +99,10 @@ class LogTest extends JUnitSuite {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singleMessageSet("test".getBytes)
|
||||||
val setSize = set.sizeInBytes
|
val setSize = set.sizeInBytes
|
||||||
val msgPerSeg = 10
|
val msgPerSeg = 10
|
||||||
val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
|
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
|
||||||
|
|
||||||
// create a log
|
// create a log
|
||||||
val log = new Log(logDir, time.scheduler, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), needsRecovery = false, time.scheduler, time = time)
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
||||||
|
|
||||||
// segments expire in size
|
// segments expire in size
|
||||||
|
@ -114,7 +118,7 @@ class LogTest extends JUnitSuite {
|
||||||
@Test
|
@Test
|
||||||
def testLoadEmptyLog() {
|
def testLoadEmptyLog() {
|
||||||
createEmptyLogs(logDir, 0)
|
createEmptyLogs(logDir, 0)
|
||||||
val log = new Log(logDir, time.scheduler, 1024, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig, needsRecovery = false, time.scheduler, time = time)
|
||||||
log.append(TestUtils.singleMessageSet("test".getBytes))
|
log.append(TestUtils.singleMessageSet("test".getBytes))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -123,7 +127,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testAppendAndReadWithSequentialOffsets() {
|
def testAppendAndReadWithSequentialOffsets() {
|
||||||
val log = new Log(logDir, time.scheduler, 71, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = 71), needsRecovery = false, time.scheduler, time = time)
|
||||||
val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray
|
val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray
|
||||||
|
|
||||||
for(i <- 0 until messages.length)
|
for(i <- 0 until messages.length)
|
||||||
|
@ -142,7 +146,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testAppendAndReadWithNonSequentialOffsets() {
|
def testAppendAndReadWithNonSequentialOffsets() {
|
||||||
val log = new Log(logDir, time.scheduler, 71, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = 71), needsRecovery = false, time.scheduler, time = time)
|
||||||
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
|
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
|
||||||
val messages = messageIds.map(id => new Message(id.toString.getBytes))
|
val messages = messageIds.map(id => new Message(id.toString.getBytes))
|
||||||
|
|
||||||
|
@ -165,7 +169,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testReadAtLogGap() {
|
def testReadAtLogGap() {
|
||||||
val log = new Log(logDir, time.scheduler, 300, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = 300), needsRecovery = false, time.scheduler, time = time)
|
||||||
|
|
||||||
// keep appending until we have two segments with only a single message in the second segment
|
// keep appending until we have two segments with only a single message in the second segment
|
||||||
while(log.numberOfSegments == 1)
|
while(log.numberOfSegments == 1)
|
||||||
|
@ -185,7 +189,7 @@ class LogTest extends JUnitSuite {
|
||||||
@Test
|
@Test
|
||||||
def testReadOutOfRange() {
|
def testReadOutOfRange() {
|
||||||
createEmptyLogs(logDir, 1024)
|
createEmptyLogs(logDir, 1024)
|
||||||
val log = new Log(logDir, time.scheduler, 1024, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = 1024), needsRecovery = false, time.scheduler, time = time)
|
||||||
assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1024, 1000).sizeInBytes)
|
assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1024, 1000).sizeInBytes)
|
||||||
try {
|
try {
|
||||||
log.read(0, 1024)
|
log.read(0, 1024)
|
||||||
|
@ -208,7 +212,7 @@ class LogTest extends JUnitSuite {
|
||||||
@Test
|
@Test
|
||||||
def testLogRolls() {
|
def testLogRolls() {
|
||||||
/* create a multipart log with 100 messages */
|
/* create a multipart log with 100 messages */
|
||||||
val log = new Log(logDir, time.scheduler, 100, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = 100), needsRecovery = false, time.scheduler, time = time)
|
||||||
val numMessages = 100
|
val numMessages = 100
|
||||||
val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes))
|
val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes))
|
||||||
val offsets = messageSets.map(log.append(_).firstOffset)
|
val offsets = messageSets.map(log.append(_).firstOffset)
|
||||||
|
@ -232,7 +236,7 @@ class LogTest extends JUnitSuite {
|
||||||
@Test
|
@Test
|
||||||
def testCompressedMessages() {
|
def testCompressedMessages() {
|
||||||
/* this log should roll after every messageset */
|
/* this log should roll after every messageset */
|
||||||
val log = new Log(logDir, time.scheduler, 10, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = 10), needsRecovery = false, time.scheduler, time = time)
|
||||||
|
|
||||||
/* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
|
/* 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("hello".getBytes), new Message("there".getBytes)))
|
||||||
|
@ -255,7 +259,7 @@ class LogTest extends JUnitSuite {
|
||||||
for(messagesToAppend <- List(0, 1, 25)) {
|
for(messagesToAppend <- List(0, 1, 25)) {
|
||||||
logDir.mkdirs()
|
logDir.mkdirs()
|
||||||
// first test a log segment starting at 0
|
// first test a log segment starting at 0
|
||||||
val log = new Log(logDir, time.scheduler, 100, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = 100), needsRecovery = false, time.scheduler, time = time)
|
||||||
for(i <- 0 until messagesToAppend)
|
for(i <- 0 until messagesToAppend)
|
||||||
log.append(TestUtils.singleMessageSet(i.toString.getBytes))
|
log.append(TestUtils.singleMessageSet(i.toString.getBytes))
|
||||||
|
|
||||||
|
@ -289,7 +293,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
// append messages to log
|
// append messages to log
|
||||||
val maxMessageSize = second.sizeInBytes - 1
|
val maxMessageSize = second.sizeInBytes - 1
|
||||||
val log = new Log(logDir, time.scheduler, 100, maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(maxMessageSize = maxMessageSize), needsRecovery = false, time.scheduler, time = time)
|
||||||
|
|
||||||
// should be able to append the small message
|
// should be able to append the small message
|
||||||
log.append(first)
|
log.append(first)
|
||||||
|
@ -311,7 +315,8 @@ class LogTest extends JUnitSuite {
|
||||||
val messageSize = 100
|
val messageSize = 100
|
||||||
val segmentSize = 7 * messageSize
|
val segmentSize = 7 * messageSize
|
||||||
val indexInterval = 3 * messageSize
|
val indexInterval = 3 * messageSize
|
||||||
var log = new Log(logDir, time.scheduler, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
|
val config = logConfig.copy(segmentSize = segmentSize, indexInterval = indexInterval, maxIndexSize = 4096)
|
||||||
|
var log = new Log(logDir, config, needsRecovery = false, time.scheduler, time)
|
||||||
for(i <- 0 until numMessages)
|
for(i <- 0 until numMessages)
|
||||||
log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(messageSize)))
|
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)
|
assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset)
|
||||||
|
@ -319,15 +324,14 @@ class LogTest extends JUnitSuite {
|
||||||
val numIndexEntries = log.activeSegment.index.entries
|
val numIndexEntries = log.activeSegment.index.entries
|
||||||
log.close()
|
log.close()
|
||||||
|
|
||||||
// test non-recovery case
|
log = new Log(logDir, config, needsRecovery = false, time.scheduler, time)
|
||||||
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 %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 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)
|
assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries)
|
||||||
log.close()
|
log.close()
|
||||||
|
|
||||||
// test recovery case
|
// test recovery case
|
||||||
log = new Log(logDir, time.scheduler, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
|
log = new Log(logDir, config, needsRecovery = true, time.scheduler, time)
|
||||||
assertEquals("Should have %d messages when log is reopened with recovery".format(numMessages), numMessages, log.logEndOffset)
|
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 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)
|
assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries)
|
||||||
|
@ -341,7 +345,8 @@ class LogTest extends JUnitSuite {
|
||||||
def testIndexRebuild() {
|
def testIndexRebuild() {
|
||||||
// publish the messages and close the log
|
// publish the messages and close the log
|
||||||
val numMessages = 200
|
val numMessages = 200
|
||||||
var log = new Log(logDir, time.scheduler, 200, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = 1, maxIndexSize = 4096)
|
val config = logConfig.copy(segmentSize = 200, indexInterval = 1)
|
||||||
|
var log = new Log(logDir, config, needsRecovery = true, time.scheduler, time)
|
||||||
for(i <- 0 until numMessages)
|
for(i <- 0 until numMessages)
|
||||||
log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10)))
|
log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10)))
|
||||||
val indexFiles = log.logSegments.map(_.index.file)
|
val indexFiles = log.logSegments.map(_.index.file)
|
||||||
|
@ -351,8 +356,7 @@ class LogTest extends JUnitSuite {
|
||||||
indexFiles.foreach(_.delete())
|
indexFiles.foreach(_.delete())
|
||||||
|
|
||||||
// reopen the log
|
// reopen the log
|
||||||
log = new Log(logDir, time.scheduler, 200, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = 1, maxIndexSize = 4096)
|
log = new Log(logDir, config, needsRecovery = true, time.scheduler, time)
|
||||||
|
|
||||||
assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
|
assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
|
||||||
for(i <- 0 until numMessages)
|
for(i <- 0 until numMessages)
|
||||||
assertEquals(i, log.read(i, 100, None).head.offset)
|
assertEquals(i, log.read(i, 100, None).head.offset)
|
||||||
|
@ -367,10 +371,10 @@ class LogTest extends JUnitSuite {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singleMessageSet("test".getBytes())
|
||||||
val setSize = set.sizeInBytes
|
val setSize = set.sizeInBytes
|
||||||
val msgPerSeg = 10
|
val msgPerSeg = 10
|
||||||
val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
|
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
|
||||||
|
|
||||||
// create a log
|
// create a log
|
||||||
val log = new Log(logDir, time.scheduler, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time)
|
val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), needsRecovery = false, scheduler = time.scheduler, time = time)
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
||||||
|
|
||||||
for (i<- 1 to msgPerSeg)
|
for (i<- 1 to msgPerSeg)
|
||||||
|
@ -421,8 +425,9 @@ class LogTest extends JUnitSuite {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singleMessageSet("test".getBytes())
|
||||||
val setSize = set.sizeInBytes
|
val setSize = set.sizeInBytes
|
||||||
val msgPerSeg = 10
|
val msgPerSeg = 10
|
||||||
val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
|
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
|
||||||
val log = new Log(logDir, time.scheduler, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time)
|
val config = logConfig.copy(segmentSize = segmentSize)
|
||||||
|
val log = new Log(logDir, config, needsRecovery = false, scheduler = time.scheduler, time = time)
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
||||||
for (i<- 1 to msgPerSeg)
|
for (i<- 1 to msgPerSeg)
|
||||||
log.append(set)
|
log.append(set)
|
||||||
|
@ -430,10 +435,10 @@ class LogTest extends JUnitSuite {
|
||||||
for (i<- 1 to msgPerSeg)
|
for (i<- 1 to msgPerSeg)
|
||||||
log.append(set)
|
log.append(set)
|
||||||
assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments)
|
assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments)
|
||||||
assertEquals("The index of the first segment should be trim to empty", 0, log.logSegments.toList(0).index.maxEntries)
|
assertEquals("The index of the first segment should be trimmed to empty", 0, log.logSegments.toList(0).index.maxEntries)
|
||||||
log.truncateTo(0)
|
log.truncateTo(0)
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
||||||
assertEquals("The index of segment 1 should be resized to maxIndexSize", log.maxIndexSize/8, log.logSegments.toList(0).index.maxEntries)
|
assertEquals("The index of segment 1 should be resized to maxIndexSize", log.config.maxIndexSize/8, log.logSegments.toList(0).index.maxEntries)
|
||||||
for (i<- 1 to msgPerSeg)
|
for (i<- 1 to msgPerSeg)
|
||||||
log.append(set)
|
log.append(set)
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
||||||
|
@ -449,12 +454,12 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singleMessageSet("test".getBytes())
|
||||||
val log = new Log(logDir,
|
val log = new Log(logDir,
|
||||||
|
logConfig.copy(segmentSize = set.sizeInBytes * 5,
|
||||||
|
maxIndexSize = 1000,
|
||||||
|
indexInterval = 1),
|
||||||
|
needsRecovery = false,
|
||||||
time.scheduler,
|
time.scheduler,
|
||||||
maxSegmentSize = set.sizeInBytes * 5,
|
time)
|
||||||
maxMessageSize = config.messageMaxBytes,
|
|
||||||
maxIndexSize = 1000,
|
|
||||||
indexIntervalBytes = 1,
|
|
||||||
needsRecovery = false)
|
|
||||||
|
|
||||||
assertTrue("The first index file should have been replaced with a larger file", bogusIndex1.length > 0)
|
assertTrue("The first index file should have been replaced with a larger file", bogusIndex1.length > 0)
|
||||||
assertFalse("The second index file should have been deleted.", bogusIndex2.exists)
|
assertFalse("The second index file should have been deleted.", bogusIndex2.exists)
|
||||||
|
@ -472,27 +477,26 @@ class LogTest extends JUnitSuite {
|
||||||
@Test
|
@Test
|
||||||
def testReopenThenTruncate() {
|
def testReopenThenTruncate() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singleMessageSet("test".getBytes())
|
||||||
|
val config = logConfig.copy(segmentSize = set.sizeInBytes * 5,
|
||||||
|
maxIndexSize = 1000,
|
||||||
|
indexInterval = 10000)
|
||||||
|
|
||||||
// create a log
|
// create a log
|
||||||
var log = new Log(logDir,
|
var log = new Log(logDir,
|
||||||
|
config,
|
||||||
|
needsRecovery = true,
|
||||||
time.scheduler,
|
time.scheduler,
|
||||||
maxSegmentSize = set.sizeInBytes * 5,
|
time)
|
||||||
maxMessageSize = config.messageMaxBytes,
|
|
||||||
maxIndexSize = 1000,
|
|
||||||
indexIntervalBytes = 10000,
|
|
||||||
needsRecovery = true)
|
|
||||||
|
|
||||||
// add enough messages to roll over several segments then close and re-open and attempt to truncate
|
// add enough messages to roll over several segments then close and re-open and attempt to truncate
|
||||||
for(i <- 0 until 100)
|
for(i <- 0 until 100)
|
||||||
log.append(set)
|
log.append(set)
|
||||||
log.close()
|
log.close()
|
||||||
log = new Log(logDir,
|
log = new Log(logDir,
|
||||||
|
config,
|
||||||
|
needsRecovery = true,
|
||||||
time.scheduler,
|
time.scheduler,
|
||||||
maxSegmentSize = set.sizeInBytes * 5,
|
time)
|
||||||
maxMessageSize = config.messageMaxBytes,
|
|
||||||
maxIndexSize = 1000,
|
|
||||||
indexIntervalBytes = 10000,
|
|
||||||
needsRecovery = true)
|
|
||||||
log.truncateTo(3)
|
log.truncateTo(3)
|
||||||
assertEquals("All but one segment should be deleted.", 1, log.numberOfSegments)
|
assertEquals("All but one segment should be deleted.", 1, log.numberOfSegments)
|
||||||
assertEquals("Log end offset should be 3.", 3, log.logEndOffset)
|
assertEquals("Log end offset should be 3.", 3, log.logEndOffset)
|
||||||
|
@ -505,14 +509,15 @@ class LogTest extends JUnitSuite {
|
||||||
def testAsyncDelete() {
|
def testAsyncDelete() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singleMessageSet("test".getBytes())
|
||||||
val asyncDeleteMs = 1000
|
val asyncDeleteMs = 1000
|
||||||
|
val config = logConfig.copy(segmentSize = set.sizeInBytes * 5,
|
||||||
|
fileDeleteDelayMs = asyncDeleteMs,
|
||||||
|
maxIndexSize = 1000,
|
||||||
|
indexInterval = 10000)
|
||||||
val log = new Log(logDir,
|
val log = new Log(logDir,
|
||||||
|
config,
|
||||||
|
needsRecovery = true,
|
||||||
time.scheduler,
|
time.scheduler,
|
||||||
maxSegmentSize = set.sizeInBytes * 5,
|
time)
|
||||||
maxMessageSize = config.messageMaxBytes,
|
|
||||||
maxIndexSize = 1000,
|
|
||||||
indexIntervalBytes = 10000,
|
|
||||||
segmentDeleteDelayMs = asyncDeleteMs,
|
|
||||||
needsRecovery = true)
|
|
||||||
|
|
||||||
// append some messages to create some segments
|
// append some messages to create some segments
|
||||||
for(i <- 0 until 100)
|
for(i <- 0 until 100)
|
||||||
|
@ -520,15 +525,20 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
// files should be renamed
|
// files should be renamed
|
||||||
val segments = log.logSegments.toArray
|
val segments = log.logSegments.toArray
|
||||||
|
val oldFiles = segments.map(_.log.file) ++ segments.map(_.index.file)
|
||||||
log.deleteOldSegments((s) => true)
|
log.deleteOldSegments((s) => true)
|
||||||
|
|
||||||
assertEquals("Only one segment should remain.", 1, log.numberOfSegments)
|
assertEquals("Only one segment should remain.", 1, log.numberOfSegments)
|
||||||
val renamed = segments.map(segment => new File(segment.log.file.getPath + Log.DeletedFileSuffix))
|
assertTrue("All log and index files should end in .deleted", segments.forall(_.log.file.getName.endsWith(Log.DeletedFileSuffix)) &&
|
||||||
assertTrue("Files should all be renamed to .deleted.", renamed.forall(_.exists))
|
segments.forall(_.index.file.getName.endsWith(Log.DeletedFileSuffix)))
|
||||||
|
assertTrue("The .deleted files should still be there.", segments.forall(_.log.file.exists) &&
|
||||||
|
segments.forall(_.index.file.exists))
|
||||||
|
assertTrue("The original file should be gone.", oldFiles.forall(!_.exists))
|
||||||
|
|
||||||
// when enough time passes the files should be deleted
|
// when enough time passes the files should be deleted
|
||||||
|
val deletedFiles = segments.map(_.log.file) ++ segments.map(_.index.file)
|
||||||
time.sleep(asyncDeleteMs + 1)
|
time.sleep(asyncDeleteMs + 1)
|
||||||
assertTrue("Files should all be gone.", renamed.forall(!_.exists))
|
assertTrue("Files should all be gone.", deletedFiles.forall(!_.exists))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -537,13 +547,12 @@ class LogTest extends JUnitSuite {
|
||||||
@Test
|
@Test
|
||||||
def testOpenDeletesObsoleteFiles() {
|
def testOpenDeletesObsoleteFiles() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singleMessageSet("test".getBytes())
|
||||||
|
val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, maxIndexSize = 1000)
|
||||||
var log = new Log(logDir,
|
var log = new Log(logDir,
|
||||||
|
config,
|
||||||
|
needsRecovery = false,
|
||||||
time.scheduler,
|
time.scheduler,
|
||||||
maxSegmentSize = set.sizeInBytes * 5,
|
time)
|
||||||
maxMessageSize = config.messageMaxBytes,
|
|
||||||
maxIndexSize = 1000,
|
|
||||||
indexIntervalBytes = 10000,
|
|
||||||
needsRecovery = false)
|
|
||||||
|
|
||||||
// append some messages to create some segments
|
// append some messages to create some segments
|
||||||
for(i <- 0 until 100)
|
for(i <- 0 until 100)
|
||||||
|
@ -553,12 +562,10 @@ class LogTest extends JUnitSuite {
|
||||||
log.close()
|
log.close()
|
||||||
|
|
||||||
log = new Log(logDir,
|
log = new Log(logDir,
|
||||||
|
config,
|
||||||
|
needsRecovery = false,
|
||||||
time.scheduler,
|
time.scheduler,
|
||||||
maxSegmentSize = set.sizeInBytes * 5,
|
time)
|
||||||
maxMessageSize = config.messageMaxBytes,
|
|
||||||
maxIndexSize = 1000,
|
|
||||||
indexIntervalBytes = 10000,
|
|
||||||
needsRecovery = false)
|
|
||||||
assertEquals("The deleted segments should be gone.", 1, log.numberOfSegments)
|
assertEquals("The deleted segments should be gone.", 1, log.numberOfSegments)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,87 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package kafka.log
|
||||||
|
|
||||||
|
import java.nio._
|
||||||
|
import org.junit._
|
||||||
|
import org.scalatest.junit.JUnitSuite
|
||||||
|
import junit.framework.Assert._
|
||||||
|
|
||||||
|
class OffsetMapTest extends JUnitSuite {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testBasicValidation() {
|
||||||
|
validateMap(10)
|
||||||
|
validateMap(100)
|
||||||
|
validateMap(1000)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testClear() {
|
||||||
|
val map = new SkimpyOffsetMap(4000, 0.75)
|
||||||
|
for(i <- 0 until 10)
|
||||||
|
map.put(key(i), i)
|
||||||
|
for(i <- 0 until 10)
|
||||||
|
assertEquals(i.toLong, map.get(key(i)))
|
||||||
|
map.clear()
|
||||||
|
for(i <- 0 until 10)
|
||||||
|
assertEquals(map.get(key(i)), -1L)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testCapacity() {
|
||||||
|
val map = new SkimpyOffsetMap(1024, 0.75)
|
||||||
|
var i = 0
|
||||||
|
while(map.size < map.capacity) {
|
||||||
|
map.put(key(i), i)
|
||||||
|
i += 1
|
||||||
|
}
|
||||||
|
// now the map is full, it should throw an exception
|
||||||
|
intercept[IllegalStateException] {
|
||||||
|
map.put(key(i), i)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def key(key: Int) = ByteBuffer.wrap(key.toString.getBytes)
|
||||||
|
|
||||||
|
def validateMap(items: Int) {
|
||||||
|
val map = new SkimpyOffsetMap(items * 2 * 24, 0.75)
|
||||||
|
for(i <- 0 until items)
|
||||||
|
map.put(key(i), i)
|
||||||
|
var misses = 0
|
||||||
|
for(i <- 0 until items) {
|
||||||
|
map.get(key(i)) match {
|
||||||
|
case -1L => misses += 1
|
||||||
|
case offset => assertEquals(i.toLong, offset)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
println("Miss rate: " + (misses.toDouble / items))
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
object OffsetMapTest {
|
||||||
|
def main(args: Array[String]) {
|
||||||
|
if(args.length != 1) {
|
||||||
|
System.err.println("USAGE: java OffsetMapTest size")
|
||||||
|
System.exit(1)
|
||||||
|
}
|
||||||
|
val test = new OffsetMapTest()
|
||||||
|
test.validateMap(args(0).toInt)
|
||||||
|
}
|
||||||
|
}
|
|
@ -16,13 +16,14 @@
|
||||||
*/
|
*/
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import kafka.log.LogManager
|
import kafka.log._
|
||||||
|
import java.io.File
|
||||||
import org.I0Itec.zkclient.ZkClient
|
import org.I0Itec.zkclient.ZkClient
|
||||||
import org.scalatest.junit.JUnit3Suite
|
import org.scalatest.junit.JUnit3Suite
|
||||||
import org.easymock.EasyMock
|
import org.easymock.EasyMock
|
||||||
import org.junit._
|
import org.junit._
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import kafka.common.KafkaException
|
import kafka.common._
|
||||||
import kafka.cluster.Replica
|
import kafka.cluster.Replica
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
|
|
||||||
|
@ -30,7 +31,14 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
|
||||||
|
|
||||||
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_))
|
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_))
|
||||||
val topic = "foo"
|
val topic = "foo"
|
||||||
val logManagers = configs.map(config => new LogManager(config, new KafkaScheduler(1), new MockTime))
|
val logManagers = configs.map(config => new LogManager(logDirs = config.logDirs.map(new File(_)).toArray,
|
||||||
|
topicConfigs = Map(),
|
||||||
|
defaultConfig = LogConfig(),
|
||||||
|
cleanerConfig = CleanerConfig(),
|
||||||
|
flushCheckMs = 30000,
|
||||||
|
retentionCheckMs = 30000,
|
||||||
|
scheduler = new KafkaScheduler(1),
|
||||||
|
time = new MockTime))
|
||||||
|
|
||||||
@After
|
@After
|
||||||
def teardown() {
|
def teardown() {
|
||||||
|
@ -133,7 +141,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
|
||||||
}
|
}
|
||||||
|
|
||||||
def hwmFor(replicaManager: ReplicaManager, topic: String, partition: Int): Long = {
|
def hwmFor(replicaManager: ReplicaManager, topic: String, partition: Int): Long = {
|
||||||
replicaManager.highWatermarkCheckpoints(replicaManager.config.logDirs(0)).read(topic, partition)
|
replicaManager.highWatermarkCheckpoints(replicaManager.config.logDirs(0)).read.getOrElse(TopicAndPartition(topic, partition), 0L)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -205,7 +205,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
props.put("enable.zookeeper", "false")
|
props.put("enable.zookeeper", "false")
|
||||||
props.put("num.partitions", "20")
|
props.put("num.partitions", "20")
|
||||||
props.put("log.retention.hours", "10")
|
props.put("log.retention.hours", "10")
|
||||||
props.put("log.cleanup.interval.mins", "5")
|
props.put("log.retention.check.interval.ms", (5*1000*60).toString)
|
||||||
props.put("log.segment.bytes", logSize.toString)
|
props.put("log.segment.bytes", logSize.toString)
|
||||||
props.put("zk.connect", zkConnect.toString)
|
props.put("zk.connect", zkConnect.toString)
|
||||||
props
|
props
|
||||||
|
|
|
@ -18,11 +18,13 @@ package kafka.server
|
||||||
|
|
||||||
import org.scalatest.junit.JUnit3Suite
|
import org.scalatest.junit.JUnit3Suite
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
|
import java.io.File
|
||||||
import kafka.admin.CreateTopicCommand
|
import kafka.admin.CreateTopicCommand
|
||||||
import kafka.utils.TestUtils._
|
import kafka.utils.TestUtils._
|
||||||
import kafka.utils.IntEncoder
|
import kafka.utils.IntEncoder
|
||||||
import kafka.utils.{Utils, TestUtils}
|
import kafka.utils.{Utils, TestUtils}
|
||||||
import kafka.zk.ZooKeeperTestHarness
|
import kafka.zk.ZooKeeperTestHarness
|
||||||
|
import kafka.common._
|
||||||
import kafka.producer.{ProducerConfig, KeyedMessage, Producer}
|
import kafka.producer.{ProducerConfig, KeyedMessage, Producer}
|
||||||
|
|
||||||
class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
|
@ -44,8 +46,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
val message = "hello"
|
val message = "hello"
|
||||||
|
|
||||||
var producer: Producer[Int, String] = null
|
var producer: Producer[Int, String] = null
|
||||||
var hwFile1: HighwaterMarkCheckpoint = new HighwaterMarkCheckpoint(configProps1.logDirs(0))
|
var hwFile1: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename))
|
||||||
var hwFile2: HighwaterMarkCheckpoint = new HighwaterMarkCheckpoint(configProps2.logDirs(0))
|
var hwFile2: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps2.logDirs(0), ReplicaManager.HighWatermarkFilename))
|
||||||
var servers: Seq[KafkaServer] = Seq.empty[KafkaServer]
|
var servers: Seq[KafkaServer] = Seq.empty[KafkaServer]
|
||||||
|
|
||||||
val producerProps = getProducerConfig(TestUtils.getBrokerListStrFromConfigs(configs), 64*1024, 100000, 10000)
|
val producerProps = getProducerConfig(TestUtils.getBrokerListStrFromConfigs(configs), 64*1024, 100000, 10000)
|
||||||
|
@ -80,9 +82,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
|
|
||||||
servers.foreach(server => server.replicaManager.checkpointHighWatermarks())
|
servers.foreach(server => server.replicaManager.checkpointHighWatermarks())
|
||||||
producer.close()
|
producer.close()
|
||||||
val leaderHW = hwFile1.read(topic, 0)
|
val leaderHW = hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)
|
||||||
assertEquals(numMessages, leaderHW)
|
assertEquals(numMessages, leaderHW)
|
||||||
val followerHW = hwFile2.read(topic, 0)
|
val followerHW = hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)
|
||||||
assertEquals(numMessages, followerHW)
|
assertEquals(numMessages, followerHW)
|
||||||
servers.foreach(server => { server.shutdown(); Utils.rm(server.config.logDirs(0))})
|
servers.foreach(server => { server.shutdown(); Utils.rm(server.config.logDirs(0))})
|
||||||
}
|
}
|
||||||
|
@ -104,7 +106,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
// NOTE: this is to avoid transient test failures
|
// NOTE: this is to avoid transient test failures
|
||||||
assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1))
|
assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1))
|
||||||
|
|
||||||
assertEquals(0L, hwFile1.read(topic, 0))
|
assertEquals(0L, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
|
|
||||||
sendMessages(1)
|
sendMessages(1)
|
||||||
Thread.sleep(1000)
|
Thread.sleep(1000)
|
||||||
|
@ -112,7 +114,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
|
|
||||||
// kill the server hosting the preferred replica
|
// kill the server hosting the preferred replica
|
||||||
server1.shutdown()
|
server1.shutdown()
|
||||||
assertEquals(hw, hwFile1.read(topic, 0))
|
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
|
|
||||||
// check if leader moves to the other server
|
// check if leader moves to the other server
|
||||||
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
|
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
|
||||||
|
@ -125,10 +127,10 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
assertTrue("Leader must remain on broker 1, in case of zookeeper session expiration it can move to broker 0",
|
assertTrue("Leader must remain on broker 1, in case of zookeeper session expiration it can move to broker 0",
|
||||||
leader.isDefined && (leader.get == 0 || leader.get == 1))
|
leader.isDefined && (leader.get == 0 || leader.get == 1))
|
||||||
|
|
||||||
assertEquals(hw, hwFile1.read(topic, 0))
|
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
// since server 2 was never shut down, the hw value of 30 is probably not checkpointed to disk yet
|
// since server 2 was never shut down, the hw value of 30 is probably not checkpointed to disk yet
|
||||||
server2.shutdown()
|
server2.shutdown()
|
||||||
assertEquals(hw, hwFile2.read(topic, 0))
|
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
|
|
||||||
server2.startup()
|
server2.startup()
|
||||||
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
|
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
|
||||||
|
@ -144,8 +146,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
// shutdown the servers to allow the hw to be checkpointed
|
// shutdown the servers to allow the hw to be checkpointed
|
||||||
servers.foreach(server => server.shutdown())
|
servers.foreach(server => server.shutdown())
|
||||||
producer.close()
|
producer.close()
|
||||||
assertEquals(hw, hwFile1.read(topic, 0))
|
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
assertEquals(hw, hwFile2.read(topic, 0))
|
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
servers.foreach(server => Utils.rm(server.config.logDirs))
|
servers.foreach(server => Utils.rm(server.config.logDirs))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -155,8 +157,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
server2 = TestUtils.createServer(configs.last)
|
server2 = TestUtils.createServer(configs.last)
|
||||||
servers ++= List(server1, server2)
|
servers ++= List(server1, server2)
|
||||||
|
|
||||||
hwFile1 = new HighwaterMarkCheckpoint(server1.config.logDirs(0))
|
hwFile1 = new OffsetCheckpoint(new File(server1.config.logDirs(0), ReplicaManager.HighWatermarkFilename))
|
||||||
hwFile2 = new HighwaterMarkCheckpoint(server2.config.logDirs(0))
|
hwFile2 = new OffsetCheckpoint(new File(server2.config.logDirs(0), ReplicaManager.HighWatermarkFilename))
|
||||||
|
|
||||||
producer = new Producer[Int, String](new ProducerConfig(producerProps))
|
producer = new Producer[Int, String](new ProducerConfig(producerProps))
|
||||||
|
|
||||||
|
@ -176,9 +178,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
// shutdown the servers to allow the hw to be checkpointed
|
// shutdown the servers to allow the hw to be checkpointed
|
||||||
servers.foreach(server => server.shutdown())
|
servers.foreach(server => server.shutdown())
|
||||||
producer.close()
|
producer.close()
|
||||||
val leaderHW = hwFile1.read(topic, 0)
|
val leaderHW = hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)
|
||||||
assertEquals(hw, leaderHW)
|
assertEquals(hw, leaderHW)
|
||||||
val followerHW = hwFile2.read(topic, 0)
|
val followerHW = hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)
|
||||||
assertEquals(hw, followerHW)
|
assertEquals(hw, followerHW)
|
||||||
servers.foreach(server => Utils.rm(server.config.logDirs))
|
servers.foreach(server => Utils.rm(server.config.logDirs))
|
||||||
}
|
}
|
||||||
|
@ -189,8 +191,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
server2 = TestUtils.createServer(configs.last)
|
server2 = TestUtils.createServer(configs.last)
|
||||||
servers ++= List(server1, server2)
|
servers ++= List(server1, server2)
|
||||||
|
|
||||||
hwFile1 = new HighwaterMarkCheckpoint(server1.config.logDirs(0))
|
hwFile1 = new OffsetCheckpoint(new File(server1.config.logDirs(0), ReplicaManager.HighWatermarkFilename))
|
||||||
hwFile2 = new HighwaterMarkCheckpoint(server2.config.logDirs(0))
|
hwFile2 = new OffsetCheckpoint(new File(server2.config.logDirs(0), ReplicaManager.HighWatermarkFilename))
|
||||||
|
|
||||||
producer = new Producer[Int, String](new ProducerConfig(producerProps))
|
producer = new Producer[Int, String](new ProducerConfig(producerProps))
|
||||||
|
|
||||||
|
@ -212,21 +214,21 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
// kill the server hosting the preferred replica
|
// kill the server hosting the preferred replica
|
||||||
server1.shutdown()
|
server1.shutdown()
|
||||||
server2.shutdown()
|
server2.shutdown()
|
||||||
assertEquals(hw, hwFile1.read(topic, 0))
|
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
assertEquals(hw, hwFile2.read(topic, 0))
|
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
|
|
||||||
server2.startup()
|
server2.startup()
|
||||||
// check if leader moves to the other server
|
// check if leader moves to the other server
|
||||||
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
|
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
|
||||||
assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1))
|
assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1))
|
||||||
|
|
||||||
assertEquals(hw, hwFile1.read(topic, 0))
|
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
|
|
||||||
// bring the preferred replica back
|
// bring the preferred replica back
|
||||||
server1.startup()
|
server1.startup()
|
||||||
|
|
||||||
assertEquals(hw, hwFile1.read(topic, 0))
|
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
assertEquals(hw, hwFile2.read(topic, 0))
|
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
|
|
||||||
sendMessages(2)
|
sendMessages(2)
|
||||||
hw += 2
|
hw += 2
|
||||||
|
@ -237,8 +239,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
// shutdown the servers to allow the hw to be checkpointed
|
// shutdown the servers to allow the hw to be checkpointed
|
||||||
servers.foreach(server => server.shutdown())
|
servers.foreach(server => server.shutdown())
|
||||||
producer.close()
|
producer.close()
|
||||||
assertEquals(hw, hwFile1.read(topic, 0))
|
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
assertEquals(hw, hwFile2.read(topic, 0))
|
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
|
||||||
servers.foreach(server => Utils.rm(server.config.logDirs))
|
servers.foreach(server => Utils.rm(server.config.logDirs))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -68,7 +68,6 @@ class SimpleFetchTest extends JUnit3Suite {
|
||||||
|
|
||||||
val logManager = EasyMock.createMock(classOf[kafka.log.LogManager])
|
val logManager = EasyMock.createMock(classOf[kafka.log.LogManager])
|
||||||
EasyMock.expect(logManager.getLog(topic, partitionId)).andReturn(Some(log)).anyTimes()
|
EasyMock.expect(logManager.getLog(topic, partitionId)).andReturn(Some(log)).anyTimes()
|
||||||
EasyMock.expect(logManager.config).andReturn(configs.head).anyTimes()
|
|
||||||
EasyMock.replay(logManager)
|
EasyMock.replay(logManager)
|
||||||
|
|
||||||
val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager])
|
val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager])
|
||||||
|
@ -135,7 +134,6 @@ class SimpleFetchTest extends JUnit3Suite {
|
||||||
|
|
||||||
val logManager = EasyMock.createMock(classOf[kafka.log.LogManager])
|
val logManager = EasyMock.createMock(classOf[kafka.log.LogManager])
|
||||||
EasyMock.expect(logManager.getLog(topic, 0)).andReturn(Some(log)).anyTimes()
|
EasyMock.expect(logManager.getLog(topic, 0)).andReturn(Some(log)).anyTimes()
|
||||||
EasyMock.expect(logManager.config).andReturn(configs.head).anyTimes()
|
|
||||||
EasyMock.replay(logManager)
|
EasyMock.replay(logManager)
|
||||||
|
|
||||||
val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager])
|
val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager])
|
||||||
|
|
|
@ -32,7 +32,6 @@ import java.util.concurrent.TimeUnit
|
||||||
*
|
*
|
||||||
* Incrementing the time to the exact next execution time of a task will result in that task executing (it as if execution itself takes no time).
|
* Incrementing the time to the exact next execution time of a task will result in that task executing (it as if execution itself takes no time).
|
||||||
*/
|
*/
|
||||||
@nonthreadsafe
|
|
||||||
class MockScheduler(val time: Time) extends Scheduler {
|
class MockScheduler(val time: Time) extends Scheduler {
|
||||||
|
|
||||||
/* a priority queue of tasks ordered by next execution time */
|
/* a priority queue of tasks ordered by next execution time */
|
||||||
|
@ -41,7 +40,9 @@ class MockScheduler(val time: Time) extends Scheduler {
|
||||||
def startup() {}
|
def startup() {}
|
||||||
|
|
||||||
def shutdown() {
|
def shutdown() {
|
||||||
tasks.clear()
|
this synchronized {
|
||||||
|
tasks.clear()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -50,23 +51,26 @@ class MockScheduler(val time: Time) extends Scheduler {
|
||||||
* If you are using the scheduler associated with a MockTime instance this call be triggered automatically.
|
* If you are using the scheduler associated with a MockTime instance this call be triggered automatically.
|
||||||
*/
|
*/
|
||||||
def tick() {
|
def tick() {
|
||||||
val now = time.milliseconds
|
this synchronized {
|
||||||
while(!tasks.isEmpty && tasks.head.nextExecution <= now) {
|
val now = time.milliseconds
|
||||||
/* pop and execute the task with the lowest next execution time */
|
while(!tasks.isEmpty && tasks.head.nextExecution <= now) {
|
||||||
val curr = tasks.head
|
/* pop and execute the task with the lowest next execution time */
|
||||||
this.tasks = tasks.tail
|
val curr = tasks.dequeue
|
||||||
curr.fun()
|
curr.fun()
|
||||||
/* if the task is periodic, reschedule it and re-enqueue */
|
/* if the task is periodic, reschedule it and re-enqueue */
|
||||||
if(curr.periodic) {
|
if(curr.periodic) {
|
||||||
curr.nextExecution += curr.period
|
curr.nextExecution += curr.period
|
||||||
this.tasks += curr
|
this.tasks += curr
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def schedule(name: String, fun: ()=>Unit, delay: Long = 0, period: Long = -1, unit: TimeUnit = TimeUnit.MILLISECONDS) {
|
def schedule(name: String, fun: ()=>Unit, delay: Long = 0, period: Long = -1, unit: TimeUnit = TimeUnit.MILLISECONDS) {
|
||||||
tasks += MockTask(name, fun, time.milliseconds + delay, period = period)
|
this synchronized {
|
||||||
tick()
|
tasks += MockTask(name, fun, time.milliseconds + delay, period = period)
|
||||||
|
tick()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -44,6 +44,8 @@ import kafka.common.TopicAndPartition
|
||||||
*/
|
*/
|
||||||
object TestUtils extends Logging {
|
object TestUtils extends Logging {
|
||||||
|
|
||||||
|
val IoTmpDir = System.getProperty("java.io.tmpdir")
|
||||||
|
|
||||||
val Letters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"
|
val Letters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"
|
||||||
val Digits = "0123456789"
|
val Digits = "0123456789"
|
||||||
val LettersAndDigits = Letters + Digits
|
val LettersAndDigits = Letters + Digits
|
||||||
|
@ -74,8 +76,7 @@ object TestUtils extends Logging {
|
||||||
* Create a temporary directory
|
* Create a temporary directory
|
||||||
*/
|
*/
|
||||||
def tempDir(): File = {
|
def tempDir(): File = {
|
||||||
val ioDir = System.getProperty("java.io.tmpdir")
|
val f = new File(IoTmpDir, "kafka-" + random.nextInt(1000000))
|
||||||
val f = new File(ioDir, "kafka-" + random.nextInt(1000000))
|
|
||||||
f.mkdirs()
|
f.mkdirs()
|
||||||
f.deleteOnExit()
|
f.deleteOnExit()
|
||||||
f
|
f
|
||||||
|
@ -154,8 +155,8 @@ object TestUtils extends Logging {
|
||||||
* Wrap the message in a message set
|
* Wrap the message in a message set
|
||||||
* @param payload The bytes of the message
|
* @param payload The bytes of the message
|
||||||
*/
|
*/
|
||||||
def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec) =
|
def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec, key: Array[Byte] = null) =
|
||||||
new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload))
|
new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Generate an array of random bytes
|
* Generate an array of random bytes
|
||||||
|
@ -497,7 +498,7 @@ object TestUtils extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
object TestZKUtils {
|
object TestZKUtils {
|
||||||
val zookeeperConnect = "127.0.0.1:2182"
|
val zookeeperConnect = "127.0.0.1:" + TestUtils.choosePort()
|
||||||
}
|
}
|
||||||
|
|
||||||
class IntEncoder(props: VerifiableProperties = null) extends Encoder[Int] {
|
class IntEncoder(props: VerifiableProperties = null) extends Encoder[Int] {
|
||||||
|
|
|
@ -19,6 +19,7 @@ package kafka.utils
|
||||||
|
|
||||||
import java.util.Arrays
|
import java.util.Arrays
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
|
import java.io._
|
||||||
import org.apache.log4j.Logger
|
import org.apache.log4j.Logger
|
||||||
import org.scalatest.junit.JUnitSuite
|
import org.scalatest.junit.JUnitSuite
|
||||||
import org.junit.Test
|
import org.junit.Test
|
||||||
|
@ -62,6 +63,25 @@ class UtilsTest extends JUnitSuite {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testReplaceSuffix() {
|
||||||
|
assertEquals("blah.foo.text", Utils.replaceSuffix("blah.foo.txt", ".txt", ".text"))
|
||||||
|
assertEquals("blah.foo", Utils.replaceSuffix("blah.foo.txt", ".txt", ""))
|
||||||
|
assertEquals("txt.txt", Utils.replaceSuffix("txt.txt.txt", ".txt", ""))
|
||||||
|
assertEquals("foo.txt", Utils.replaceSuffix("foo", "", ".txt"))
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testReadInt() {
|
||||||
|
val values = Array(0, 1, -1, Byte.MaxValue, Short.MaxValue, 2 * Short.MaxValue, Int.MaxValue/2, Int.MinValue/2, Int.MaxValue, Int.MinValue, Int.MaxValue)
|
||||||
|
val buffer = ByteBuffer.allocate(4 * values.size)
|
||||||
|
for(i <- 0 until values.length) {
|
||||||
|
buffer.putInt(i*4, values(i))
|
||||||
|
assertEquals("Written value should match read value.", values(i), Utils.readInt(buffer.array, i*4))
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testCsvList() {
|
def testCsvList() {
|
||||||
val emptyString:String = ""
|
val emptyString:String = ""
|
||||||
|
|
Loading…
Reference in New Issue