This is an automated email from the ASF dual-hosted git repository.

chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 57f8f66a288 KAFKA-20030 Document considerations and side effects of 
dynamic partition expansion (#21294)
57f8f66a288 is described below

commit 57f8f66a2885c558b80516c5b2deef9362c6e5f5
Author: Lan Ding <[email protected]>
AuthorDate: Sun Jan 18 00:46:00 2026 +0800

    KAFKA-20030 Document considerations and side effects of dynamic partition 
expansion (#21294)
    
    Add documentation for considerations and side effects of dynamic
    partition expansion.
    
    Reviewers: TengYao Chi <[email protected]>, Chia-Ping Tsai
     <[email protected]>
---
 docs/operations/basic-kafka-operations.md | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)

diff --git a/docs/operations/basic-kafka-operations.md 
b/docs/operations/basic-kafka-operations.md
index 700aa47c10d..4c3d3961a27 100644
--- a/docs/operations/basic-kafka-operations.md
+++ b/docs/operations/basic-kafka-operations.md
@@ -56,7 +56,12 @@ To add partitions you can do
     $ bin/kafka-topics.sh --bootstrap-server localhost:9092 --alter --topic 
my_topic_name \
         --partitions 40
 
-Be aware that one use case for partitions is to semantically partition data, 
and adding partitions doesn't change the partitioning of existing data so this 
may disturb consumers if they rely on that partition. That is if data is 
partitioned by `hash(key) % number_of_partitions` then this partitioning will 
potentially be shuffled by adding partitions but Kafka will not attempt to 
automatically redistribute data in any way. 
+**Note:** Dynamically increasing the number of partitions for a topic has 
several important considerations and potential side effects:
+
+  * **Key Distribution Changes**: If data is partitioned by `hash(key) % 
number_of_partitions`, the default partitioner's mapping logic changes when the 
partition count increases. This means that messages with the same key may be 
routed to different partitions after the expansion, potentially affecting 
message ordering guarantees for existing keys. Kafka will not attempt to 
automatically redistribute existing data.
+  * **Potential Data Loss with `auto.offset.reset=latest`**: Existing 
consumers configured with `auto.offset.reset=latest` might miss messages 
produced to the new partitions during the window between partition creation and 
consumer discovery. This occurs because consumers may not immediately detect 
the new partitions, and any messages produced to those partitions before the 
consumer rebalances will be skipped.
+  * **Metadata Propagation Delay**: New partitions are not immediately visible 
to producers and consumers due to metadata refresh intervals (controlled by 
`metadata.max.age.ms`). There will be a brief period where clients are unaware 
of the new partitions, which may result in uneven distribution of messages or 
consumer lag.
+  * **Risks with Internal Topics**: Users should **never** manually increase 
partitions for Kafka's internal state topics such as `__consumer_offsets`, 
`__transaction_state`, `__share_group_state`,  or `__cluster_metadata`. Doing 
so can break coordinator mapping logic, cause state inconsistencies, and lead 
to data corruption or system failures. These topics are managed automatically 
by Kafka and should not be modified manually. 
 
 To add configs: 
     

Reply via email to