KAFKA-7487: DumpLogSegments misreports offset mismatches#5756
KAFKA-7487: DumpLogSegments misreports offset mismatches#5756ijuma merged 3 commits intoapache:trunkfrom
Conversation
32e4201 to
2f13f89
Compare
9e31496 to
8464a0c
Compare
|
Failures are unrelated and the JIRA reporter verified that these changes fix the issue. |
There was a problem hiding this comment.
Nit: it will be good to have a log statement to print the file. Help user quickly identify such sparse files.
There was a problem hiding this comment.
This is pretty common and not a reason to be worried. If we want to highlight this, it's best to do it in a separate PR, I think.
8464a0c to
c9faf2f
Compare
- Compare last offset of first batch with index offset
- Early exit from loop due to zero entries must happen before checking for mismatch
- {TimeIndex,OffsetIndex}.entry should return absolute offset like other methods. These methods are
only used by DumpLogSegments.
- Add OffsetIndex, TimeIndex and DumpLogSegments tests
- Remove unnecessary casts by using covariant returns in OffsetIndex and TimeIndex
- Minor clean-ups
c9faf2f to
e3c3e47
Compare
|
@omkreddy, do you have the cycles to review this? I've rebased and fixed the conflicts. There were more than expected as some code was moved to |
|
retest this please |
|
Retest this please |
|
One unrelated failure and one build that timed out after 4 hours while running Streams tests. |
|
Since we had some successful builds (I was hoping to have 2 greens) and the failures are all after all the Core tests have passed (and this mostly changes a Core tool), I'll go ahead and merge this. |
* 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) ...
- Compare last offset of first batch (instead of first offset) with index offset
- Early exit from loop due to zero entries must happen before checking for mismatch
- {TimeIndex,OffsetIndex}.entry should return absolute offset like other methods.
These methods are only used by DumpLogSegments.
- DumpLogSegments now calls `closeHandlers` on OffsetIndex, TimeIndex
and FileRecords.
- Add OffsetIndex, TimeIndex and DumpLogSegments tests
- Remove unnecessary casts by using covariant returns in OffsetIndex and TimeIndex
- Minor clean-ups
- Fix `checkArgs` so that it does what it says only.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Sriharsha Chintalapani <sriharsha@apache.org>
These methods are only used by DumpLogSegments.
closeHandlerson OffsetIndex, TimeIndexand FileRecords.
checkArgsso that it does what it says only.Committer Checklist (excluded from commit message)