Skip to content

KAFKA-7693: Fix SequenceNumber overflow in client TransactionManager.#5989

Merged
hachikuji merged 7 commits intoapache:trunkfrom
mingaliu:1202clientsequnceid
Jan 25, 2019
Merged

KAFKA-7693: Fix SequenceNumber overflow in client TransactionManager.#5989
hachikuji merged 7 commits intoapache:trunkfrom
mingaliu:1202clientsequnceid

Conversation

@mingaliu
Copy link
Copy Markdown
Contributor

@mingaliu mingaliu commented Dec 3, 2018

This is to fix KAFKA-7693. The bug is found during operating the Kafka cluster.
The problem is SequenceNumber is Int and should wrap around when it hit the int.MaxValue.
The bug here is it doesn't wrap around and become negative and throw the Exception mentioned in the JIRA-7693.

Verified in the unitest and our daily testing environment.

Committer Checklist (excluded from commit message)

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

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Dec 3, 2018

Cc @hachikuji

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Dec 3, 2018

Thanks for the PR. Would it be possible to add a test for this?

@ijuma ijuma requested review from hachikuji and junrao December 8, 2018 17:34
@mingaliu
Copy link
Copy Markdown
Contributor Author

@ijuma Thanks for the comments, I added the unitest, please review again.

@mingaliu
Copy link
Copy Markdown
Contributor Author

Tested multiple times and succeed locally, this is due to other random test failure. @ijuma, @hachikuji , please review.

@mingaliu
Copy link
Copy Markdown
Contributor Author

Please review or sign off, @junrao , @hachikuji , thanks!

Copy link
Copy Markdown
Contributor

@hachikuji hachikuji 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!

@hachikuji hachikuji merged commit 0f926f0 into apache:trunk Jan 25, 2019
hachikuji pushed a commit that referenced this pull request Jan 25, 2019
The problem is that the sequence number is an Int and should wrap around when it reaches the Int.MaxValue. The bug here is it doesn't wrap around and become negative and raises an error.

Reviewers: Jason Gustafson <jason@confluent.io>
hachikuji pushed a commit that referenced this pull request Jan 25, 2019
The problem is that the sequence number is an Int and should wrap around when it reaches the Int.MaxValue. The bug here is it doesn't wrap around and become negative and raises an error.

Reviewers: Jason Gustafson <jason@confluent.io>
hachikuji pushed a commit that referenced this pull request Jan 25, 2019
The problem is that the sequence number is an Int and should wrap around when it reaches the Int.MaxValue. The bug here is it doesn't wrap around and become negative and raises an error.

Reviewers: Jason Gustafson <jason@confluent.io>
jarekr pushed a commit to confluentinc/kafka that referenced this pull request Apr 18, 2019
* ak/trunk:
  MINOR: Update usage of deprecated API (apache#6146)
  KAFKA-4217: Add KStream.flatTransform (apache#5273)
  MINOR: Update Gradle to 5.1.1 (apache#6160)
  KAFKA-3522: Generalize Segments (apache#6170)
  Added quotes around the class path (apache#4469)
  KAFKA-7837: Ensure offline partitions are picked up as soon as possible when shrinking ISR (apache#6202)
  MINOR: In the MetadataResponse schema, ignorable should be a boolean
  KAFKA-7838: Log leader and follower end offsets when shrinking ISR (apache#6168)
  KAFKA-5692: Change PreferredReplicaLeaderElectionCommand to use Admin… (apache#3848)
  MINOR: clarify why suppress can sometimes drop tombstones (apache#6195)
  MINOR: Upgrade ducktape to 0.7.5 (apache#6197)
  MINOR: Improve IntegrationTestUtils documentation (apache#5664)
  MINOR: upgrade to jdk8 8u202
  KAFKA-7693; Fix SequenceNumber overflow in producer (apache#5989)
  KAFKA-7692; Fix ProducerStateManager SequenceNumber overflow (apache#5990)
  MINOR: update copyright year in the NOTICE file. (apache#6196)
  KAFKA-7793: Improve the Trogdor command line. (apache#6133)
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
The problem is that the sequence number is an Int and should wrap around when it reaches the Int.MaxValue. The bug here is it doesn't wrap around and become negative and raises an error.

Reviewers: Jason Gustafson <jason@confluent.io>
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.

3 participants