From e7edb5e1e933f5535378d546bcf4d8b178d2e69c Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Mon, 28 Jan 2013 19:31:17 -0800 Subject: [PATCH] KAFKA-631 Implement a log cleaner for Kafka. Reviewed by Neha. --- config/log4j.properties | 15 +- config/server.properties | 1 + .../main/scala/kafka/cluster/Partition.scala | 8 +- .../OptimisticLockFailureException.scala | 23 + .../main/scala/kafka/log/CleanerConfig.scala | 41 ++ .../main/scala/kafka/log/FileMessageSet.scala | 30 +- core/src/main/scala/kafka/log/Log.scala | 265 ++++++--- .../src/main/scala/kafka/log/LogCleaner.scala | 557 ++++++++++++++++++ core/src/main/scala/kafka/log/LogConfig.scala | 51 ++ .../src/main/scala/kafka/log/LogManager.scala | 120 ++-- .../src/main/scala/kafka/log/LogSegment.scala | 24 +- .../main/scala/kafka/log/OffsetIndex.scala | 33 +- core/src/main/scala/kafka/log/OffsetMap.scala | 136 +++++ .../kafka/message/ByteBufferMessageSet.scala | 4 +- .../server/HighwaterMarkCheckpoint.scala | 118 ---- .../main/scala/kafka/server/KafkaConfig.scala | 33 +- .../main/scala/kafka/server/KafkaServer.scala | 51 +- .../scala/kafka/server/OffsetCheckpoint.scala | 103 ++++ .../scala/kafka/server/ReplicaManager.scala | 14 +- .../src/main/scala/kafka/utils/FileLock.scala | 94 +-- core/src/main/scala/kafka/utils/Logging.scala | 5 +- .../main/scala/kafka/utils/Throttler.scala | 26 +- core/src/main/scala/kafka/utils/Utils.scala | 36 +- .../kafka/utils/VerifiableProperties.scala | 35 +- .../scala/other/kafka/StressTestLog.scala | 12 +- .../scala/other/kafka/TestLogCleaning.scala | 216 +++++++ .../other/kafka/TestLogPerformance.scala | 3 +- .../scala/unit/kafka/log/CleanerTest.scala | 227 +++++++ .../kafka/log/LogCleanerIntegrationTest.scala | 117 ++++ .../scala/unit/kafka/log/LogManagerTest.scala | 57 +- .../scala/unit/kafka/log/LogSegmentTest.scala | 15 + .../test/scala/unit/kafka/log/LogTest.scala | 139 ++--- .../scala/unit/kafka/log/OffsetMapTest.scala | 87 +++ .../server/HighwatermarkPersistenceTest.scala | 16 +- .../unit/kafka/server/LogOffsetTest.scala | 2 +- .../unit/kafka/server/LogRecoveryTest.scala | 48 +- .../unit/kafka/server/SimpleFetchTest.scala | 2 - .../unit/kafka/utils/MockScheduler.scala | 32 +- .../scala/unit/kafka/utils/TestUtils.scala | 11 +- .../scala/unit/kafka/utils/UtilsTest.scala | 20 + 40 files changed, 2280 insertions(+), 547 deletions(-) create mode 100644 core/src/main/scala/kafka/common/OptimisticLockFailureException.scala create mode 100644 core/src/main/scala/kafka/log/CleanerConfig.scala create mode 100644 core/src/main/scala/kafka/log/LogCleaner.scala create mode 100644 core/src/main/scala/kafka/log/LogConfig.scala create mode 100644 core/src/main/scala/kafka/log/OffsetMap.scala delete mode 100644 core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala create mode 100644 core/src/main/scala/kafka/server/OffsetCheckpoint.scala create mode 100644 core/src/test/scala/other/kafka/TestLogCleaning.scala create mode 100644 core/src/test/scala/unit/kafka/log/CleanerTest.scala create mode 100644 core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala create mode 100644 core/src/test/scala/unit/kafka/log/OffsetMapTest.scala diff --git a/config/log4j.properties b/config/log4j.properties index 00f891cc5c6..b36d3e03cba 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -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 #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 diff --git a/config/server.properties b/config/server.properties index 04408ddd8a7..5a16caf3332 100644 --- a/config/server.properties +++ b/config/server.properties @@ -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 diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 71eb98096f2..af80631b4ec 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -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, 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) } diff --git a/core/src/main/scala/kafka/common/OptimisticLockFailureException.scala b/core/src/main/scala/kafka/common/OptimisticLockFailureException.scala new file mode 100644 index 00000000000..0e69110e716 --- /dev/null +++ b/core/src/main/scala/kafka/common/OptimisticLockFailureException.scala @@ -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) \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/CleanerConfig.scala b/core/src/main/scala/kafka/log/CleanerConfig.scala new file mode 100644 index 00000000000..999fee868c1 --- /dev/null +++ b/core/src/main/scala/kafka/log/CleanerConfig.scala @@ -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") { +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index 37e8d87401a..a74abfef53d 100644 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -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, * 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 } } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 5ea94890005..ac12b74c0e5 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -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, /* 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, 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, * 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, 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, // 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, */ 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, } 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, * @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, activeSegment.truncateTo(targetOffset) this.nextOffset.set(targetOffset) } + truncates.getAndIncrement } } @@ -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, */ 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, * @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 diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala new file mode 100644 index 00000000000..368a12be1f9 --- /dev/null +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -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 +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala new file mode 100644 index 00000000000..5a10bef3b36 --- /dev/null +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -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) + + \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index c5ab8a262b2..438d802ab23 100644 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -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: *
    @@ -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, 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, 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, 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, */ 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, 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, (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, 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 => diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 39dd9c203ce..120ebeb7f37 100644 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -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, * 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, } } + /** + * 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 */ diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index 1662f107b4d..23e659f742f 100644 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -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 = 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 = } 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 = /** 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 diff --git a/core/src/main/scala/kafka/log/OffsetMap.scala b/core/src/main/scala/kafka/log/OffsetMap.scala new file mode 100644 index 00000000000..623681361e4 --- /dev/null +++ b/core/src/main/scala/kafka/log/OffsetMap.scala @@ -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) + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 873699ff7e2..1b9c8f89ea4 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -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 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 diff --git a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala b/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala deleted file mode 100644 index 5aa01414c8c..00000000000 --- a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala +++ /dev/null @@ -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() - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 51ea727ee77..f4ba59cfde6 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -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 /* 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 */ diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 9258b133c0a..da6f716676e 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -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 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 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) + } + } diff --git a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala new file mode 100644 index 00000000000..79f29df770d --- /dev/null +++ b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala @@ -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() + } + } + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 7810c214c74..710c08b0134 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -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 object ReplicaManager { val UnknownLogEndOffset = -1L + val HighWatermarkFilename = "replication-offset-checkpoint" } 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, } ) 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, 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) + } } } diff --git a/core/src/main/scala/kafka/utils/FileLock.scala b/core/src/main/scala/kafka/utils/FileLock.scala index ad7a59742b8..a5761b9db31 100644 --- a/core/src/main/scala/kafka/utils/FileLock.scala +++ b/core/src/main/scala/kafka/utils/FileLock.scala @@ -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() } + } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/Logging.scala b/core/src/main/scala/kafka/utils/Logging.scala index d9f010bfe7a..2890e7f51dd 100644 --- a/core/src/main/scala/kafka/utils/Logging.scala +++ b/core/src/main/scala/kafka/utils/Logging.scala @@ -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()) diff --git a/core/src/main/scala/kafka/utils/Throttler.scala b/core/src/main/scala/kafka/utils/Throttler.scala index 9e53b03c612..c6c3c75ee84 100644 --- a/core/src/main/scala/kafka/utils/Throttler.scala +++ b/core/src/main/scala/kafka/utils/Throttler.scala @@ -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, 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, 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 } diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 2a01f696f93..1c882269df7 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -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 { * @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 { */ 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) + } + } diff --git a/core/src/main/scala/kafka/utils/VerifiableProperties.scala b/core/src/main/scala/kafka/utils/VerifiableProperties.scala index d694ba98522..a2ac55ccee9 100644 --- a/core/src/main/scala/kafka/utils/VerifiableProperties.scala +++ b/core/src/main/scala/kafka/utils/VerifiableProperties.scala @@ -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 { /** * 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 { 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 { 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 { /** * 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 { } override def toString(): String = props.toString + } diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index 55429afb0c9..c6e7a571042 100644 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -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() diff --git a/core/src/test/scala/other/kafka/TestLogCleaning.scala b/core/src/test/scala/other/kafka/TestLogCleaning.scala new file mode 100644 index 00000000000..d9c721ba3ea --- /dev/null +++ b/core/src/test/scala/other/kafka/TestLogCleaning.scala @@ -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 + } + +} \ No newline at end of file diff --git a/core/src/test/scala/other/kafka/TestLogPerformance.scala b/core/src/test/scala/other/kafka/TestLogPerformance.scala index a7b661a2e53..d91011eb2f6 100644 --- a/core/src/test/scala/other/kafka/TestLogPerformance.scala +++ b/core/src/test/scala/other/kafka/TestLogPerformance.scala @@ -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) diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala new file mode 100644 index 00000000000..cce2319d23b --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -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 + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala new file mode 100644 index 00000000000..5a489f9eafe --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -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) + } + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index f48c709d8e1..fad3baac1c6 100644 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -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 { @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 { 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 { 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 { @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 { 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 { */ @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 { @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 { */ 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 diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index bffe4a451a0..a185ce4f7b2 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -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. diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 0fc74fa2d4c..23e0e6562dc 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -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 { @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 { 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 { 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 { @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 { */ @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 { */ @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 { */ @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 { @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 { @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 { @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 { 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 { // 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 { 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 { 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 { 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 { 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 { 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 { 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 { 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 { 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 { @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 { 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 { // 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 { @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 { 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) } diff --git a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala new file mode 100644 index 00000000000..99a0c4bfefb --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala @@ -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) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index b6095857e39..8a3e33b5a39 100644 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -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 { 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 { } 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) } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index e3752cb191d..f857171ef68 100644 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -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 diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index cd724a337f8..48487e84c15 100644 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -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 { 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 { 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 { // 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 { // 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 { 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 { // 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 { 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 { // 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 { 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 { // 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 { // 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)) } diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 46193899492..571e2df48bd 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -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 { 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]) diff --git a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala index 4853f2bbbfc..d5896ed4d3b 100644 --- a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala +++ b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala @@ -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 { def startup() {} def shutdown() { - tasks.clear() + this synchronized { + tasks.clear() + } } /** @@ -50,23 +51,26 @@ class MockScheduler(val time: Time) extends Scheduler { * If you are using the scheduler associated with a MockTime instance this call be triggered automatically. */ 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() + } } } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 5547d63a405..d12d24ef457 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -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 { * 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 { * 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 { } 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] { diff --git a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala index cce6c8ea29f..48dd3352dda 100644 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -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 { 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() {