[
https://issues.apache.org/jira/browse/KAFKA-1414?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14056358#comment-14056358
]
Alexey Ozeritskiy commented on KAFKA-1414:
------------------------------------------
1. Ok
2. If any thread fails we get ExecutionException at
{code}
jobs.foreach(_.get())
{code}
We can get the original exception by calling e.getCause() and rethrow it. Is
this ok ?
> Speedup broker startup after hard reset
> ---------------------------------------
>
> Key: KAFKA-1414
> URL: https://issues.apache.org/jira/browse/KAFKA-1414
> Project: Kafka
> Issue Type: Improvement
> Components: log
> Affects Versions: 0.8.2, 0.9.0, 0.8.1.1
> Reporter: Dmitry Bugaychenko
> Assignee: Jay Kreps
> Attachments: parallel-dir-loading-0.8.patch,
> parallel-dir-loading-trunk-threadpool.patch, parallel-dir-loading-trunk.patch
>
>
> After hard reset due to power failure broker takes way too much time
> recovering unflushed segments in a single thread. This could be easiliy
> improved launching multiple threads (one per data dirrectory, assuming that
> typically each data directory is on a dedicated drive). Localy we trie this
> simple patch to LogManager.loadLogs and it seems to work, however I'm too new
> to scala, so do not take it literally:
> {code}
> /**
> * Recover and load all logs in the given data directories
> */
> private def loadLogs(dirs: Seq[File]) {
> val threads : Array[Thread] = new Array[Thread](dirs.size)
> var i: Int = 0
> val me = this
> for(dir <- dirs) {
> val thread = new Thread( new Runnable {
> def run()
> {
> val recoveryPoints = me.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))
> for(dir <- subDirs) {
> if(dir.isDirectory) {
> info("Loading log '" + dir.getName + "'")
> val topicPartition = Log.parseTopicPartitionName(dir.getName)
> val config = topicConfigs.getOrElse(topicPartition.topic,
> defaultConfig)
> val log = new Log(dir,
> config,
> recoveryPoints.getOrElse(topicPartition, 0L),
> scheduler,
> time)
> val previous = addLogWithLock(topicPartition, log)
> if(previous != null)
> throw new IllegalArgumentException("Duplicate log
> directories found: %s, %s!".format(log.dir.getAbsolutePath,
> previous.dir.getAbsolutePath))
> }
> }
> cleanShutDownFile.delete()
> }
> }
> })
> thread.start()
> threads(i) = thread
> i = i + 1
> }
> for(thread <- threads) {
> thread.join()
> }
> }
> def addLogWithLock(topicPartition: TopicAndPartition, log: Log): Log = {
> logCreationOrDeletionLock synchronized {
> this.logs.put(topicPartition, log)
> }
> }
> {code}
--
This message was sent by Atlassian JIRA
(v6.2#6252)