KAFKA-18262: Remove DefaultPartitioner and UniformStickyPartitioner#18204
Merged
chia7712 merged 1 commit intoapache:trunkfrom Dec 20, 2024
Merged
KAFKA-18262: Remove DefaultPartitioner and UniformStickyPartitioner#18204chia7712 merged 1 commit intoapache:trunkfrom
chia7712 merged 1 commit intoapache:trunkfrom
Conversation
Yunyung
reviewed
Dec 16, 2024
Collaborator
Yunyung
left a comment
There was a problem hiding this comment.
LGTM, i left one minor.
00c0175 to
fe57284
Compare
chia7712
reviewed
Dec 18, 2024
Member
chia7712
left a comment
There was a problem hiding this comment.
@FrankYang0529 thanks for this patch. I'd like to push this to 4.0.0
| <li>The <code>kafka.common.requests.DescribeLogDirsResponse.ReplicaInfo</code> class was removed. Please use the <code>kafka.clients.admin.DescribeLogDirsResult.descriptions()</code> class | ||
| and <code>kafka.clients.admin.DescribeLogDirsResult.allDescriptions()</code>instead. | ||
| </li> | ||
| <li>The <code>org.apache.kafka.clients.producer.UniformStickyPartitioner</code> class was removed. |
Member
There was a problem hiding this comment.
Please add DefaultPartitioner as well
| final boolean autoComplete, | ||
| final Serializer<K> keySerializer, | ||
| final Serializer<V> valueSerializer) { | ||
| this(cluster, autoComplete, new org.apache.kafka.clients.producer.internals.DefaultPartitioner(), keySerializer, valueSerializer); |
Member
There was a problem hiding this comment.
MockProducer has a null partitioner and it may cause NPE. Could you please fix that too? We can pick up a partition based on cluster.partitionsForTopic(topic); if the partitioner is null
4eda25b to
7b64d48
Compare
Signed-off-by: PoAn Yang <payang@apache.org>
7b64d48 to
2c81bc7
Compare
Member
Author
|
Hi @chia7712, could you help me review this PR when you have time? Thank you. |
chia7712
approved these changes
Dec 20, 2024
chia7712
pushed a commit
that referenced
this pull request
Dec 20, 2024
…18204) Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
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
…pache#18204) Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
The DefaultPartitioner and UniformStickyPartitioner have been deprecated by KAFKA-10888 since 2022, so we should remove them from version 4.0.
Committer Checklist (excluded from commit message)