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-14482: Move LogLoader to storage module #17042

Merged
merged 3 commits into from
Sep 16, 2024

Conversation

mimaison
Copy link
Member

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Copy link
Contributor

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

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

@mimaison thanks for nice rewriting.

@@ -1775,7 +1776,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
producerExpireCheck.cancel(true)
leaderEpochCache.foreach(_.clear())
val deletedSegments = localLog.deleteAllSegments()
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe deleteAllSegments can return java collection directly

Copy link
Member Author

Choose a reason for hiding this comment

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

Good point!

* @param function The function to be executed.
* @return The value returned by the function after a successful invocation
*/
public static <T> T maybeHandleIOException(LogDirFailureChannel logDirFailureChannel,
Copy link
Contributor

Choose a reason for hiding this comment

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

Could you please replace LocalLog.scala#maybeHandleIOException by java version?

Copy link
Member Author

Choose a reason for hiding this comment

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

This requires a bunch of changes in LocalLog. I think it's best if we do those when we move LocalLog.

Copy link
Contributor

Choose a reason for hiding this comment

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

my point was to leverage LocalLog.java#maybeHandleIOException to implement LocalLog.scala#maybeHandleIOException. That can remove the duplicate code. for example:

  private def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
    JLocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, () => msg, () => fun)
  }

Copy link
Member Author

Choose a reason for hiding this comment

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

Ah right, I see what you meant now. Done

@@ -906,7 +907,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
updateHighWatermarkWithLogEndOffset()

// update the producer state
updatedProducers.values.foreach(producerAppendInfo => producerStateManager.update(producerAppendInfo))
updatedProducers.values.asScala.foreach(producerAppendInfo => producerStateManager.update(producerAppendInfo))
Copy link
Contributor

Choose a reason for hiding this comment

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

how about updatedProducers.values.forEach

Copy link
Member Author

Choose a reason for hiding this comment

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

Right we don't need this conversion.

}

private[log] def splitOverflowedSegment(segment: LogSegment): List[LogSegment] = lock synchronized {
val result = UnifiedLog.splitOverflowedSegment(segment, localLog.segments, dir, topicPartition, config, scheduler, logDirFailureChannel, logIdent)
deleteProducerSnapshots(result.deletedSegments, asyncDelete = true)
result.newSegments.toList
result.newSegments.asScala.toList
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 the return value is never used, so maybe we can remove it.

Copy link
Member Author

Choose a reason for hiding this comment

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

The return value is used in a few places in LogLoaderTest.

Copy link
Contributor

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

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

LGTM

@chia7712 chia7712 merged commit f1c011a into apache:trunk Sep 16, 2024
7 of 9 checks passed
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.

2 participants