Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-12553: Refactor recovery logic to introduce LogLoader #10478

Merged
merged 7 commits into from
Apr 20, 2021

Conversation

kowshik
Copy link
Contributor

@kowshik kowshik commented Apr 5, 2021

In this PR, I have refactored the recovery logic code introducing a new class kafka.log.LogLoader responsible for all activities related with recovery of log segments from disk. With this change, the recovery logic has been moved out of the Log class and into the new LogLoader class.

Advantages:
This refactor has the following advantages over the existing code:

  • As such, the recovery logic is invoked once only during Log instantiation. Some parts of the recovery logic are fairly independent from the rest of the Log class. By moving the independent private logic to a separate LogLoader class, the existing Log class has become more modular, and the constructor behavior is a lot simpler now. Therefore, this makes the code more maintainable.
  • This PR takes us a step closer towards the Log layer reactor work (KAFKA-12554). The Log recovery logic reads and writes to LeaderEpochFileCache and ProducerStateManager instances, so as such the logic does not fit very well into the definition of a "local log". By extracting it out of the Log class, in the future this will make it much easier to clearly define the separation of concerns between LocalLog and UnifiedLog.

Note to reviewers:
Most of this PR is about moving code around (including tests) and sharing static functions wherever possible. In order to start reviewing this PR, it is best to start looking at the sequence of operations when a Log instance is created (see Log.apply()):

  1. First LogLoader loads/recovers the segments from disk and using the metadata it also suitably sets up the related components viz. log start offset, recovery point, next offset, leader epoch cache and producer state manager. The entry point for the LogLoader is the LogLoader.load() API. The implementation is pretty much identical to the previously existing Log recovery logic.
  2. Next, the log components setup by the LogLoader are passed to the Log instance during construction. Some of these components are further mutable by the Log instance during its lifecycle. The rest of the Log class pretty much remains intact, except that some private methods have been converted to static functions to be shared with the recovery logic.

Tests:
I'm relying on the newly added LogLoaderTest.scala for testing. I have moved 35 recovery related tests out of LogTest.scala and into the new LogLoaderTest.scala test suite.

Copy link
Contributor Author

@kowshik kowshik left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

First pass

core/src/main/scala/kafka/log/LogLoader.scala Outdated Show resolved Hide resolved
core/src/main/scala/kafka/log/Log.scala Outdated Show resolved Hide resolved
core/src/main/scala/kafka/log/Log.scala Outdated Show resolved Hide resolved
core/src/main/scala/kafka/log/Log.scala Outdated Show resolved Hide resolved
core/src/main/scala/kafka/log/Log.scala Outdated Show resolved Hide resolved
core/src/main/scala/kafka/log/Log.scala Show resolved Hide resolved
@kowshik kowshik force-pushed the refactor_extract_recovery_logic branch 16 times, most recently from c978b67 to a236063 Compare April 7, 2021 23:39
@kowshik kowshik changed the title KAFKA-12553: Refactor recovery logic KAFKA-12553: Refactor recovery logic to introduce LogLoader Apr 8, 2021
@kowshik kowshik marked this pull request as ready for review April 8, 2021 00:10
@kowshik
Copy link
Contributor Author

kowshik commented Apr 8, 2021

cc @dhruvilshah3 @junrao @satishd -- this PR is ready for a review.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kowshik : Thanks for the PR. A few comments below.

@@ -2586,11 +1999,15 @@ object Log {
logDirFailureChannel: LogDirFailureChannel,
lastShutdownClean: Boolean = true,
topicId: Option[Uuid],
keepPartitionMetadataFile: Boolean): Log = {
keepPartitionMetadataFile: Boolean = true): Log = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the default value needed? If so, it seems many tests are not taking advantage of this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. I've removed it. It's not needed, I believe it showed up accidentally. My bad.

val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
new Log(dir, config, logStartOffset, recoveryPoint, scheduler, brokerTopicStats, time, maxProducerIdExpirationMs,
producerIdExpirationCheckIntervalMs, topicPartition, producerStateManager, logDirFailureChannel, lastShutdownClean, topicId, keepPartitionMetadataFile)
val logLoader = new LogLoader(dir, topicPartition, config, scheduler, time, logDirFailureChannel)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need LogLoader to be a class? Another option is to have LogLoader as an Object with a single public method load().

Copy link
Contributor Author

@kowshik kowshik Apr 12, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great point. I tried this, but few problems I ran into were:

  1. A test tries to intercept the recovery logic by overriding def recoverLog in LogLoader. Ex: LogLoaderTest.testLogRecoveryIsCalledUponBrokerCrash.
  2. Some tests like to inject a custom ProducerStateManager, ex: LogLoaderTest.testSkipTruncateAndReloadIfNewMessageFormatAndCleanShutdown.
  3. Some tests like to inject a custom LogSegments, ex: LogLoaderTest. testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat.

I'm working on solving these issues, and will post a PR update soon.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. I've updated the PR converting LogLoader into an object.

