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-14544 The "is-future" should be removed from metrics tags after future log becomes current log #12979

Merged
merged 9 commits into from Dec 26, 2022

Conversation

chia7712
Copy link
Contributor

@chia7712 chia7712 commented Dec 12, 2022

we don't remove "is-future=true" tag from future log after the future log becomes "current" log. It causes two potential issues:

  1. the metrics monitors can't get metrics of Log if they don't trace the property "is-future=true".
  2. all Log metrics of specify partition get removed if the partition is moved to another folder again.

https://issues.apache.org/jira/browse/KAFKA-14544

Committer Checklist (excluded from commit message)

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

@showuon
Copy link
Contributor

showuon commented Dec 20, 2022

Thanks for the patch. I'll take a look this week.

@chia7712 chia7712 changed the title MINOR: remove "is-future" from metrics tags after replace current log… KAFKA-14544 The "is-future" should be removed from metrics tags after future log becomes current log Dec 21, 2022
@chia7712
Copy link
Contributor Author

file a jira as this is related to true bug

Copy link
Contributor

@showuon showuon left a comment

Choose a reason for hiding this comment

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

@chia7712 , thanks for the PR. Left one comment.

Comment on lines 1084 to 1086
destLog.removeLogMetrics()
destLog.renameDir(UnifiedLog.logDirName(topicPartition), true)
destLog.updateHighWatermark(sourceLog.highWatermark)
Copy link
Contributor

@showuon showuon Dec 22, 2022

Choose a reason for hiding this comment

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

Do you think we should remove log metrics after destLog got renamed? I'm thinking if we got stuck for some time during renameDir and it'll be helpful to have metrics reporting it. WDYT?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

it'll be helpful to have metrics reporting it

make sense. will copy that

Copy link
Contributor

@showuon showuon left a comment

Choose a reason for hiding this comment

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

LGTM! Thanks for the fix!

@chia7712
Copy link
Contributor Author

the new test is unstable. Will fix it later.

@chia7712
Copy link
Contributor Author

@showuon Could you take a look again? the latest commit includes extra changes to address your comment.

Comment on lines 1805 to +1809
private[log] def removeLogMetrics(): Unit = {
removeMetric(LogMetricNames.NumLogSegments, tags)
removeMetric(LogMetricNames.LogStartOffset, tags)
removeMetric(LogMetricNames.LogEndOffset, tags)
removeMetric(LogMetricNames.Size, tags)
metricNames.foreach {
case (name, tags) => removeMetric(name, tags)
}
metricNames = Map.empty
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 explain why we need this change to remove metrics, instead of the original one?

Copy link
Contributor

Choose a reason for hiding this comment

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

Is it because when we tried to removeMetrics, the tags might become "non-future"?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Is it because when we tried to removeMetrics, the tags might become "non-future"?

yep. The tags DOES NOT include "future" after the log dir get renamed. Hence, we have to keep previous metrics name in order to delete correct metrics.

Copy link
Contributor

@showuon showuon left a comment

Choose a reason for hiding this comment

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

LGTM! Thanks.

@chia7712 chia7712 merged commit 13d1c08 into apache:trunk Dec 26, 2022
guozhangwang pushed a commit to guozhangwang/kafka that referenced this pull request Jan 25, 2023
… future log becomes current log (apache#12979)

Reviewers: Luke Chen <showuon@gmail.com>
@chia7712 chia7712 deleted the minor-12979 branch March 25, 2024 15:20
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
2 participants