mirror of https://github.com/apache/kafka.git
kafka-1414; Speedup broker startup after hard reset; patched by Anton Karamanov; reviewed by Jay Kreps and Jun Rao
This commit is contained in:
parent
fa34841d98
commit
f489493c38
|
@ -62,6 +62,10 @@ log.dirs=/tmp/kafka-logs
|
||||||
# the brokers.
|
# the brokers.
|
||||||
num.partitions=1
|
num.partitions=1
|
||||||
|
|
||||||
|
# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
|
||||||
|
# This value is recommended to be increased for installations with data dirs located in RAID array.
|
||||||
|
num.recovery.threads.per.data.dir=1
|
||||||
|
|
||||||
############################# Log Flush Policy #############################
|
############################# Log Flush Policy #############################
|
||||||
|
|
||||||
# Messages are immediately written to the filesystem but by default we only fsync() to sync
|
# Messages are immediately written to the filesystem but by default we only fsync() to sync
|
||||||
|
|
|
@ -23,6 +23,7 @@ import kafka.utils._
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
import kafka.common.{TopicAndPartition, KafkaException}
|
import kafka.common.{TopicAndPartition, KafkaException}
|
||||||
import kafka.server.{RecoveringFromUncleanShutdown, BrokerState, OffsetCheckpoint}
|
import kafka.server.{RecoveringFromUncleanShutdown, BrokerState, OffsetCheckpoint}
|
||||||
|
import java.util.concurrent.{Executors, ExecutorService, ExecutionException, Future}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning.
|
* The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning.
|
||||||
|
@ -39,6 +40,7 @@ class LogManager(val logDirs: Array[File],
|
||||||
val topicConfigs: Map[String, LogConfig],
|
val topicConfigs: Map[String, LogConfig],
|
||||||
val defaultConfig: LogConfig,
|
val defaultConfig: LogConfig,
|
||||||
val cleanerConfig: CleanerConfig,
|
val cleanerConfig: CleanerConfig,
|
||||||
|
ioThreads: Int,
|
||||||
val flushCheckMs: Long,
|
val flushCheckMs: Long,
|
||||||
val flushCheckpointMs: Long,
|
val flushCheckpointMs: Long,
|
||||||
val retentionCheckMs: Long,
|
val retentionCheckMs: Long,
|
||||||
|
@ -54,7 +56,7 @@ class LogManager(val logDirs: Array[File],
|
||||||
createAndValidateLogDirs(logDirs)
|
createAndValidateLogDirs(logDirs)
|
||||||
private val dirLocks = lockLogDirs(logDirs)
|
private val dirLocks = lockLogDirs(logDirs)
|
||||||
private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap
|
private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap
|
||||||
loadLogs(logDirs)
|
loadLogs()
|
||||||
|
|
||||||
private val cleaner: LogCleaner =
|
private val cleaner: LogCleaner =
|
||||||
if(cleanerConfig.enableCleaner)
|
if(cleanerConfig.enableCleaner)
|
||||||
|
@ -101,36 +103,71 @@ class LogManager(val logDirs: Array[File],
|
||||||
/**
|
/**
|
||||||
* Recover and load all logs in the given data directories
|
* Recover and load all logs in the given data directories
|
||||||
*/
|
*/
|
||||||
private def loadLogs(dirs: Seq[File]) {
|
private def loadLogs(): Unit = {
|
||||||
for(dir <- dirs) {
|
info("Loading logs.")
|
||||||
val recoveryPoints = this.recoveryPointCheckpoints(dir).read
|
|
||||||
/* load the logs */
|
|
||||||
val subDirs = dir.listFiles()
|
|
||||||
if(subDirs != null) {
|
|
||||||
val cleanShutDownFile = new File(dir, Log.CleanShutdownFile)
|
|
||||||
if(cleanShutDownFile.exists())
|
|
||||||
info("Found clean shutdown file. Skipping recovery for all logs in data directory '%s'".format(dir.getAbsolutePath))
|
|
||||||
else
|
|
||||||
brokerState.newState(RecoveringFromUncleanShutdown)
|
|
||||||
|
|
||||||
for(dir <- subDirs) {
|
val threadPools = mutable.ArrayBuffer.empty[ExecutorService]
|
||||||
if(dir.isDirectory) {
|
val jobs = mutable.Map.empty[File, Seq[Future[_]]]
|
||||||
info("Loading log '" + dir.getName + "'")
|
|
||||||
val topicPartition = Log.parseTopicPartitionName(dir.getName)
|
for (dir <- this.logDirs) {
|
||||||
val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
|
val pool = Executors.newFixedThreadPool(ioThreads)
|
||||||
val log = new Log(dir,
|
threadPools.append(pool)
|
||||||
config,
|
|
||||||
recoveryPoints.getOrElse(topicPartition, 0L),
|
val cleanShutdownFile = new File(dir, Log.CleanShutdownFile)
|
||||||
scheduler,
|
|
||||||
time)
|
if (cleanShutdownFile.exists) {
|
||||||
val previous = this.logs.put(topicPartition, log)
|
debug(
|
||||||
if(previous != null)
|
"Found clean shutdown file. " +
|
||||||
throw new IllegalArgumentException("Duplicate log directories found: %s, %s!".format(log.dir.getAbsolutePath, previous.dir.getAbsolutePath))
|
"Skipping recovery for all logs in data directory: " +
|
||||||
|
dir.getAbsolutePath)
|
||||||
|
} else {
|
||||||
|
// log recovery itself is being performed by `Log` class during initialization
|
||||||
|
brokerState.newState(RecoveringFromUncleanShutdown)
|
||||||
|
}
|
||||||
|
|
||||||
|
val recoveryPoints = this.recoveryPointCheckpoints(dir).read
|
||||||
|
|
||||||
|
val jobsForDir = for {
|
||||||
|
dirContent <- Option(dir.listFiles).toList
|
||||||
|
logDir <- dirContent if logDir.isDirectory
|
||||||
|
} yield {
|
||||||
|
Utils.runnable {
|
||||||
|
debug("Loading log '" + logDir.getName + "'")
|
||||||
|
|
||||||
|
val topicPartition = Log.parseTopicPartitionName(logDir.getName)
|
||||||
|
val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
|
||||||
|
val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L)
|
||||||
|
|
||||||
|
val current = new Log(logDir, config, logRecoveryPoint, scheduler, time)
|
||||||
|
val previous = this.logs.put(topicPartition, current)
|
||||||
|
|
||||||
|
if (previous != null) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Duplicate log directories found: %s, %s!".format(
|
||||||
|
current.dir.getAbsolutePath, previous.dir.getAbsolutePath))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
cleanShutDownFile.delete()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
jobs(cleanShutdownFile) = jobsForDir.map(pool.submit).toSeq
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
try {
|
||||||
|
for ((cleanShutdownFile, dirJobs) <- jobs) {
|
||||||
|
dirJobs.foreach(_.get)
|
||||||
|
cleanShutdownFile.delete()
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
case e: ExecutionException => {
|
||||||
|
error("There was an error in one of the threads during logs loading: " + e.getCause)
|
||||||
|
throw e.getCause
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
threadPools.foreach(_.shutdown())
|
||||||
|
}
|
||||||
|
|
||||||
|
info("Logs loading complete.")
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -160,31 +197,69 @@ class LogManager(val logDirs: Array[File],
|
||||||
if(cleanerConfig.enableCleaner)
|
if(cleanerConfig.enableCleaner)
|
||||||
cleaner.startup()
|
cleaner.startup()
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Close all the logs
|
* Close all the logs
|
||||||
*/
|
*/
|
||||||
def shutdown() {
|
def shutdown() {
|
||||||
info("Shutting down.")
|
info("Shutting down.")
|
||||||
|
|
||||||
|
val threadPools = mutable.ArrayBuffer.empty[ExecutorService]
|
||||||
|
val jobs = mutable.Map.empty[File, Seq[Future[_]]]
|
||||||
|
|
||||||
|
// stop the cleaner first
|
||||||
|
if (cleaner != null) {
|
||||||
|
Utils.swallow(cleaner.shutdown())
|
||||||
|
}
|
||||||
|
|
||||||
|
// close logs in each dir
|
||||||
|
for (dir <- this.logDirs) {
|
||||||
|
debug("Flushing and closing logs at " + dir)
|
||||||
|
|
||||||
|
val pool = Executors.newFixedThreadPool(ioThreads)
|
||||||
|
threadPools.append(pool)
|
||||||
|
|
||||||
|
val logsInDir = logsByDir.getOrElse(dir.toString, Map()).values
|
||||||
|
|
||||||
|
val jobsForDir = logsInDir map { log =>
|
||||||
|
Utils.runnable {
|
||||||
|
// flush the log to ensure latest possible recovery point
|
||||||
|
log.flush()
|
||||||
|
log.close()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
jobs(dir) = jobsForDir.map(pool.submit).toSeq
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
// stop the cleaner first
|
for ((dir, dirJobs) <- jobs) {
|
||||||
if(cleaner != null)
|
dirJobs.foreach(_.get)
|
||||||
Utils.swallow(cleaner.shutdown())
|
|
||||||
// flush the logs to ensure latest possible recovery point
|
// update the last flush point
|
||||||
allLogs.foreach(_.flush())
|
debug("Updating recovery points at " + dir)
|
||||||
// close the logs
|
checkpointLogsInDir(dir)
|
||||||
allLogs.foreach(_.close())
|
|
||||||
// update the last flush point
|
// mark that the shutdown was clean by creating marker file
|
||||||
checkpointRecoveryPointOffsets()
|
debug("Writing clean shutdown marker at " + dir)
|
||||||
// mark that the shutdown was clean by creating the clean shutdown marker file
|
Utils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile())
|
||||||
logDirs.foreach(dir => Utils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile()))
|
}
|
||||||
|
} catch {
|
||||||
|
case e: ExecutionException => {
|
||||||
|
error("There was an error in one of the threads during LogManager shutdown: " + e.getCause)
|
||||||
|
throw e.getCause
|
||||||
|
}
|
||||||
} finally {
|
} finally {
|
||||||
|
threadPools.foreach(_.shutdown())
|
||||||
// regardless of whether the close succeeded, we need to unlock the data directories
|
// regardless of whether the close succeeded, we need to unlock the data directories
|
||||||
dirLocks.foreach(_.destroy())
|
dirLocks.foreach(_.destroy())
|
||||||
}
|
}
|
||||||
|
|
||||||
info("Shutdown complete.")
|
info("Shutdown complete.")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset
|
* Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset
|
||||||
*
|
*
|
||||||
|
@ -230,14 +305,19 @@ class LogManager(val logDirs: Array[File],
|
||||||
* to avoid recovering the whole log on startup.
|
* to avoid recovering the whole log on startup.
|
||||||
*/
|
*/
|
||||||
def checkpointRecoveryPointOffsets() {
|
def checkpointRecoveryPointOffsets() {
|
||||||
val recoveryPointsByDir = this.logsByTopicPartition.groupBy(_._2.dir.getParent.toString)
|
this.logDirs.foreach(checkpointLogsInDir)
|
||||||
for(dir <- logDirs) {
|
}
|
||||||
val recoveryPoints = recoveryPointsByDir.get(dir.toString)
|
|
||||||
if(recoveryPoints.isDefined)
|
/**
|
||||||
this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint))
|
* Make a checkpoint for all logs in provided directory.
|
||||||
|
*/
|
||||||
|
private def checkpointLogsInDir(dir: File): Unit = {
|
||||||
|
val recoveryPoints = this.logsByDir.get(dir.toString)
|
||||||
|
if (recoveryPoints.isDefined) {
|
||||||
|
this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the log if it exists, otherwise return None
|
* Get the log if it exists, otherwise return None
|
||||||
*/
|
*/
|
||||||
|
@ -366,12 +446,21 @@ class LogManager(val logDirs: Array[File],
|
||||||
* Get all the partition logs
|
* Get all the partition logs
|
||||||
*/
|
*/
|
||||||
def allLogs(): Iterable[Log] = logs.values
|
def allLogs(): Iterable[Log] = logs.values
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a map of TopicAndPartition => Log
|
* Get a map of TopicAndPartition => Log
|
||||||
*/
|
*/
|
||||||
def logsByTopicPartition = logs.toMap
|
def logsByTopicPartition = logs.toMap
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Map of log dir to logs by topic and partitions in that dir
|
||||||
|
*/
|
||||||
|
private def logsByDir = {
|
||||||
|
this.logsByTopicPartition.groupBy {
|
||||||
|
case (_, log) => log.dir.getParent
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Flush any log which has exceeded its flush interval and has unwritten messages.
|
* Flush any log which has exceeded its flush interval and has unwritten messages.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -190,6 +190,9 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
|
||||||
/* the frequency with which we update the persistent record of the last flush which acts as the log recovery point */
|
/* the frequency with which we update the persistent record of the last flush which acts as the log recovery point */
|
||||||
val logFlushOffsetCheckpointIntervalMs = props.getIntInRange("log.flush.offset.checkpoint.interval.ms", 60000, (0, Int.MaxValue))
|
val logFlushOffsetCheckpointIntervalMs = props.getIntInRange("log.flush.offset.checkpoint.interval.ms", 60000, (0, Int.MaxValue))
|
||||||
|
|
||||||
|
/* the number of threads per data directory to be used for log recovery at startup and flushing at shutdown */
|
||||||
|
val numRecoveryThreadsPerDataDir = props.getIntInRange("num.recovery.threads.per.data.dir", 1, (1, Int.MaxValue))
|
||||||
|
|
||||||
/* enable auto creation of topic on the server */
|
/* enable auto creation of topic on the server */
|
||||||
val autoCreateTopicsEnable = props.getBoolean("auto.create.topics.enable", true)
|
val autoCreateTopicsEnable = props.getBoolean("auto.create.topics.enable", true)
|
||||||
|
|
||||||
|
|
|
@ -303,6 +303,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
||||||
topicConfigs = configs,
|
topicConfigs = configs,
|
||||||
defaultConfig = defaultLogConfig,
|
defaultConfig = defaultLogConfig,
|
||||||
cleanerConfig = cleanerConfig,
|
cleanerConfig = cleanerConfig,
|
||||||
|
ioThreads = config.numRecoveryThreadsPerDataDir,
|
||||||
flushCheckMs = config.logFlushSchedulerIntervalMs,
|
flushCheckMs = config.logFlushSchedulerIntervalMs,
|
||||||
flushCheckpointMs = config.logFlushOffsetCheckpointIntervalMs,
|
flushCheckpointMs = config.logFlushOffsetCheckpointIntervalMs,
|
||||||
retentionCheckMs = config.logCleanupIntervalMs,
|
retentionCheckMs = config.logCleanupIntervalMs,
|
||||||
|
|
|
@ -93,16 +93,14 @@ class KafkaScheduler(val threads: Int,
|
||||||
debug("Scheduling task %s with initial delay %d ms and period %d ms."
|
debug("Scheduling task %s with initial delay %d ms and period %d ms."
|
||||||
.format(name, TimeUnit.MILLISECONDS.convert(delay, unit), TimeUnit.MILLISECONDS.convert(period, unit)))
|
.format(name, TimeUnit.MILLISECONDS.convert(delay, unit), TimeUnit.MILLISECONDS.convert(period, unit)))
|
||||||
ensureStarted
|
ensureStarted
|
||||||
val runnable = new Runnable {
|
val runnable = Utils.runnable {
|
||||||
def run() = {
|
try {
|
||||||
try {
|
trace("Begining execution of scheduled task '%s'.".format(name))
|
||||||
trace("Begining execution of scheduled task '%s'.".format(name))
|
fun()
|
||||||
fun()
|
} catch {
|
||||||
} catch {
|
case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t)
|
||||||
case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t)
|
} finally {
|
||||||
} finally {
|
trace("Completed execution of scheduled task '%s'.".format(name))
|
||||||
trace("Completed execution of scheduled task '%s'.".format(name))
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if(period >= 0)
|
if(period >= 0)
|
||||||
|
|
|
@ -49,9 +49,9 @@ object Utils extends Logging {
|
||||||
* @param fun A function
|
* @param fun A function
|
||||||
* @return A Runnable that just executes the function
|
* @return A Runnable that just executes the function
|
||||||
*/
|
*/
|
||||||
def runnable(fun: () => Unit): Runnable =
|
def runnable(fun: => Unit): Runnable =
|
||||||
new Runnable() {
|
new Runnable {
|
||||||
def run() = fun()
|
def run() = fun
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -35,21 +35,11 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
var logManager: LogManager = null
|
var logManager: LogManager = null
|
||||||
val name = "kafka"
|
val name = "kafka"
|
||||||
val veryLargeLogFlushInterval = 10000000L
|
val veryLargeLogFlushInterval = 10000000L
|
||||||
val cleanerConfig = CleanerConfig(enableCleaner = false)
|
|
||||||
|
|
||||||
override def setUp() {
|
override def setUp() {
|
||||||
super.setUp()
|
super.setUp()
|
||||||
logDir = TestUtils.tempDir()
|
logDir = TestUtils.tempDir()
|
||||||
logManager = new LogManager(logDirs = Array(logDir),
|
logManager = createLogManager()
|
||||||
topicConfigs = Map(),
|
|
||||||
defaultConfig = logConfig,
|
|
||||||
cleanerConfig = cleanerConfig,
|
|
||||||
flushCheckMs = 1000L,
|
|
||||||
flushCheckpointMs = 100000L,
|
|
||||||
retentionCheckMs = 1000L,
|
|
||||||
scheduler = time.scheduler,
|
|
||||||
time = time,
|
|
||||||
brokerState = new BrokerState())
|
|
||||||
logManager.startup
|
logManager.startup
|
||||||
logDir = logManager.logDirs(0)
|
logDir = logManager.logDirs(0)
|
||||||
}
|
}
|
||||||
|
@ -125,18 +115,7 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
logManager.shutdown()
|
logManager.shutdown()
|
||||||
|
|
||||||
val config = logConfig.copy(segmentSize = 10 * (setSize - 1), retentionSize = 5L * 10L * setSize + 10L)
|
val config = logConfig.copy(segmentSize = 10 * (setSize - 1), retentionSize = 5L * 10L * setSize + 10L)
|
||||||
logManager = new LogManager(
|
logManager = createLogManager()
|
||||||
logDirs = Array(logDir),
|
|
||||||
topicConfigs = Map(),
|
|
||||||
defaultConfig = config,
|
|
||||||
cleanerConfig = cleanerConfig,
|
|
||||||
flushCheckMs = 1000L,
|
|
||||||
flushCheckpointMs = 100000L,
|
|
||||||
retentionCheckMs = 1000L,
|
|
||||||
scheduler = time.scheduler,
|
|
||||||
brokerState = new BrokerState(),
|
|
||||||
time = time
|
|
||||||
)
|
|
||||||
logManager.startup
|
logManager.startup
|
||||||
|
|
||||||
// create a log
|
// create a log
|
||||||
|
@ -176,18 +155,7 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
def testTimeBasedFlush() {
|
def testTimeBasedFlush() {
|
||||||
logManager.shutdown()
|
logManager.shutdown()
|
||||||
val config = logConfig.copy(flushMs = 1000)
|
val config = logConfig.copy(flushMs = 1000)
|
||||||
logManager = new LogManager(
|
logManager = createLogManager()
|
||||||
logDirs = Array(logDir),
|
|
||||||
topicConfigs = Map(),
|
|
||||||
defaultConfig = config,
|
|
||||||
cleanerConfig = cleanerConfig,
|
|
||||||
flushCheckMs = 1000L,
|
|
||||||
flushCheckpointMs = 10000L,
|
|
||||||
retentionCheckMs = 1000L,
|
|
||||||
scheduler = time.scheduler,
|
|
||||||
brokerState = new BrokerState(),
|
|
||||||
time = time
|
|
||||||
)
|
|
||||||
logManager.startup
|
logManager.startup
|
||||||
val log = logManager.createLog(TopicAndPartition(name, 0), config)
|
val log = logManager.createLog(TopicAndPartition(name, 0), config)
|
||||||
val lastFlush = log.lastFlushTime
|
val lastFlush = log.lastFlushTime
|
||||||
|
@ -209,19 +177,8 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
TestUtils.tempDir(),
|
TestUtils.tempDir(),
|
||||||
TestUtils.tempDir())
|
TestUtils.tempDir())
|
||||||
logManager.shutdown()
|
logManager.shutdown()
|
||||||
logManager = new LogManager(
|
logManager = createLogManager()
|
||||||
logDirs = dirs,
|
|
||||||
topicConfigs = Map(),
|
|
||||||
defaultConfig = logConfig,
|
|
||||||
cleanerConfig = cleanerConfig,
|
|
||||||
flushCheckMs = 1000L,
|
|
||||||
flushCheckpointMs = 10000L,
|
|
||||||
retentionCheckMs = 1000L,
|
|
||||||
scheduler = time.scheduler,
|
|
||||||
brokerState = new BrokerState(),
|
|
||||||
time = time
|
|
||||||
)
|
|
||||||
|
|
||||||
// verify that logs are always assigned to the least loaded partition
|
// verify that logs are always assigned to the least loaded partition
|
||||||
for(partition <- 0 until 20) {
|
for(partition <- 0 until 20) {
|
||||||
logManager.createLog(TopicAndPartition("test", partition), logConfig)
|
logManager.createLog(TopicAndPartition("test", partition), logConfig)
|
||||||
|
@ -237,18 +194,7 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
@Test
|
@Test
|
||||||
def testTwoLogManagersUsingSameDirFails() {
|
def testTwoLogManagersUsingSameDirFails() {
|
||||||
try {
|
try {
|
||||||
new LogManager(
|
createLogManager()
|
||||||
logDirs = Array(logDir),
|
|
||||||
topicConfigs = Map(),
|
|
||||||
defaultConfig = logConfig,
|
|
||||||
cleanerConfig = cleanerConfig,
|
|
||||||
flushCheckMs = 1000L,
|
|
||||||
flushCheckpointMs = 10000L,
|
|
||||||
retentionCheckMs = 1000L,
|
|
||||||
scheduler = time.scheduler,
|
|
||||||
brokerState = new BrokerState(),
|
|
||||||
time = time
|
|
||||||
)
|
|
||||||
fail("Should not be able to create a second log manager instance with the same data directory")
|
fail("Should not be able to create a second log manager instance with the same data directory")
|
||||||
} catch {
|
} catch {
|
||||||
case e: KafkaException => // this is good
|
case e: KafkaException => // this is good
|
||||||
|
@ -270,16 +216,8 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
def testRecoveryDirectoryMappingWithTrailingSlash() {
|
def testRecoveryDirectoryMappingWithTrailingSlash() {
|
||||||
logManager.shutdown()
|
logManager.shutdown()
|
||||||
logDir = TestUtils.tempDir()
|
logDir = TestUtils.tempDir()
|
||||||
logManager = new LogManager(logDirs = Array(new File(logDir.getAbsolutePath + File.separator)),
|
logManager = TestUtils.createLogManager(
|
||||||
topicConfigs = Map(),
|
logDirs = Array(new File(logDir.getAbsolutePath + File.separator)))
|
||||||
defaultConfig = logConfig,
|
|
||||||
cleanerConfig = cleanerConfig,
|
|
||||||
flushCheckMs = 1000L,
|
|
||||||
flushCheckpointMs = 100000L,
|
|
||||||
retentionCheckMs = 1000L,
|
|
||||||
scheduler = time.scheduler,
|
|
||||||
time = time,
|
|
||||||
brokerState = new BrokerState())
|
|
||||||
logManager.startup
|
logManager.startup
|
||||||
verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager)
|
verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager)
|
||||||
}
|
}
|
||||||
|
@ -293,16 +231,7 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
logDir = new File("data" + File.separator + logDir.getName)
|
logDir = new File("data" + File.separator + logDir.getName)
|
||||||
logDir.mkdirs()
|
logDir.mkdirs()
|
||||||
logDir.deleteOnExit()
|
logDir.deleteOnExit()
|
||||||
logManager = new LogManager(logDirs = Array(logDir),
|
logManager = createLogManager()
|
||||||
topicConfigs = Map(),
|
|
||||||
defaultConfig = logConfig,
|
|
||||||
cleanerConfig = cleanerConfig,
|
|
||||||
flushCheckMs = 1000L,
|
|
||||||
flushCheckpointMs = 100000L,
|
|
||||||
retentionCheckMs = 1000L,
|
|
||||||
scheduler = time.scheduler,
|
|
||||||
time = time,
|
|
||||||
brokerState = new BrokerState())
|
|
||||||
logManager.startup
|
logManager.startup
|
||||||
verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager)
|
verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager)
|
||||||
}
|
}
|
||||||
|
@ -327,4 +256,12 @@ class LogManagerTest extends JUnit3Suite {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private def createLogManager(logDirs: Array[File] = Array(this.logDir)): LogManager = {
|
||||||
|
TestUtils.createLogManager(
|
||||||
|
defaultConfig = logConfig,
|
||||||
|
logDirs = logDirs,
|
||||||
|
time = this.time)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -32,16 +32,11 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
|
||||||
|
|
||||||
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_))
|
val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_))
|
||||||
val topic = "foo"
|
val topic = "foo"
|
||||||
val logManagers = configs.map(config => new LogManager(logDirs = config.logDirs.map(new File(_)).toArray,
|
val logManagers = configs map { config =>
|
||||||
topicConfigs = Map(),
|
TestUtils.createLogManager(
|
||||||
defaultConfig = LogConfig(),
|
logDirs = config.logDirs.map(new File(_)).toArray,
|
||||||
cleanerConfig = CleanerConfig(),
|
cleanerConfig = CleanerConfig())
|
||||||
flushCheckMs = 30000,
|
}
|
||||||
flushCheckpointMs = 10000L,
|
|
||||||
retentionCheckMs = 30000,
|
|
||||||
scheduler = new KafkaScheduler(1),
|
|
||||||
brokerState = new BrokerState(),
|
|
||||||
time = new MockTime))
|
|
||||||
|
|
||||||
@After
|
@After
|
||||||
def teardown() {
|
def teardown() {
|
||||||
|
@ -147,4 +142,4 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
|
||||||
replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs(0)).getAbsolutePath).read.getOrElse(TopicAndPartition(topic, partition), 0L)
|
replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs(0)).getAbsolutePath).read.getOrElse(TopicAndPartition(topic, partition), 0L)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,7 +18,6 @@
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import kafka.utils.{MockScheduler, MockTime, TestUtils}
|
import kafka.utils.{MockScheduler, MockTime, TestUtils}
|
||||||
import kafka.log.{CleanerConfig, LogManager, LogConfig}
|
|
||||||
|
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
import java.io.File
|
import java.io.File
|
||||||
|
@ -37,7 +36,7 @@ class ReplicaManagerTest extends JUnit3Suite {
|
||||||
val props = TestUtils.createBrokerConfig(1)
|
val props = TestUtils.createBrokerConfig(1)
|
||||||
val config = new KafkaConfig(props)
|
val config = new KafkaConfig(props)
|
||||||
val zkClient = EasyMock.createMock(classOf[ZkClient])
|
val zkClient = EasyMock.createMock(classOf[ZkClient])
|
||||||
val mockLogMgr = createLogManager(config.logDirs.map(new File(_)).toArray)
|
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
|
||||||
val time: MockTime = new MockTime()
|
val time: MockTime = new MockTime()
|
||||||
val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false))
|
val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false))
|
||||||
val partition = rm.getOrCreatePartition(topic, 1, 1)
|
val partition = rm.getOrCreatePartition(topic, 1, 1)
|
||||||
|
@ -51,26 +50,11 @@ class ReplicaManagerTest extends JUnit3Suite {
|
||||||
props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
|
props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
|
||||||
val config = new KafkaConfig(props)
|
val config = new KafkaConfig(props)
|
||||||
val zkClient = EasyMock.createMock(classOf[ZkClient])
|
val zkClient = EasyMock.createMock(classOf[ZkClient])
|
||||||
val mockLogMgr = createLogManager(config.logDirs.map(new File(_)).toArray)
|
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
|
||||||
val time: MockTime = new MockTime()
|
val time: MockTime = new MockTime()
|
||||||
val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false))
|
val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false))
|
||||||
val partition = rm.getOrCreatePartition(topic, 1, 1)
|
val partition = rm.getOrCreatePartition(topic, 1, 1)
|
||||||
partition.getOrCreateReplica(1)
|
partition.getOrCreateReplica(1)
|
||||||
rm.checkpointHighWatermarks()
|
rm.checkpointHighWatermarks()
|
||||||
}
|
}
|
||||||
|
|
||||||
private def createLogManager(logDirs: Array[File]): LogManager = {
|
|
||||||
val time = new MockTime()
|
|
||||||
return new LogManager(logDirs,
|
|
||||||
topicConfigs = Map(),
|
|
||||||
defaultConfig = new LogConfig(),
|
|
||||||
cleanerConfig = CleanerConfig(enableCleaner = false),
|
|
||||||
flushCheckMs = 1000L,
|
|
||||||
flushCheckpointMs = 100000L,
|
|
||||||
retentionCheckMs = 1000L,
|
|
||||||
scheduler = time.scheduler,
|
|
||||||
brokerState = new BrokerState(),
|
|
||||||
time = time)
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,6 +39,7 @@ import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder}
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.admin.AdminUtils
|
import kafka.admin.AdminUtils
|
||||||
import kafka.producer.ProducerConfig
|
import kafka.producer.ProducerConfig
|
||||||
|
import kafka.log._
|
||||||
|
|
||||||
import junit.framework.AssertionFailedError
|
import junit.framework.AssertionFailedError
|
||||||
import junit.framework.Assert._
|
import junit.framework.Assert._
|
||||||
|
@ -689,6 +690,30 @@ object TestUtils extends Logging {
|
||||||
def checkIfReassignPartitionPathExists(zkClient: ZkClient): Boolean = {
|
def checkIfReassignPartitionPathExists(zkClient: ZkClient): Boolean = {
|
||||||
ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath)
|
ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create new LogManager instance with default configuration for testing
|
||||||
|
*/
|
||||||
|
def createLogManager(
|
||||||
|
logDirs: Array[File] = Array.empty[File],
|
||||||
|
defaultConfig: LogConfig = LogConfig(),
|
||||||
|
cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false),
|
||||||
|
time: MockTime = new MockTime()) =
|
||||||
|
{
|
||||||
|
new LogManager(
|
||||||
|
logDirs = logDirs,
|
||||||
|
topicConfigs = Map(),
|
||||||
|
defaultConfig = defaultConfig,
|
||||||
|
cleanerConfig = cleanerConfig,
|
||||||
|
ioThreads = 4,
|
||||||
|
flushCheckMs = 1000L,
|
||||||
|
flushCheckpointMs = 10000L,
|
||||||
|
retentionCheckMs = 1000L,
|
||||||
|
scheduler = time.scheduler,
|
||||||
|
time = time,
|
||||||
|
brokerState = new BrokerState())
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
object TestZKUtils {
|
object TestZKUtils {
|
||||||
|
|
Loading…
Reference in New Issue