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-7487: DumpLogSegments misreports offset mismatches #5756

Merged
merged 3 commits into from Feb 18, 2019

Conversation

@ijuma
Copy link
Contributor

commented Oct 7, 2018

  • 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.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)
@ijuma ijuma force-pushed the ijuma:kafka-7487-dump-log-segments-mismatches branch Oct 7, 2018
@ijuma ijuma requested a review from hachikuji Oct 7, 2018
@ijuma ijuma force-pushed the ijuma:kafka-7487-dump-log-segments-mismatches branch 2 times, most recently Oct 7, 2018
@ijuma

This comment has been minimized.

Copy link
Contributor Author

commented Oct 8, 2018

Failures are unrelated and the JIRA reporter verified that these changes fix the issue.

Copy link
Contributor

left a comment

Minor nits. LGTM.

core/src/main/scala/kafka/tools/DumpLogSegments.scala Outdated

// since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one
if (entry.offset == index.baseOffset && i > 0)
return

This comment has been minimized.

Copy link
@harshach

harshach Oct 19, 2018

Contributor

Nit: it will be good to have a log statement to print the file. Help user quickly identify such sparse files.

This comment has been minimized.

Copy link
@ijuma

ijuma Oct 20, 2018

Author Contributor

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.

This comment has been minimized.

Copy link
@ijuma

ijuma Oct 25, 2018

Author Contributor

Are you OK with this @harshach?


// since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one
if (entry.offset == timeIndex.baseOffset && i > 0)
return

This comment has been minimized.

Copy link
@harshach

harshach Oct 19, 2018

Contributor

Nit: same as before

@ijuma ijuma force-pushed the ijuma:kafka-7487-dump-log-segments-mismatches branch Oct 20, 2018
ijuma added 2 commits Oct 7, 2018
- 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
@ijuma ijuma force-pushed the ijuma:kafka-7487-dump-log-segments-mismatches branch to e3c3e47 Feb 18, 2019
@ijuma

This comment has been minimized.

Copy link
Contributor Author

commented Feb 18, 2019

@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 checkArgs that should not have been, in my opinion. So, I moved it back.

@ijuma ijuma requested a review from omkreddy Feb 18, 2019
Copy link
Contributor

left a comment

@ijuma Thanks for the PR. LGTM

@omkreddy

This comment has been minimized.

Copy link
Contributor

commented Feb 18, 2019

retest this please

@ijuma

This comment has been minimized.

Copy link
Contributor Author

commented Feb 18, 2019

Retest this please

@ijuma

This comment has been minimized.

Copy link
Contributor Author

commented Feb 18, 2019

One unrelated failure and one build that timed out after 4 hours while running Streams tests.

org.apache.kafka.common.KafkaException: Could not produce message to topic=test-topic
	at org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster.produce(EmbeddedKafkaCluster.java:257)
	at org.apache.kafka.connect.integration.ExampleConnectIntegrationTest.testProduceConsumeConnector(ExampleConnectIntegrationTest.java:129)
@ijuma

This comment has been minimized.

Copy link
Contributor Author

commented Feb 18, 2019

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.

@ijuma ijuma merged commit e17352f into apache:trunk Feb 18, 2019
0 of 2 checks passed
0 of 2 checks passed
JDK 11 and Scala 2.12 FAILURE 10400 tests run, 69 skipped, 0 failed.
Details
JDK 8 and Scala 2.11 FAILURE 10567 tests run, 69 skipped, 1 failed.
Details
@ijuma ijuma deleted the ijuma:kafka-7487-dump-log-segments-mismatches branch Feb 18, 2019
jarekr pushed a commit to confluentinc/kafka that referenced this pull request Apr 18, 2019
* 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)
  ...
Pengxiaolong added a commit to Pengxiaolong/kafka that referenced this pull request Jun 14, 2019
- 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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants
You can’t perform that action at this time.