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-18207: Serde for handling transaction records #18136

Merged
merged 3 commits into from
Dec 19, 2024

Conversation

mimaison
Copy link
Member

This removes transaction-coordinator as a dependency of storage. This prevented making transaction-coordinator depend on coordinator-common because coordinator-common itself depends on storage thus making a loop. transaction-coordinator was only used in storage tests to get one constant.

The new Serde, TransactionCoordinatorRecordSerde, mimics GroupCoordinatorRecordSerde and is also used in DumpLogSegments instead of TransactionLog. Following this PR, we will now be able to move DumpLogSegments to tools as TransactionLog was the last class it depended onto from core.

We should be able to refactor TransactionLog to use TransactionCoordinatorRecordSerde, or even completely replace it. But let's do that in follow up PRs.

Committer Checklist (excluded from commit message)

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

@github-actions github-actions bot added core Kafka Broker storage Pull requests that target the storage module build Gradle build or GitHub Actions transactions Transactions and EOS labels Dec 11, 2024
Copy link
Member

@AndrewJSchofield AndrewJSchofield left a comment

Choose a reason for hiding this comment

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

Overall, looks good to me and achieves the overall aim nicely. Just one question though before I approve.

It seems that the dump log support for the transaction coordinator records was very rudimentary previously, and I think this PR will improve it just by using the JsonConverters instead of TransactionLog.formatRecordKeyAndValue. However, the values can be quite complicated these days and I wonder whether it's worth adding a complicated record to the tests also.

@@ -144,32 +144,6 @@ object TransactionLog {
}
}

Copy link
Member

Choose a reason for hiding this comment

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

nit: I'd remove this blank line too.

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

).getMessage
)

// A valid key and value should work.
Copy link
Member

Choose a reason for hiding this comment

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

I wonder whether we ought to have some testing of a more complicated value too. There are tagged fields and so on to consider.

Copy link
Member Author

Choose a reason for hiding this comment

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

I've added a testcase with all fields set for TransactionValue. Is this what you had in mind?

Copy link
Member

Choose a reason for hiding this comment

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

I think that will do. The point is to catch a situation where we inadvertently break this.

@mimaison mimaison merged commit 3fafa09 into apache:trunk Dec 19, 2024
9 checks passed
@mimaison mimaison deleted the kafka-18207 branch December 19, 2024 20:39
ijuma added a commit to ijuma/kafka that referenced this pull request Dec 20, 2024
…e-old-protocol-versions

* apache-github/trunk:
  KAFKA-18312: Added entityType: topicName to SubscribedTopicNames in ShareGroupHeartbeatRequest.json (apache#18285)
  HOTFIX: fix incompatible types: Optional<TimestampAndOffset> cannot be converted to Option<TimestampAndOffset> (apache#18284)
  MINOR Fix some test-catalog issues (apache#18272)
  KAFKA-18180: Move OffsetResultHolder to storage module (apache#18100)
  KAFKA-18301; Make coordinator records first class citizen (apache#18261)
  KAFKA-18262 Remove DefaultPartitioner and UniformStickyPartitioner (apache#18204)
  KAFKA-18296 Remove deprecated KafkaBasedLog constructor (apache#18257)
  KAFKA-12829: Remove old Processor and ProcessorSupplier interfaces (apache#18238)
  KAFKA-18292 Remove deprecated methods of UpdateFeaturesOptions (apache#18245)
  KAFKA-12829: Remove deprecated Topology#addProcessor of old Processor API (apache#18154)
  KAFKA-18035, KAFKA-18306, KAFKA-18092: Address TransactionsTest flaky tests (apache#18264)
  MINOR: change the default linger time in the new coordinator (apache#18274)
  KAFKA-18305: validate controller.listener.names is not in inter.broker.listener.name for kcontrollers (apache#18222)
  KAFKA-18207: Serde for handling transaction records (apache#18136)
  KAFKA-13722: Refactor Kafka Streams store interfaces (apache#18243)
  KAFKA-17131: Refactor TimeDefinitions (apache#18241)
  MINOR: Fix MessageFormatters (apache#18266)
  Mark flaky tests for Dec 18, 2024 (apache#18263)
tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
build Gradle build or GitHub Actions core Kafka Broker storage Pull requests that target the storage module transactions Transactions and EOS
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants