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-6569: Move OffsetIndex/TimeIndex logger to companion object #4586
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the PR. Would it be worth it to maintain the existing behaviour by having each AbstractIndex subclass provide the Logger? Something like:
class OffsetIndex {
def logger = OffsetIndex.logger
}
@ambroff what do you think of the feedback above? |
We identified that we spend a lot of time in reflection when creating OffsetIndex, TimeIndex, or other implementations of AbstractIndex[K, V], because of the Logging mixin. When the broker is bootstrapping it's just doing this in a tight loop, so this time adds up. This patch moves the logging to a companion objects, statically initializing the logger.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This LGTM with the commit I added. @omkreddy, can you please take a quick look at this too? It's a pretty small change.
@@ -51,7 +50,7 @@ import org.apache.kafka.common.record.RecordBatch | |||
*/ | |||
// Avoid shadowing mutable file in AbstractIndex | |||
class TimeIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: Boolean = true) | |||
extends AbstractIndex[Long, Long](_file, baseOffset, maxIndexSize, writable) with Logging { | |||
extends AbstractIndex[Long, Long](_file, baseOffset, maxIndexSize, writable) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This class also should provide the Logger, Right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There are no log statements in this class.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There is one statement at https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/log/TimeIndex.scala#L129
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch! CoreUtils was exposing its logger, so I fixed both in a new commit.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the updated PR. LGTM.
Tests passed, merging to trunk. |
* ak/trunk: (45 commits) KAFKA-7487: DumpLogSegments misreports offset mismatches (apache#5756) MINOR: improve JavaDocs about auto-repartitioning in Streams DSL (apache#6269) KAFKA-7935: UNSUPPORTED_COMPRESSION_TYPE if ReplicaManager.getLogConfig returns None (apache#6274) KAFKA-7895: Fix stream-time reckoning for suppress (apache#6278) KAFKA-6569: Move OffsetIndex/TimeIndex logger to companion object (apache#4586) MINOR: add log indicating the suppression time (apache#6260) MINOR: Make info logs for KafkaConsumer a bit more verbose (apache#6279) KAFKA-7758: Reuse KGroupedStream/KGroupedTable with named repartition topics (apache#6265) KAFKA-7884; Docs for message.format.version should display valid values (apache#6209) MINOR: Save failed test output to build output directory MINOR: add test for StreamsSmokeTestDriver (apache#6231) MINOR: Fix bugs identified by compiler warnings (apache#6258) KAFKA-6474: Rewrite tests to use new public TopologyTestDriver [part 4] (apache#5433) MINOR: fix bypasses in ChangeLogging stores (apache#6266) MINOR: Make MockClient#poll() more thread-safe (apache#5942) MINOR: drop dbAccessor reference on close (apache#6254) KAFKA-7811: Avoid unnecessary lock acquire when KafkaConsumer commits offsets (apache#6119) KAFKA-7916: Unify store wrapping code for clarity (apache#6255) MINOR: Add missing Alter Operation to Topic supported operations list in AclCommand KAFKA-7921: log at error level for missing source topic (apache#6262) ...
…ache#4586) We identified that we spend a lot of time in the creation of Logger instances when creating OffsetIndex/TimeIndex due to the Logging mixin. When the broker is bootstrapping it's just doing this in a tight loop, so the time adds up. This patch moves the logger to the companion objects of OffsetIndex, TimeIndex and AbstractIndex resolving this issue. Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com> Co-authored-by: Kyle Ambroff <kyle@ambroff.com> Co-authored-by: Ismael Juma <ismael@juma.me.uk>
We identified that we spend a lot of time in reflection when creating
OffsetIndex, TimeIndex, or other implementations of
AbstractIndex[K, V], because of the Logging mixin. When the broker is
bootstrapping it's just doing this in a tight loop, so this time adds
up.
This patch moves the logging to a companion objects, statically
initializing the logger.
More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.
Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.
Committer Checklist (excluded from commit message)