@volatile var topicId: Option[Uuid],
val keepPartitionMetadataFile: Boolean) extends Logging with KafkaMetricsGroup {
val keepPartitionMetadataFile: Boolean = true) extends Logging with KafkaMetricsGroup {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the default value needed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. I've removed it. It's not needed, I believe it showed up accidentally. My bad.

sealed trait SegmentDeletionReason {
def logReason(log: Log, toDelete: List[LogSegment]): Unit
abstract class SegmentDeletionReason {
def logReason(logger: Logging, toDelete: List[LogSegment]): Unit = {}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, this method is being used, but no subclass implements it. Are we losing logging because of this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great catch. After an investigation, I realized this change is no longer required and I've reverted it now.

writeNormalSegment(nextOffset)
}

def allRecords(log: Log): List[Record] = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could this be private?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. With def verifyRecordsInLog eliminated, I was able to eliminate this one too. Related comment: #10478 (comment).

recordsFound.toList
}

def verifyRecordsInLog(log: Log, expectedRecords: List[Record]): Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this method used?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch. It was unused and I've eliminated it.

Copy link
Member

@satishd satishd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @kowshik for the PR, overall LGTM. Left a minor comment.

core/src/main/scala/kafka/log/Log.scala Show resolved Hide resolved
@kowshik
Copy link
Contributor Author

kowshik commented Apr 12, 2021

@junrao @satishd: Thanks for the review! I've responded to your comments and addressed them in 8f068caa78a9998c14a18f4583eaaa8a90cba71c, except for this comment - I'm still working on this and will update the PR soon.

@kowshik kowshik force-pushed the refactor_extract_recovery_logic branch from 8f068ca to debcbd5 Compare April 12, 2021 09:09
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kowshik : Thanks for the updated PR. A few more comments.

*
* @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that
* overflow index offset; or when we find an unexpected
* number of .log files with overflow
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that this comment is no longer accurate. The only place that can throw LogSegmentOffsetOverflowException is from completeSwapOperations(). The rest of the places are all wrapped with retryOnOffsetOverflow().

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. I've fixed it.

time = params.time,
initFileSize = params.config.initFileSize))
}
(params.recoveryPointCheckpoint, 0L)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since the nextOffset is 0, recoveryPoint needs to be 0 too.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great catch. I've fixed it now.

* @param config The configuration settings for the log being loaded
* @param scheduler The thread pool scheduler used for background actions
* @param time The time instance used for checking the clock
* @param logDirFailureChannel The LogDirFailureChannel instance to asynchronously handle log
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently, if we hit any IOException during loading, we just kill the broker. So, it seems that there is no need to pass in logDirFailureChannel during log loading.

Copy link
Contributor Author

@kowshik kowshik Apr 15, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the recovery path, the logDirFailureChannel is used to asynchronously handle IOException within the Log.deleteSegmentFiles() calls. This was the existing behavior prior to this refactor as well. Should we change the existing behavior?https://github.com/apache/kafka/blob/193a9dfc44fa2c0df163a8518a804c4579d37e36/core/src/main/scala/kafka/log/Log.scala#L2377

* to reflect the contents of the loaded log.
*
* This method does not need to convert IOException to KafkaStorageException because it is only
* called before all logs are loaded.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The loading code still converts IOException to KafkaStorageException through Log.deleteSegmentFiles().

Copy link
Contributor Author

@kowshik kowshik Apr 15, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree this comment was a bit misleading. I've improved it now to say the following, do you feel its better?

In the context of the calling thread, this function does not need to convert
IOException to KafkaStorageException because it is only called before all logs are loaded.

In the recovery path, we always asynchronously delete segments. The IOException conversion happens only within the background work that's scheduled for segment deletions. https://github.com/apache/kafka/blob/193a9dfc44fa2c0df163a8518a804c4579d37e36/core/src/main/scala/kafka/log/Log.scala#L2377

@@ -222,25 +222,31 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
*
* @param _dir The directory in which log segments are created.
* @param config The log configuration settings
* @param segments The log segments, these may be non-empty when recovered from disk
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The segments is always non-empty since we initialize it with an empty segment if no segments are present.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. Fixed it.

@kowshik
Copy link
Contributor Author

kowshik commented Apr 16, 2021

@junrao: Thanks for the review! I have addressed the latest comments and also fixed the broken unit test in PartitionLockTest suite. This PR is ready for review.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kowshik : Thanks for the updated PR. LGTM. Could you rebase?

@kowshik kowshik force-pushed the refactor_extract_recovery_logic branch from bdf0567 to eaf8519 Compare April 20, 2021 07:10
@kowshik
Copy link
Contributor Author

kowshik commented Apr 20, 2021

Thanks for the review @junrao and @satishd! I've updated this PR after rebasing onto latest commit in trunk.

@junrao junrao merged commit 0b46441 into apache:trunk Apr 20, 2021
@kowshik
Copy link
Contributor Author

kowshik commented Apr 20, 2021

I checked the failed tests in CI this morning and none of them are related with this PR:

org.apache.kafka.connect.*
kafka.admin.ResetConsumerGroupOffsetTest.testResetOffsetsToZonedDateTime()
kafka.server.RaftClusterTest.testCreateClusterAndCreateListDeleteTopic()
kafka.integration.MetricsDuringTopicCreationDeletionTest.testMetricsDuringTopicCreateDelete()

junrao pushed a commit that referenced this pull request May 24, 2021
…ns (#10742)

When #10478 was merged, we accidentally lost the identifier/prefix string that we used to previously log to stderr from some of the functions in the Log class. In this PR, I have reinstated the identifier/prefix logging in these functions, so that the debuggability is restored.

Reviewers: Luke Chen <showuon@gmail.com>, Cong Ding <cong@ccding.com>, Jun Rao <junrao@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants