Browse Source

KAFKA-631 Implement a log cleaner for Kafka. Reviewed by Neha.

pull/2/head
Jay Kreps 12 years ago
parent
commit
e7edb5e1e9
  1. 15
      config/log4j.properties
  2. 1
      config/server.properties
  3. 8
      core/src/main/scala/kafka/cluster/Partition.scala
  4. 23
      core/src/main/scala/kafka/common/OptimisticLockFailureException.scala
  5. 41
      core/src/main/scala/kafka/log/CleanerConfig.scala
  6. 30
      core/src/main/scala/kafka/log/FileMessageSet.scala
  7. 265
      core/src/main/scala/kafka/log/Log.scala
  8. 557
      core/src/main/scala/kafka/log/LogCleaner.scala
  9. 51
      core/src/main/scala/kafka/log/LogConfig.scala
  10. 120
      core/src/main/scala/kafka/log/LogManager.scala
  11. 24
      core/src/main/scala/kafka/log/LogSegment.scala
  12. 33
      core/src/main/scala/kafka/log/OffsetIndex.scala
  13. 136
      core/src/main/scala/kafka/log/OffsetMap.scala
  14. 4
      core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
  15. 118
      core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala
  16. 33
      core/src/main/scala/kafka/server/KafkaConfig.scala
  17. 51
      core/src/main/scala/kafka/server/KafkaServer.scala
  18. 103
      core/src/main/scala/kafka/server/OffsetCheckpoint.scala
  19. 14
      core/src/main/scala/kafka/server/ReplicaManager.scala
  20. 94
      core/src/main/scala/kafka/utils/FileLock.scala
  21. 5
      core/src/main/scala/kafka/utils/Logging.scala
  22. 26
      core/src/main/scala/kafka/utils/Throttler.scala
  23. 36
      core/src/main/scala/kafka/utils/Utils.scala
  24. 35
      core/src/main/scala/kafka/utils/VerifiableProperties.scala
  25. 12
      core/src/test/scala/other/kafka/StressTestLog.scala
  26. 216
      core/src/test/scala/other/kafka/TestLogCleaning.scala
  27. 3
      core/src/test/scala/other/kafka/TestLogPerformance.scala
  28. 227
      core/src/test/scala/unit/kafka/log/CleanerTest.scala
  29. 117
      core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
  30. 57
      core/src/test/scala/unit/kafka/log/LogManagerTest.scala
  31. 15
      core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
  32. 139
      core/src/test/scala/unit/kafka/log/LogTest.scala
  33. 87
      core/src/test/scala/unit/kafka/log/OffsetMapTest.scala
  34. 16
      core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
  35. 2
      core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
  36. 48
      core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
  37. 2
      core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
  38. 32
      core/src/test/scala/unit/kafka/utils/MockScheduler.scala
  39. 11
      core/src/test/scala/unit/kafka/utils/TestUtils.scala
  40. 20
      core/src/test/scala/unit/kafka/utils/UtilsTest.scala

15
config/log4j.properties

@ -36,6 +36,12 @@ log4j.appender.requestAppender.File=kafka-request.log @@ -36,6 +36,12 @@ log4j.appender.requestAppender.File=kafka-request.log
log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout
log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
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
#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender
#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender
@ -44,13 +50,18 @@ log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n @@ -44,13 +50,18 @@ log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG
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.logger.kafka.server.KafkaApis=TRACE, requestAppender
#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.logger.kafka.controller=TRACE, stateChangeAppender
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

1
config/server.properties

@ -114,3 +114,4 @@ kafka.csv.metrics.dir=/tmp/kafka_metrics @@ -114,3 +114,4 @@ kafka.csv.metrics.dir=/tmp/kafka_metrics
# Disable csv reporting by default.
kafka.csv.metrics.reporter.enabled=false
log.cleanup.policy=delete

8
core/src/main/scala/kafka/cluster/Partition.scala

@ -23,7 +23,7 @@ import kafka.api.LeaderAndIsr @@ -23,7 +23,7 @@ import kafka.api.LeaderAndIsr
import kafka.server.ReplicaManager
import com.yammer.metrics.core.Gauge
import kafka.metrics.KafkaMetricsGroup
import kafka.common.ErrorMapping
import kafka.common._
import kafka.controller.{LeaderIsrAndControllerEpoch, KafkaController}
@ -75,11 +75,11 @@ class Partition(val topic: String, @@ -75,11 +75,11 @@ class Partition(val topic: String,
case None =>
if (isReplicaLocal(replicaId)) {
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))
addReplicaIfNotExists(localReplica)
}
else {
} else {
val remoteReplica = new Replica(replicaId, this, time)
addReplicaIfNotExists(remoteReplica)
}

23
core/src/main/scala/kafka/common/OptimisticLockFailureException.scala

@ -0,0 +1,23 @@ @@ -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)

41
core/src/main/scala/kafka/log/CleanerConfig.scala

@ -0,0 +1,41 @@ @@ -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") {
}

30
core/src/main/scala/kafka/log/FileMessageSet.scala

@ -38,7 +38,7 @@ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} @@ -38,7 +38,7 @@ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
* @param isSlice Should the start and end parameters be used for slicing?
*/
@nonthreadsafe
class FileMessageSet private[kafka](val file: File,
class FileMessageSet private[kafka](@volatile var file: File,
private[log] val channel: FileChannel,
private[log] val start: Int,
private[log] val end: Int,
@ -223,14 +223,36 @@ class FileMessageSet private[kafka](val file: File, @@ -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
* given size falls on a valid message boundary.
* @param targetSize The size to truncate to.
* @return The number of bytes truncated off
*/
def truncateTo(targetSize: Int) = {
if(targetSize > sizeInBytes || targetSize < 0)
def truncateTo(targetSize: Int): Int = {
val originalSize = sizeInBytes
if(targetSize > originalSize || targetSize < 0)
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.position(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
}
}

265
core/src/main/scala/kafka/log/Log.scala

@ -49,15 +49,9 @@ import com.yammer.metrics.core.Gauge @@ -49,15 +49,9 @@ import com.yammer.metrics.core.Gauge
*/
@threadsafe
class Log(val dir: File,
val config: LogConfig,
val needsRecovery: Boolean,
val scheduler: Scheduler,
val maxSegmentSize: Int,
val maxMessageSize: Int,
val flushInterval: Int = Int.MaxValue,
val rollIntervalMs: Long = Long.MaxValue,
val needsRecovery: Boolean,
val maxIndexSize: Int = (10*1024*1024),
val indexIntervalBytes: Int = 4096,
val segmentDeleteDelayMs: Long = 60000,
time: Time = SystemTime) extends Logging with KafkaMetricsGroup {
import kafka.log.Log._
@ -73,6 +67,9 @@ class Log(val dir: File, @@ -73,6 +67,9 @@ class Log(val dir: File,
/* the actual segments of the log */
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 */
private val nextOffset: AtomicLong = new AtomicLong(activeSegment.nextOffset())
@ -90,58 +87,82 @@ class Log(val dir: File, @@ -90,58 +87,82 @@ class Log(val dir: File,
private def loadSegments(): ConcurrentNavigableMap[Long, LogSegment] = {
// open all the segments read-only
val logSegments = new ConcurrentSkipListMap[Long, LogSegment]
val ls = dir.listFiles()
if(ls != null) {
for(file <- ls if file.isFile) {
if(!file.canRead)
throw new IOException("Could not read file " + file)
val filename = file.getName
if(filename.endsWith(DeletedFileSuffix)) {
// if the file ends in .deleted, delete it
val deleted = file.delete()
if(!deleted)
warn("Attempt to delete defunct segment file %s failed.".format(filename))
} else if(filename.endsWith(IndexFileSuffix)) {
// if it is an index file, make sure it has a corresponding .log file
val logFile = new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix))
if(!logFile.exists) {
warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath))
file.delete()
}
} else if(filename.endsWith(LogFileSuffix)) {
// if 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 = indexIntervalBytes,
maxIndexSize = maxIndexSize)
if(!hasIndex) {
// this can only happen if someone manually deletes the index file
error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath))
segment.recover(maxMessageSize)
}
logSegments.put(start, segment)
// create the log directory if it doesn't exist
dir.mkdirs()
// first do a pass through the files in the log directory and remove any temporary files
// and complete any interrupted swap operations
for(file <- dir.listFiles if file.isFile) {
if(!file.canRead)
throw new IOException("Could not read file " + file)
val filename = file.getName
if(filename.endsWith(DeletedFileSuffix) || filename.endsWith(CleanedFileSuffix)) {
// if the file ends in .deleted or .cleaned, delete it
file.delete()
} else if(filename.endsWith(SwapFileSuffix)) {
// we crashed in the middle of a swap operation, to recover:
// if a log, swap it in and delete the .index file
// if an index just delete it, it will be rebuilt
val baseName = new File(Utils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
if(baseName.getPath.endsWith(IndexFileSuffix)) {
file.delete()
} else if(baseName.getPath.endsWith(LogFileSuffix)){
// delete the index
val index = new File(Utils.replaceSuffix(baseName.getPath, LogFileSuffix, IndexFileSuffix))
index.delete()
// complete the swap operation
val renamed = file.renameTo(baseName)
if(renamed)
info("Found log file %s from interrupted swap operation, repairing.".format(file.getPath))
else
throw new KafkaException("Failed to rename file %s.".format(file.getPath))
}
}
}
// 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) {
// no existing segments, create a new mutable segment beginning at offset 0
logSegments.put(0,
new LogSegment(dir = dir,
startOffset = 0,
indexIntervalBytes = indexIntervalBytes,
maxIndexSize = maxIndexSize))
indexIntervalBytes = config.indexInterval,
maxIndexSize = config.maxIndexSize))
} else {
// reset the index size of the currently active log segment to allow more entries
val active = logSegments.lastEntry.getValue
active.index.resize(maxIndexSize)
active.index.resize(config.maxIndexSize)
// run recovery on the active segment if necessary
if(needsRecovery) {
info("Recovering active segment of %s.".format(name))
active.recover(maxMessageSize)
active.recover(config.maxMessageSize)
}
}
logSegments
@ -152,6 +173,11 @@ class Log(val dir: File, @@ -152,6 +173,11 @@ class Log(val dir: File,
* Take care! this is an O(n) operation.
*/
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
@ -194,22 +220,24 @@ class Log(val dir: File, @@ -194,22 +220,24 @@ class Log(val dir: File,
val segment = maybeRoll()
if(assignOffsets) {
// assign offsets to the messageset
// assign offsets to the messageset
appendInfo.firstOffset = nextOffset.get
val offsetCounter = new AtomicLong(nextOffset.get)
validMessages = validMessages.assignOffsets(offsetCounter, appendInfo.codec)
appendInfo.lastOffset = offsetCounter.get - 1
val offset = new AtomicLong(nextOffset.get)
validMessages = validMessages.assignOffsets(offset, appendInfo.codec)
appendInfo.lastOffset = offset.get - 1
} else {
// we are taking the offsets we are given
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
trace("Appending message set to %s with offsets %d to %d.".format(name, appendInfo.firstOffset, appendInfo.lastOffset))
segment.append(appendInfo.firstOffset, validMessages)
// increment the log end offset
nextOffset.set(appendInfo.lastOffset + 1)
// maybe flush the log and index
maybeFlush(appendInfo.count)
@ -263,8 +291,8 @@ class Log(val dir: File, @@ -263,8 +291,8 @@ class Log(val dir: File,
// check the validity of the message by checking CRC and message size
val m = messageAndOffset.message
m.ensureValid()
if(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), 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), config.maxMessageSize))
messageCount += 1;
@ -372,18 +400,21 @@ class Log(val dir: File, @@ -372,18 +400,21 @@ class Log(val dir: File,
*/
private def maybeRoll(): LogSegment = {
val segment = activeSegment
if (segment.size > maxSegmentSize) {
info("Rolling %s due to full data log".format(name))
roll()
} else if (segment.size > 0 && time.milliseconds - segment.created > rollIntervalMs) {
info("Rolling %s due to time based rolling".format(name))
roll()
} else if (segment.index.isFull) {
info("Rolling %s due to full index maxIndexSize = %d, entries = %d, maxEntries = %d"
.format(name, segment.index.maxIndexSize, segment.index.entries(), segment.index.maxEntries))
if (segment.size > config.segmentSize ||
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()
} else
} else {
segment
}
}
/**
@ -412,11 +443,11 @@ class Log(val dir: File, @@ -412,11 +443,11 @@ class Log(val dir: File,
}
val segment = new LogSegment(dir,
startOffset = newOffset,
indexIntervalBytes = indexIntervalBytes,
maxIndexSize = maxIndexSize)
val prev = segments.put(segment.baseOffset, segment)
indexIntervalBytes = config.indexInterval,
maxIndexSize = config.maxIndexSize)
val prev = addSegment(segment)
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
}
}
@ -426,12 +457,12 @@ class Log(val dir: File, @@ -426,12 +457,12 @@ class Log(val dir: File,
* @param numberOfMessages The number of messages that are being appended
*/
private def maybeFlush(numberOfMessages : Int) {
if(unflushed.addAndGet(numberOfMessages) >= flushInterval)
if(unflushed.addAndGet(numberOfMessages) >= config.flushInterval)
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 = {
if (unflushed.get == 0)
@ -475,6 +506,7 @@ class Log(val dir: File, @@ -475,6 +506,7 @@ class Log(val dir: File,
activeSegment.truncateTo(targetOffset)
this.nextOffset.set(targetOffset)
}
truncates.getAndIncrement
}
}
@ -487,12 +519,12 @@ class Log(val dir: File, @@ -487,12 +519,12 @@ class Log(val dir: File,
lock synchronized {
val segmentsToDelete = logSegments.toList
segmentsToDelete.foreach(deleteSegment(_))
segments.put(newOffset,
new LogSegment(dir,
newOffset,
indexIntervalBytes = indexIntervalBytes,
maxIndexSize = maxIndexSize))
addSegment(new LogSegment(dir,
newOffset,
indexIntervalBytes = config.indexInterval,
maxIndexSize = config.maxIndexSize))
this.nextOffset.set(newOffset)
truncates.getAndIncrement
}
}
@ -511,7 +543,13 @@ class Log(val dir: File, @@ -511,7 +543,13 @@ class Log(val dir: File,
*/
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:
@ -526,37 +564,82 @@ class Log(val dir: File, @@ -526,37 +564,82 @@ class Log(val dir: File,
* @param segment The log segment to schedule for deletion
*/
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 {
segments.remove(segment.baseOffset)
val deletedLog = new File(segment.log.file.getPath + Log.DeletedFileSuffix)
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)
asyncDeleteSegment(segment)
}
}
/**
* 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
*/
object Log {
/** a log file */
val LogFileSuffix = ".log"
/** an index file */
val IndexFileSuffix = ".index"
/** a file that is scheduled to be 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

557
core/src/main/scala/kafka/log/LogCleaner.scala

@ -0,0 +1,557 @@ @@ -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
}

51
core/src/main/scala/kafka/log/LogConfig.scala

@ -0,0 +1,51 @@ @@ -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)

120
core/src/main/scala/kafka/log/LogManager.scala

@ -36,32 +36,31 @@ import kafka.server.KafkaConfig @@ -36,32 +36,31 @@ import kafka.server.KafkaConfig
* A background thread handles log retention by periodically truncating excess log segments.
*/
@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,
private val time: Time) extends Logging {
val CleanShutdownFile = ".kafka_cleanshutdown"
val LockFile = ".lock"
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 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]()
createAndValidateLogDirs(logDirs)
private var dirLocks = lockLogDirs(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:
* <ol>
@ -114,18 +113,11 @@ class LogManager(val config: KafkaConfig, @@ -114,18 +113,11 @@ class LogManager(val config: KafkaConfig,
if(dir.isDirectory){
info("Loading log '" + dir.getName + "'")
val topicPartition = parseTopicPartitionName(dir.getName)
val rollIntervalMs = logRollMsMap.get(topicPartition.topic).getOrElse(this.logRollDefaultIntervalMs)
val maxLogFileSize = logFileSizeMap.get(topicPartition.topic).getOrElse(config.logSegmentBytes)
val log = new Log(dir,
val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
val log = new Log(dir,
config,
needsRecovery,
scheduler,
maxLogFileSize,
config.messageMaxBytes,
logFlushInterval,
rollIntervalMs,
needsRecovery,
config.logIndexSizeMaxBytes,
config.logIndexIntervalBytes,
config.logDeleteDelayMs,
time)
val previous = this.logs.put(topicPartition, log)
if(previous != null)
@ -142,20 +134,41 @@ class LogManager(val config: KafkaConfig, @@ -142,20 +134,41 @@ class LogManager(val config: KafkaConfig,
def startup() {
/* Schedule the cleanup task to delete old logs */
if(scheduler != null) {
info("Starting log cleanup with a period of %d ms.".format(logCleanupIntervalMs))
scheduler.schedule("kafka-log-cleaner",
info("Starting log cleanup with a period of %d ms.".format(retentionCheckMs))
scheduler.schedule("kafka-log-retention",
cleanupLogs,
delay = InitialTaskDelayMs,
period = logCleanupIntervalMs,
period = retentionCheckMs,
TimeUnit.MILLISECONDS)
info("Starting log flusher with a default period of %d ms with the following overrides: %s."
.format(config.logFlushIntervalMs, logFlushIntervals.map(e => e._1.toString + "=" + e._2.toString).mkString(", ")))
info("Starting log flusher with a default period of %d ms.".format(flushCheckMs))
scheduler.schedule("kafka-log-flusher",
flushDirtyLogs,
delay = InitialTaskDelayMs,
period = config.logFlushSchedulerIntervalMs,
period = flushCheckMs,
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, @@ -197,18 +210,10 @@ class LogManager(val config: KafkaConfig,
val dataDir = nextLogDir()
val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition)
dir.mkdirs()
val rollIntervalMs = logRollMsMap.get(topicAndPartition.topic).getOrElse(this.logRollDefaultIntervalMs)
val maxLogFileSize = logFileSizeMap.get(topicAndPartition.topic).getOrElse(config.logSegmentBytes)
log = new Log(dir,
defaultConfig,
needsRecovery = false,
scheduler,
maxLogFileSize,
config.messageMaxBytes,
logFlushInterval,
rollIntervalMs,
needsRecovery = false,
config.logIndexSizeMaxBytes,
config.logIndexIntervalBytes,
config.logDeleteDelayMs,
time)
info("Created log for topic %s partition %d in %s.".format(topicAndPartition.topic, topicAndPartition.partition, dataDir.getAbsolutePath))
logs.put(topicAndPartition, log)
@ -242,8 +247,7 @@ class LogManager(val config: KafkaConfig, @@ -242,8 +247,7 @@ class LogManager(val config: KafkaConfig,
private def cleanupExpiredSegments(log: Log): Int = {
val startMs = time.milliseconds
val topic = parseTopicPartitionName(log.name).topic
val logCleanupThresholdMs = logRetentionMsMap.get(topic).getOrElse(this.logCleanupDefaultAgeMs)
log.deleteOldSegments(startMs - _.lastModified > logCleanupThresholdMs)
log.deleteOldSegments(startMs - _.lastModified > log.config.retentionMs)
}
/**
@ -252,10 +256,9 @@ class LogManager(val config: KafkaConfig, @@ -252,10 +256,9 @@ class LogManager(val config: KafkaConfig,
*/
private def cleanupSegmentsToMaintainSize(log: Log): Int = {
val topic = parseTopicPartitionName(log.dir.getName).topic
val maxLogRetentionSize = logRetentionSizeMap.get(topic).getOrElse(config.logRetentionBytes)
if(maxLogRetentionSize < 0 || log.size < maxLogRetentionSize)
if(log.config.retentionSize < 0 || log.size < log.config.retentionSize)
return 0
var diff = log.size - maxLogRetentionSize
var diff = log.size - log.config.retentionSize
def shouldDelete(segment: LogSegment) = {
if(diff - segment.size >= 0) {
diff -= segment.size
@ -274,7 +277,7 @@ class LogManager(val config: KafkaConfig, @@ -274,7 +277,7 @@ class LogManager(val config: KafkaConfig,
debug("Beginning log cleanup...")
var total = 0
val startMs = time.milliseconds
for(log <- allLogs) {
for(log <- allLogs; if !log.config.dedupe) {
debug("Garbage collecting '" + log.name + "'")
total += cleanupExpiredSegments(log) + cleanupSegmentsToMaintainSize(log)
}
@ -282,23 +285,6 @@ class LogManager(val config: KafkaConfig, @@ -282,23 +285,6 @@ class LogManager(val config: KafkaConfig,
(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
*/
@ -312,13 +298,9 @@ class LogManager(val config: KafkaConfig, @@ -312,13 +298,9 @@ class LogManager(val config: KafkaConfig,
for ((topicAndPartition, log) <- logs) {
try {
val timeSinceLastFlush = time.milliseconds - log.lastFlushTime
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 +
debug("Checking if flush is needed on " + topicAndPartition.topic + " flush interval " + log.config.flushMs +
" last flushed " + log.lastFlushTime + " time since last flush: " + timeSinceLastFlush)
if(timeSinceLastFlush >= logFlushInterval)
if(timeSinceLastFlush >= log.config.flushMs)
log.flush
} catch {
case e =>

24
core/src/main/scala/kafka/log/LogSegment.scala

@ -79,7 +79,7 @@ class LogSegment(val log: FileMessageSet, @@ -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.
*/
@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)
log.searchFor(offset, max(mapping.position, startingFilePosition))
}
@ -168,18 +168,20 @@ class LogSegment(val log: FileMessageSet, @@ -168,18 +168,20 @@ class LogSegment(val log: FileMessageSet,
* 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.
* @param offset The offset to truncate to
* @return The number of log bytes truncated
*/
@nonthreadsafe
def truncateTo(offset: Long) {
def truncateTo(offset: Long): Int = {
val mapping = translateOffset(offset)
if(mapping == null)
return
return 0
index.truncateTo(offset)
// after truncation, reset and allocate more space for the (new currently active) index
index.resize(index.maxIndexSize)
log.truncateTo(mapping.position)
if (log.sizeInBytes == 0)
val bytesTruncated = log.truncateTo(mapping.position)
if(log.sizeInBytes == 0)
created = time.milliseconds
bytesTruncated
}
/**
@ -210,6 +212,18 @@ class LogSegment(val log: FileMessageSet, @@ -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
*/

33
core/src/main/scala/kafka/log/OffsetIndex.scala

@ -49,7 +49,7 @@ import kafka.utils._ @@ -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
* 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 */
private var mmap: MappedByteBuffer =
@ -83,20 +83,15 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = @@ -83,20 +83,15 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
Utils.swallow(raf.close())
}
}
info("Created index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d"
.format(file.getAbsolutePath, maxEntries, maxIndexSize, entries(), lastOffset))
/**
* The maximum number of eight-byte entries this index can hold
*/
def maxEntries = mmap.limit / 8
/* the number of eight-byte entries currently in the index */
private var size = new AtomicInteger(mmap.position / 8)
/* the last offset in the index */
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
@ -109,6 +104,11 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = @@ -109,6 +104,11 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
}
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
@ -284,11 +284,26 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = @@ -284,11 +284,26 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
/** The number of entries in this index */
def entries() = size.get
/**
* The number of bytes actually used by this index
*/
def sizeInBytes() = 8 * entries
/** Close the index */
def close() {
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.
* E.g. roundToExactMultiple(67, 8) == 64

136
core/src/main/scala/kafka/log/OffsetMap.scala

@ -0,0 +1,136 @@ @@ -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)
}
}

4
core/src/main/scala/kafka/message/ByteBufferMessageSet.scala

@ -73,7 +73,7 @@ object ByteBufferMessageSet { @@ -73,7 +73,7 @@ object ByteBufferMessageSet {
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.putInt(message.size)
buffer.put(message.buffer)
@ -150,7 +150,7 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message @@ -150,7 +150,7 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message
return allDone()
val offset = topIter.getLong()
val size = topIter.getInt()
if(size < 0)
if(size < 0 || size < Message.MinHeaderSize)
throw new InvalidMessageException("Message found with corrupt size (" + size + ")")
// we have an incomplete message

118
core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala

@ -1,118 +0,0 @@ @@ -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()
}
}
}

33
core/src/main/scala/kafka/server/KafkaConfig.scala

@ -103,9 +103,37 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro @@ -103,9 +103,37 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
/* the maximum size of the log for some specific topic before deleting it */
val logRetentionBytesPerTopicMap = props.getMap("log.retention.bytes.per.topic", _.toLong > 0).mapValues(_.toLong)
/* the frequency in minutes that the log cleaner checks whether any log is eligible for deletion */
val 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 */
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 @@ -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 */
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))
/* 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 */

51
core/src/main/scala/kafka/server/KafkaServer.scala

@ -18,9 +18,12 @@ @@ -18,9 +18,12 @@
package kafka.server
import kafka.network.SocketServer
import kafka.log.LogConfig
import kafka.log.CleanerConfig
import kafka.log.LogManager
import kafka.utils._
import java.util.concurrent._
import java.io.File
import atomic.AtomicBoolean
import org.I0Itec.zkclient.ZkClient
import kafka.controller.{ControllerStats, KafkaController}
@ -56,9 +59,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg @@ -56,9 +59,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
kafkaScheduler.startup()
/* start log manager */
logManager = new LogManager(config,
kafkaScheduler,
time)
logManager = createLogManager(config)
logManager.startup()
socketServer = new SocketServer(config.brokerId,
@ -138,6 +139,50 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg @@ -138,6 +139,50 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
def awaitShutdown(): Unit = shutdownLatch.await()
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)
}
}

103
core/src/main/scala/kafka/server/OffsetCheckpoint.scala

@ -0,0 +1,103 @@ @@ -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()
}
}
}
}

14
core/src/main/scala/kafka/server/ReplicaManager.scala

@ -20,6 +20,7 @@ import kafka.cluster.{Broker, Partition, Replica} @@ -20,6 +20,7 @@ import kafka.cluster.{Broker, Partition, Replica}
import collection._
import mutable.HashMap
import org.I0Itec.zkclient.ZkClient
import java.io.{File, IOException}
import java.util.concurrent.atomic.AtomicBoolean
import kafka.utils._
import kafka.log.LogManager
@ -33,6 +34,7 @@ import kafka.controller.KafkaController @@ -33,6 +34,7 @@ import kafka.controller.KafkaController
object ReplicaManager {
val UnknownLogEndOffset = -1L
val HighWatermarkFilename = "replication-offset-checkpoint"
}
class ReplicaManager(val config: KafkaConfig,
@ -48,7 +50,7 @@ class ReplicaManager(val config: KafkaConfig, @@ -48,7 +50,7 @@ class ReplicaManager(val config: KafkaConfig,
val replicaFetcherManager = new ReplicaFetcherManager(config, this)
this.logIdent = "Replica Manager on Broker " + config.brokerId + ": "
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(
"LeaderCount",
@ -67,7 +69,7 @@ class ReplicaManager(val config: KafkaConfig, @@ -67,7 +69,7 @@ class ReplicaManager(val config: KafkaConfig,
}
)
val isrExpandRate = newMeter("IsrExpandsPerSec", "expands", TimeUnit.SECONDS)
val isrShrinkRate = newMeter("ISRShrinksPerSec", "shrinks", TimeUnit.SECONDS)
val isrShrinkRate = newMeter("IsrShrinksPerSec", "shrinks", TimeUnit.SECONDS)
def startHighWaterMarksCheckPointThread() = {
@ -265,7 +267,13 @@ class ReplicaManager(val config: KafkaConfig, @@ -265,7 +267,13 @@ class ReplicaManager(val config: KafkaConfig,
val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParent)
for((dir, reps) <- replicasByDir) {
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)
}
}
}

94
core/src/main/scala/kafka/utils/FileLock.scala

@ -9,56 +9,56 @@ import java.nio.channels._ @@ -9,56 +9,56 @@ import java.nio.channels._
* The given path will be created and opened if it doesn't exist.
*/
class FileLock(val file: File) extends Logging {
file.createNewFile()
private val channel = new RandomAccessFile(file, "rw").getChannel()
private var flock: java.nio.channels.FileLock = null
/**
* Lock the file or throw an exception if the lock is already held
*/
def lock() {
this synchronized {
trace("Acquiring lock on " + file.getAbsolutePath)
flock = channel.lock()
}
file.createNewFile() // create the file if it doesn't exist
private val channel = new RandomAccessFile(file, "rw").getChannel()
private var flock: java.nio.channels.FileLock = null
/**
* Lock the file or throw an exception if the lock is already held
*/
def lock() {
this synchronized {
trace("Acquiring lock on " + file.getAbsolutePath)
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
*/
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
}
}
/**
* Unlock the lock if it is held
*/
def unlock() {
this synchronized {
trace("Releasing lock on " + file.getAbsolutePath)
if(flock != null)
flock.release()
}
}
/**
* Unlock the lock if it is held
*/
def unlock() {
this synchronized {
trace("Releasing lock on " + file.getAbsolutePath)
if(flock != null)
flock.release()
}
/**
* Destroy this lock, closing the associated FileChannel
*/
def destroy() = {
this synchronized {
unlock()
channel.close()
}
}
/**
* Destroy this lock, closing the associated FileChannel
*/
def destroy() = {
this synchronized {
unlock()
channel.close()
}
}
}

5
core/src/main/scala/kafka/utils/Logging.scala

@ -23,12 +23,13 @@ trait Logging { @@ -23,12 +23,13 @@ trait Logging {
val loggerName = this.getClass.getName
lazy val logger = Logger.getLogger(loggerName)
protected var logIdent = ""
protected var logIdent: String = null
// Force initialization to register Log4jControllerMBean
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 = {
if (logger.isTraceEnabled())

26
core/src/main/scala/kafka/utils/Throttler.scala

@ -32,20 +32,14 @@ import scala.math._ @@ -32,20 +32,14 @@ import scala.math._
*/
@threadsafe
class Throttler(val desiredRatePerSec: Double,
val checkIntervalMs: Long,
val throttleDown: Boolean,
val time: Time) extends Logging {
val checkIntervalMs: Long = 100L,
val throttleDown: Boolean = true,
val time: Time = SystemTime) extends Logging {
private val lock = new Object
private var periodStartNs: Long = time.nanoseconds
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) {
lock synchronized {
observedSoFar += observed
@ -58,11 +52,11 @@ class Throttler(val desiredRatePerSec: Double, @@ -58,11 +52,11 @@ class Throttler(val desiredRatePerSec: Double,
val needAdjustment = !(throttleDown ^ (rateInSecs > desiredRatePerSec))
if(needAdjustment) {
// 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 sleepTime = round(observedSoFar / desiredRateMs - elapsedMs)
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)
}
}
@ -76,20 +70,20 @@ class Throttler(val desiredRatePerSec: Double, @@ -76,20 +70,20 @@ class Throttler(val desiredRatePerSec: Double,
object Throttler {
val DefaultCheckIntervalMs = 100L
def main(args: Array[String]) {
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 total = 0
while(true) {
val value = rand.nextInt(1000)
Thread.sleep(1)
throttler.maybeThrottle(value)
total += value
val now = System.currentTimeMillis
if(now - start >= 1000) {
println(total)
if(now - start >= interval) {
println(total / (interval/1000.0))
start = now
total = 0
}

36
core/src/main/scala/kafka/utils/Utils.scala

@ -27,6 +27,7 @@ import scala.collection._ @@ -27,6 +27,7 @@ import scala.collection._
import scala.collection.mutable
import java.util.Properties
import kafka.common.KafkaException
import kafka.common.KafkaStorageException
/**
@ -159,7 +160,7 @@ object Utils extends Logging { @@ -159,7 +160,7 @@ object Utils extends Logging {
* @param log The log method to use for logging. E.g. logger.warn
* @param action The action to execute
*/
def swallow(log: (Object, Throwable) => Unit, action: => Unit) = {
def swallow(log: (Object, Throwable) => Unit, action: => Unit) {
try {
action
} catch {
@ -528,4 +529,37 @@ object Utils extends Logging { @@ -528,4 +529,37 @@ object Utils extends Logging {
*/
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)
}
}

35
core/src/main/scala/kafka/utils/VerifiableProperties.scala

@ -38,10 +38,7 @@ class VerifiableProperties(val props: Properties) extends Logging { @@ -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
*/
def getInt(name: String): Int = {
require(containsKey(name), "Missing required property '" + name + "'")
return getInt(name, -1)
}
def getInt(name: String): Int = getString(name).toInt
def getIntInRange(name: String, range: (Int, Int)): Int = {
require(containsKey(name), "Missing required property '" + name + "'")
@ -92,10 +89,7 @@ class VerifiableProperties(val props: Properties) extends Logging { @@ -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
*/
def getLong(name: String): Long = {
require(containsKey(name), "Missing required property '" + name + "'")
return getLong(name, -1)
}
def getLong(name: String): Long = getString(name).toLong
/**
* Read an long from the properties instance
@ -124,6 +118,26 @@ class VerifiableProperties(val props: Properties) extends Logging { @@ -124,6 +118,26 @@ class VerifiableProperties(val props: Properties) extends Logging {
require(v >= range._1 && v <= range._2, name + " has value " + v + " which is not in the range " + range + ".")
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
@ -140,6 +154,8 @@ class VerifiableProperties(val props: Properties) extends Logging { @@ -140,6 +154,8 @@ class VerifiableProperties(val props: Properties) extends Logging {
v.toBoolean
}
}
def getBoolean(name: String) = getString(name).toBoolean
/**
* 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 { @@ -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, ...
*/
def getMap(name: String, valid: String => Boolean): Map[String, String] = {
def getMap(name: String, valid: String => Boolean = s => true): Map[String, String] = {
try {
val m = Utils.parseCsvMap(getString(name, ""))
m.foreach {
@ -189,4 +205,5 @@ class VerifiableProperties(val props: Properties) extends Logging { @@ -189,4 +205,5 @@ class VerifiableProperties(val props: Properties) extends Logging {
}
override def toString(): String = props.toString
}

12
core/src/test/scala/other/kafka/StressTestLog.scala

@ -34,15 +34,11 @@ object StressTestLog { @@ -34,15 +34,11 @@ object StressTestLog {
val dir = TestUtils.tempDir()
val time = new MockTime
val log = new Log(dir = dir,
scheduler = time.scheduler,
maxSegmentSize = 64*1024*1024,
maxMessageSize = Int.MaxValue,
flushInterval = Int.MaxValue,
rollIntervalMs = Long.MaxValue,
config = LogConfig(segmentSize = 64*1024*1024,
maxMessageSize = Int.MaxValue,
maxIndexSize = 1024*1024),
needsRecovery = false,
maxIndexSize = 1024*1024,
indexIntervalBytes = 4096,
segmentDeleteDelayMs = 60000,
scheduler = time.scheduler,
time = time)
val writer = new WriterThread(log)
writer.start()

216
core/src/test/scala/other/kafka/TestLogCleaning.scala

@ -0,0 +1,216 @@ @@ -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
}
}

3
core/src/test/scala/other/kafka/TestLogPerformance.scala

@ -34,7 +34,8 @@ object TestLogPerformance { @@ -34,7 +34,8 @@ object TestLogPerformance {
val config = new KafkaConfig(props)
val dir = TestUtils.tempDir()
val scheduler = new KafkaScheduler(1)
val log = new Log(dir, scheduler, 50*1024*1024, config.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)
new java.util.Random().nextBytes(bytes)
val message = new Message(bytes)

227
core/src/test/scala/unit/kafka/log/CleanerTest.scala

@ -0,0 +1,227 @@ @@ -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
}

117
core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala

@ -0,0 +1,117 @@ @@ -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)
}
}

57
core/src/test/scala/unit/kafka/log/LogManagerTest.scala

@ -29,21 +29,18 @@ class LogManagerTest extends JUnit3Suite { @@ -29,21 +29,18 @@ class LogManagerTest extends JUnit3Suite {
val time: MockTime = new MockTime()
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 logManager: LogManager = null
var config: KafkaConfig = null
val name = "kafka"
val veryLargeLogFlushInterval = 10000000L
val cleanerConfig = CleanerConfig(enableCleaner = false)
override def setUp() {
super.setUp()
config = new KafkaConfig(TestUtils.createBrokerConfig(0, -1)) {
override val logSegmentBytes = 1024
override val logFlushIntervalMessages = 10000
override val logRetentionHours = maxLogAgeHours
}
logManager = new LogManager(config, time.scheduler, time)
logDir = TestUtils.tempDir()
logManager = new LogManager(Array(logDir), Map(), logConfig, cleanerConfig, 1000L, 1000L, time.scheduler, time)
logManager.startup
logDir = logManager.logDirs(0)
}
@ -62,7 +59,7 @@ class LogManagerTest extends JUnit3Suite { @@ -62,7 +59,7 @@ class LogManagerTest extends JUnit3Suite {
@Test
def testCreateLog() {
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)
log.append(TestUtils.singleMessageSet("test".getBytes()))
}
@ -74,7 +71,7 @@ class LogManagerTest extends JUnit3Suite { @@ -74,7 +71,7 @@ class LogManagerTest extends JUnit3Suite {
def testGetNonExistentLog() {
val log = logManager.getLog(name, 0)
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)
}
@ -94,9 +91,9 @@ class LogManagerTest extends JUnit3Suite { @@ -94,9 +91,9 @@ class LogManagerTest extends JUnit3Suite {
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)
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("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).sizeInBytes)
@ -116,14 +113,10 @@ class LogManagerTest extends JUnit3Suite { @@ -116,14 +113,10 @@ class LogManagerTest extends JUnit3Suite {
@Test
def testCleanupSegmentsToMaintainSize() {
val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes
val props = TestUtils.createBrokerConfig(0, -1)
logManager.shutdown()
config = new KafkaConfig(props) {
override val logSegmentBytes = (10 * (setSize - 1)) // each segment will be 10 messages
override val logRetentionBytes = (5 * 10 * setSize + 10).asInstanceOf[Long]
override val logRollHours = maxRollInterval
}
logManager = new LogManager(config, time.scheduler, time)
val config = logConfig.copy(segmentSize = 10 * (setSize - 1), retentionSize = 5L * 10L * setSize + 10L)
logManager = new LogManager(Array(logDir), Map(), config, cleanerConfig, 1000L, 1000L, time.scheduler, time)
logManager.startup
// create a log
@ -138,13 +131,12 @@ class LogManagerTest extends JUnit3Suite { @@ -138,13 +131,12 @@ class LogManagerTest extends JUnit3Suite {
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.logSegmentBytes, log.numberOfSegments)
assertEquals("Check we have the expected number of segments.", numMessages * setSize / config.segmentSize, log.numberOfSegments)
// this cleanup shouldn't find any expired segments but should delete some to reduce size
time.sleep(logManager.InitialTaskDelayMs)
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("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).sizeInBytes)
try {
@ -162,14 +154,9 @@ class LogManagerTest extends JUnit3Suite { @@ -162,14 +154,9 @@ class LogManagerTest extends JUnit3Suite {
*/
@Test
def testTimeBasedFlush() {
val props = TestUtils.createBrokerConfig(0, -1)
logManager.shutdown()
config = new KafkaConfig(props) {
override val logFlushSchedulerIntervalMs = 1000
override val logFlushIntervalMs = 1000
override val logFlushIntervalMessages = Int.MaxValue
}
logManager = new LogManager(config, time.scheduler, time)
val config = logConfig.copy(flushMs = 1000)
logManager = new LogManager(Array(logDir), Map(), config, cleanerConfig, 1000L, 1000L, time.scheduler, time)
logManager.startup
val log = logManager.getOrCreateLog(name, 0)
val lastFlush = log.lastFlushTime
@ -187,13 +174,11 @@ class LogManagerTest extends JUnit3Suite { @@ -187,13 +174,11 @@ class LogManagerTest extends JUnit3Suite {
@Test
def testLeastLoadedAssignment() {
// create a log manager with multiple data directories
val props = TestUtils.createBrokerConfig(0, -1)
val dirs = Seq(TestUtils.tempDir().getAbsolutePath,
TestUtils.tempDir().getAbsolutePath,
TestUtils.tempDir().getAbsolutePath)
props.put("log.dirs", dirs.mkString(","))
val dirs = Array(TestUtils.tempDir(),
TestUtils.tempDir(),
TestUtils.tempDir())
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
for(partition <- 0 until 20) {
@ -209,7 +194,7 @@ class LogManagerTest extends JUnit3Suite { @@ -209,7 +194,7 @@ class LogManagerTest extends JUnit3Suite {
*/
def testTwoLogManagersUsingSameDirFails() {
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")
} catch {
case e: KafkaException => // this is good

15
core/src/test/scala/unit/kafka/log/LogSegmentTest.scala

@ -160,6 +160,21 @@ class LogSegmentTest extends JUnit3Suite { @@ -160,6 +160,21 @@ class LogSegmentTest extends JUnit3Suite {
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,
* and recover the segment, the entries should all be readable.

139
core/src/test/scala/unit/kafka/log/LogTest.scala

@ -34,6 +34,7 @@ class LogTest extends JUnitSuite { @@ -34,6 +34,7 @@ class LogTest extends JUnitSuite {
var logDir: File = null
val time = new MockTime
var config: KafkaConfig = null
val logConfig = LogConfig()
@Before
def setUp() {
@ -61,12 +62,15 @@ class LogTest extends JUnitSuite { @@ -61,12 +62,15 @@ class LogTest extends JUnitSuite {
@Test
def testTimeBasedLogRoll() {
val set = TestUtils.singleMessageSet("test".getBytes())
val rollMs = 1 * 60 * 60L
val time: MockTime = new MockTime()
// create a log
val log = new Log(logDir, time.scheduler, 1000, config.messageMaxBytes, 1000, rollMs, needsRecovery = false, time = time)
time.sleep(rollMs + 1)
val log = new Log(logDir,
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
log.append(set)
@ -76,13 +80,13 @@ class LogTest extends JUnitSuite { @@ -76,13 +80,13 @@ class LogTest extends JUnitSuite {
assertEquals("There should still be exactly one segment.", 1, log.numberOfSegments)
for(numSegments <- 2 until 4) {
time.sleep(rollMs + 1)
time.sleep(log.config.segmentMs + 1)
log.append(set)
assertEquals("Changing time beyond rollMs and appending should create a new segment.", numSegments, log.numberOfSegments)
}
val numSegments = log.numberOfSegments
time.sleep(rollMs + 1)
time.sleep(log.config.segmentMs + 1)
log.append(new ByteBufferMessageSet())
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 { @@ -95,10 +99,10 @@ class LogTest extends JUnitSuite {
val set = TestUtils.singleMessageSet("test".getBytes)
val setSize = set.sizeInBytes
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
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)
// segments expire in size
@ -114,7 +118,7 @@ class LogTest extends JUnitSuite { @@ -114,7 +118,7 @@ class LogTest extends JUnitSuite {
@Test
def testLoadEmptyLog() {
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))
}
@ -123,7 +127,7 @@ class LogTest extends JUnitSuite { @@ -123,7 +127,7 @@ class LogTest extends JUnitSuite {
*/
@Test
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
for(i <- 0 until messages.length)
@ -142,7 +146,7 @@ class LogTest extends JUnitSuite { @@ -142,7 +146,7 @@ class LogTest extends JUnitSuite {
*/
@Test
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 messages = messageIds.map(id => new Message(id.toString.getBytes))
@ -165,7 +169,7 @@ class LogTest extends JUnitSuite { @@ -165,7 +169,7 @@ class LogTest extends JUnitSuite {
*/
@Test
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
while(log.numberOfSegments == 1)
@ -185,7 +189,7 @@ class LogTest extends JUnitSuite { @@ -185,7 +189,7 @@ class LogTest extends JUnitSuite {
@Test
def testReadOutOfRange() {
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)
try {
log.read(0, 1024)
@ -208,7 +212,7 @@ class LogTest extends JUnitSuite { @@ -208,7 +212,7 @@ class LogTest extends JUnitSuite {
@Test
def testLogRolls() {
/* 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 messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes))
val offsets = messageSets.map(log.append(_).firstOffset)
@ -232,8 +236,8 @@ class LogTest extends JUnitSuite { @@ -232,8 +236,8 @@ class LogTest extends JUnitSuite {
@Test
def testCompressedMessages() {
/* 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 */
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)))
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes)))
@ -255,7 +259,7 @@ class LogTest extends JUnitSuite { @@ -255,7 +259,7 @@ class LogTest extends JUnitSuite {
for(messagesToAppend <- List(0, 1, 25)) {
logDir.mkdirs()
// first test a log segment starting at 0
val log = new Log(logDir, time.scheduler, 100, config.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)
log.append(TestUtils.singleMessageSet(i.toString.getBytes))
@ -289,7 +293,7 @@ class LogTest extends JUnitSuite { @@ -289,7 +293,7 @@ class LogTest extends JUnitSuite {
// append messages to log
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
log.append(first)
@ -311,7 +315,8 @@ class LogTest extends JUnitSuite { @@ -311,7 +315,8 @@ class LogTest extends JUnitSuite {
val messageSize = 100
val segmentSize = 7 * 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)
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)
@ -319,15 +324,14 @@ class LogTest extends JUnitSuite { @@ -319,15 +324,14 @@ class LogTest extends JUnitSuite {
val numIndexEntries = log.activeSegment.index.entries
log.close()
// test non-recovery case
log = new Log(logDir, time.scheduler, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, indexIntervalBytes = indexInterval, maxIndexSize = 4096)
log = new Log(logDir, config, needsRecovery = false, time.scheduler, time)
assertEquals("Should have %d messages when log is reopened w/o recovery".format(numMessages), numMessages, log.logEndOffset)
assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset)
assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries)
log.close()
// test recovery case
log = new Log(logDir, time.scheduler, segmentSize, config.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 same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset)
assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries)
@ -341,7 +345,8 @@ class LogTest extends JUnitSuite { @@ -341,7 +345,8 @@ class LogTest extends JUnitSuite {
def testIndexRebuild() {
// publish the messages and close the log
val numMessages = 200
var log = new Log(logDir, time.scheduler, 200, config.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)
log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10)))
val indexFiles = log.logSegments.map(_.index.file)
@ -351,8 +356,7 @@ class LogTest extends JUnitSuite { @@ -351,8 +356,7 @@ class LogTest extends JUnitSuite {
indexFiles.foreach(_.delete())
// 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)
for(i <- 0 until numMessages)
assertEquals(i, log.read(i, 100, None).head.offset)
@ -367,10 +371,10 @@ class LogTest extends JUnitSuite { @@ -367,10 +371,10 @@ class LogTest extends JUnitSuite {
val set = TestUtils.singleMessageSet("test".getBytes())
val setSize = set.sizeInBytes
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
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)
for (i<- 1 to msgPerSeg)
@ -421,8 +425,9 @@ class LogTest extends JUnitSuite { @@ -421,8 +425,9 @@ class LogTest extends JUnitSuite {
val set = TestUtils.singleMessageSet("test".getBytes())
val setSize = set.sizeInBytes
val msgPerSeg = 10
val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
val log = new Log(logDir, time.scheduler, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time)
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
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)
for (i<- 1 to msgPerSeg)
log.append(set)
@ -430,10 +435,10 @@ class LogTest extends JUnitSuite { @@ -430,10 +435,10 @@ class LogTest extends JUnitSuite {
for (i<- 1 to msgPerSeg)
log.append(set)
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)
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)
log.append(set)
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
@ -449,12 +454,12 @@ class LogTest extends JUnitSuite { @@ -449,12 +454,12 @@ class LogTest extends JUnitSuite {
val set = TestUtils.singleMessageSet("test".getBytes())
val log = new Log(logDir,
logConfig.copy(segmentSize = set.sizeInBytes * 5,
maxIndexSize = 1000,
indexInterval = 1),
needsRecovery = false,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 1,
needsRecovery = false)
time)
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)
@ -472,27 +477,26 @@ class LogTest extends JUnitSuite { @@ -472,27 +477,26 @@ class LogTest extends JUnitSuite {
@Test
def testReopenThenTruncate() {
val set = TestUtils.singleMessageSet("test".getBytes())
val config = logConfig.copy(segmentSize = set.sizeInBytes * 5,
maxIndexSize = 1000,
indexInterval = 10000)
// create a log
var log = new Log(logDir,
config,
needsRecovery = true,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
needsRecovery = true)
time)
// add enough messages to roll over several segments then close and re-open and attempt to truncate
for(i <- 0 until 100)
log.append(set)
log.close()
log = new Log(logDir,
config,
needsRecovery = true,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
needsRecovery = true)
time)
log.truncateTo(3)
assertEquals("All but one segment should be deleted.", 1, log.numberOfSegments)
assertEquals("Log end offset should be 3.", 3, log.logEndOffset)
@ -505,14 +509,15 @@ class LogTest extends JUnitSuite { @@ -505,14 +509,15 @@ class LogTest extends JUnitSuite {
def testAsyncDelete() {
val set = TestUtils.singleMessageSet("test".getBytes())
val asyncDeleteMs = 1000
val log = new Log(logDir,
val config = logConfig.copy(segmentSize = set.sizeInBytes * 5,
fileDeleteDelayMs = asyncDeleteMs,
maxIndexSize = 1000,
indexInterval = 10000)
val log = new Log(logDir,
config,
needsRecovery = true,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
segmentDeleteDelayMs = asyncDeleteMs,
needsRecovery = true)
time)
// append some messages to create some segments
for(i <- 0 until 100)
@ -520,15 +525,20 @@ class LogTest extends JUnitSuite { @@ -520,15 +525,20 @@ class LogTest extends JUnitSuite {
// files should be renamed
val segments = log.logSegments.toArray
val oldFiles = segments.map(_.log.file) ++ segments.map(_.index.file)
log.deleteOldSegments((s) => true)
assertEquals("Only one segment should remain.", 1, log.numberOfSegments)
val renamed = segments.map(segment => new File(segment.log.file.getPath + Log.DeletedFileSuffix))
assertTrue("Files should all be renamed to .deleted.", renamed.forall(_.exists))
assertTrue("All log and index files should end in .deleted", segments.forall(_.log.file.getName.endsWith(Log.DeletedFileSuffix)) &&
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
val deletedFiles = segments.map(_.log.file) ++ segments.map(_.index.file)
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 { @@ -537,13 +547,12 @@ class LogTest extends JUnitSuite {
@Test
def testOpenDeletesObsoleteFiles() {
val set = TestUtils.singleMessageSet("test".getBytes())
var log = new Log(logDir,
val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, maxIndexSize = 1000)
var log = new Log(logDir,
config,
needsRecovery = false,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
needsRecovery = false)
time)
// append some messages to create some segments
for(i <- 0 until 100)
@ -553,12 +562,10 @@ class LogTest extends JUnitSuite { @@ -553,12 +562,10 @@ class LogTest extends JUnitSuite {
log.close()
log = new Log(logDir,
config,
needsRecovery = false,
time.scheduler,
maxSegmentSize = set.sizeInBytes * 5,
maxMessageSize = config.messageMaxBytes,
maxIndexSize = 1000,
indexIntervalBytes = 10000,
needsRecovery = false)
time)
assertEquals("The deleted segments should be gone.", 1, log.numberOfSegments)
}

87
core/src/test/scala/unit/kafka/log/OffsetMapTest.scala

@ -0,0 +1,87 @@ @@ -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
core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala

@ -16,13 +16,14 @@ @@ -16,13 +16,14 @@
*/
package kafka.server
import kafka.log.LogManager
import kafka.log._
import java.io.File
import org.I0Itec.zkclient.ZkClient
import org.scalatest.junit.JUnit3Suite
import org.easymock.EasyMock
import org.junit._
import org.junit.Assert._
import kafka.common.KafkaException
import kafka.common._
import kafka.cluster.Replica
import kafka.utils._
@ -30,7 +31,14 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { @@ -30,7 +31,14 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_))
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
def teardown() {
@ -133,7 +141,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { @@ -133,7 +141,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
}
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)
}
}

2
core/src/test/scala/unit/kafka/server/LogOffsetTest.scala

@ -205,7 +205,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -205,7 +205,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
props.put("enable.zookeeper", "false")
props.put("num.partitions", "20")
props.put("log.retention.hours", "10")
props.put("log.cleanup.interval.mins", "5")
props.put("log.retention.check.interval.ms", (5*1000*60).toString)
props.put("log.segment.bytes", logSize.toString)
props.put("zk.connect", zkConnect.toString)
props

48
core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala

@ -18,11 +18,13 @@ package kafka.server @@ -18,11 +18,13 @@ package kafka.server
import org.scalatest.junit.JUnit3Suite
import org.junit.Assert._
import java.io.File
import kafka.admin.CreateTopicCommand
import kafka.utils.TestUtils._
import kafka.utils.IntEncoder
import kafka.utils.{Utils, TestUtils}
import kafka.zk.ZooKeeperTestHarness
import kafka.common._
import kafka.producer.{ProducerConfig, KeyedMessage, Producer}
class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
@ -44,8 +46,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -44,8 +46,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
val message = "hello"
var producer: Producer[Int, String] = null
var hwFile1: HighwaterMarkCheckpoint = new HighwaterMarkCheckpoint(configProps1.logDirs(0))
var hwFile2: HighwaterMarkCheckpoint = new HighwaterMarkCheckpoint(configProps2.logDirs(0))
var hwFile1: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename))
var hwFile2: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps2.logDirs(0), ReplicaManager.HighWatermarkFilename))
var servers: Seq[KafkaServer] = Seq.empty[KafkaServer]
val producerProps = getProducerConfig(TestUtils.getBrokerListStrFromConfigs(configs), 64*1024, 100000, 10000)
@ -80,9 +82,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -80,9 +82,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
servers.foreach(server => server.replicaManager.checkpointHighWatermarks())
producer.close()
val leaderHW = hwFile1.read(topic, 0)
val leaderHW = hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)
assertEquals(numMessages, leaderHW)
val followerHW = hwFile2.read(topic, 0)
val followerHW = hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)
assertEquals(numMessages, followerHW)
servers.foreach(server => { server.shutdown(); Utils.rm(server.config.logDirs(0))})
}
@ -104,7 +106,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -104,7 +106,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
// 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))
assertEquals(0L, hwFile1.read(topic, 0))
assertEquals(0L, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
sendMessages(1)
Thread.sleep(1000)
@ -112,7 +114,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -112,7 +114,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
// kill the server hosting the preferred replica
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
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
@ -125,10 +127,10 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -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",
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
server2.shutdown()
assertEquals(hw, hwFile2.read(topic, 0))
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
server2.startup()
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
@ -144,8 +146,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -144,8 +146,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
// shutdown the servers to allow the hw to be checkpointed
servers.foreach(server => server.shutdown())
producer.close()
assertEquals(hw, hwFile1.read(topic, 0))
assertEquals(hw, hwFile2.read(topic, 0))
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
servers.foreach(server => Utils.rm(server.config.logDirs))
}
@ -155,8 +157,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -155,8 +157,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
server2 = TestUtils.createServer(configs.last)
servers ++= List(server1, server2)
hwFile1 = new HighwaterMarkCheckpoint(server1.config.logDirs(0))
hwFile2 = new HighwaterMarkCheckpoint(server2.config.logDirs(0))
hwFile1 = new OffsetCheckpoint(new File(server1.config.logDirs(0), ReplicaManager.HighWatermarkFilename))
hwFile2 = new OffsetCheckpoint(new File(server2.config.logDirs(0), ReplicaManager.HighWatermarkFilename))
producer = new Producer[Int, String](new ProducerConfig(producerProps))
@ -176,9 +178,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -176,9 +178,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
// shutdown the servers to allow the hw to be checkpointed
servers.foreach(server => server.shutdown())
producer.close()
val leaderHW = hwFile1.read(topic, 0)
val leaderHW = hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)
assertEquals(hw, leaderHW)
val followerHW = hwFile2.read(topic, 0)
val followerHW = hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)
assertEquals(hw, followerHW)
servers.foreach(server => Utils.rm(server.config.logDirs))
}
@ -189,8 +191,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -189,8 +191,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
server2 = TestUtils.createServer(configs.last)
servers ++= List(server1, server2)
hwFile1 = new HighwaterMarkCheckpoint(server1.config.logDirs(0))
hwFile2 = new HighwaterMarkCheckpoint(server2.config.logDirs(0))
hwFile1 = new OffsetCheckpoint(new File(server1.config.logDirs(0), ReplicaManager.HighWatermarkFilename))
hwFile2 = new OffsetCheckpoint(new File(server2.config.logDirs(0), ReplicaManager.HighWatermarkFilename))
producer = new Producer[Int, String](new ProducerConfig(producerProps))
@ -212,21 +214,21 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -212,21 +214,21 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
// kill the server hosting the preferred replica
server1.shutdown()
server2.shutdown()
assertEquals(hw, hwFile1.read(topic, 0))
assertEquals(hw, hwFile2.read(topic, 0))
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
server2.startup()
// check if leader moves to the other server
leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader)
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
server1.startup()
assertEquals(hw, hwFile1.read(topic, 0))
assertEquals(hw, hwFile2.read(topic, 0))
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
sendMessages(2)
hw += 2
@ -237,8 +239,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -237,8 +239,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
// shutdown the servers to allow the hw to be checkpointed
servers.foreach(server => server.shutdown())
producer.close()
assertEquals(hw, hwFile1.read(topic, 0))
assertEquals(hw, hwFile2.read(topic, 0))
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L))
servers.foreach(server => Utils.rm(server.config.logDirs))
}

2
core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala

@ -68,7 +68,6 @@ class SimpleFetchTest extends JUnit3Suite { @@ -68,7 +68,6 @@ class SimpleFetchTest extends JUnit3Suite {
val logManager = EasyMock.createMock(classOf[kafka.log.LogManager])
EasyMock.expect(logManager.getLog(topic, partitionId)).andReturn(Some(log)).anyTimes()
EasyMock.expect(logManager.config).andReturn(configs.head).anyTimes()
EasyMock.replay(logManager)
val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager])
@ -135,7 +134,6 @@ class SimpleFetchTest extends JUnit3Suite { @@ -135,7 +134,6 @@ class SimpleFetchTest extends JUnit3Suite {
val logManager = EasyMock.createMock(classOf[kafka.log.LogManager])
EasyMock.expect(logManager.getLog(topic, 0)).andReturn(Some(log)).anyTimes()
EasyMock.expect(logManager.config).andReturn(configs.head).anyTimes()
EasyMock.replay(logManager)
val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager])

32
core/src/test/scala/unit/kafka/utils/MockScheduler.scala

@ -32,7 +32,6 @@ import java.util.concurrent.TimeUnit @@ -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).
*/
@nonthreadsafe
class MockScheduler(val time: Time) extends Scheduler {
/* a priority queue of tasks ordered by next execution time */
@ -41,7 +40,9 @@ class MockScheduler(val time: Time) extends Scheduler { @@ -41,7 +40,9 @@ class MockScheduler(val time: Time) extends Scheduler {
def startup() {}
def shutdown() {
tasks.clear()
this synchronized {
tasks.clear()
}
}
/**
@ -50,23 +51,26 @@ class MockScheduler(val time: Time) extends Scheduler { @@ -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.
*/
def tick() {
val now = time.milliseconds
while(!tasks.isEmpty && tasks.head.nextExecution <= now) {
/* pop and execute the task with the lowest next execution time */
val curr = tasks.head
this.tasks = tasks.tail
curr.fun()
/* if the task is periodic, reschedule it and re-enqueue */
if(curr.periodic) {
curr.nextExecution += curr.period
this.tasks += curr
this synchronized {
val now = time.milliseconds
while(!tasks.isEmpty && tasks.head.nextExecution <= now) {
/* pop and execute the task with the lowest next execution time */
val curr = tasks.dequeue
curr.fun()
/* if the task is periodic, reschedule it and re-enqueue */
if(curr.periodic) {
curr.nextExecution += curr.period
this.tasks += curr
}
}
}
}
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)
tick()
this synchronized {
tasks += MockTask(name, fun, time.milliseconds + delay, period = period)
tick()
}
}
}

11
core/src/test/scala/unit/kafka/utils/TestUtils.scala

@ -43,6 +43,8 @@ import kafka.common.TopicAndPartition @@ -43,6 +43,8 @@ import kafka.common.TopicAndPartition
* Utility functions to help with testing
*/
object TestUtils extends Logging {
val IoTmpDir = System.getProperty("java.io.tmpdir")
val Letters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"
val Digits = "0123456789"
@ -74,8 +76,7 @@ object TestUtils extends Logging { @@ -74,8 +76,7 @@ object TestUtils extends Logging {
* Create a temporary directory
*/
def tempDir(): File = {
val ioDir = System.getProperty("java.io.tmpdir")
val f = new File(ioDir, "kafka-" + random.nextInt(1000000))
val f = new File(IoTmpDir, "kafka-" + random.nextInt(1000000))
f.mkdirs()
f.deleteOnExit()
f
@ -154,8 +155,8 @@ object TestUtils extends Logging { @@ -154,8 +155,8 @@ object TestUtils extends Logging {
* Wrap the message in a message set
* @param payload The bytes of the message
*/
def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec) =
new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload))
def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec, key: Array[Byte] = null) =
new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key))
/**
* Generate an array of random bytes
@ -497,7 +498,7 @@ object TestUtils extends Logging { @@ -497,7 +498,7 @@ object TestUtils extends Logging {
}
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] {

20
core/src/test/scala/unit/kafka/utils/UtilsTest.scala

@ -19,6 +19,7 @@ package kafka.utils @@ -19,6 +19,7 @@ package kafka.utils
import java.util.Arrays
import java.nio.ByteBuffer
import java.io._
import org.apache.log4j.Logger
import org.scalatest.junit.JUnitSuite
import org.junit.Test
@ -61,6 +62,25 @@ class UtilsTest extends JUnitSuite { @@ -61,6 +62,25 @@ class UtilsTest extends JUnitSuite {
assertTrue(Arrays.equals(bytes, Utils.readBytes(ByteBuffer.wrap(bytes))))
}
}
@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
def testCsvList() {

Loading…
Cancel
Save