Andrew - your example would give him a replication factor of 2 though, and it sounds like he wants 3 unless I missed something. So add an additional broker id to each of the replicas arrays in your example and you'd have an RF of 3.
-alex On Sat, Dec 17, 2022 at 7:12 AM Andrew Grant <agr...@confluent.io.invalid> wrote: > Here's an example with four partitions which should be pretty easily > extended to include up to partition 49: > { > "version": 1, > "partitions": [ > { > "topic": "__consumer_offsets", > "partition": 0, > "replicas": [ 0, 1 ] > }, > { > "topic": "__consumer_offsets", > "partition": 1, > "replicas": [ 1, 2 ] > }, > { > "topic": "__consumer_offsets", > "partition": 2, > "replicas": [ 2, 3 ] > }, > { > "topic": "__consumer_offsets", > "partition": 3, > "replicas": [ 3, 4 ] > }, > { > "topic": "__consumer_offsets", > "partition": 4, > "replicas": [ 4, 0 ] > } > ] > } > > > Andrew > > On Fri, Dec 16, 2022 at 6:54 PM Chris Peart <ch...@peart.me.uk> wrote: > > > Hi Andrew, > > Would you be able to provide an example of the json with all the > > partitions in please, I tried this on our dev cluster but it didn’t work. > > Many Thanks, > > Chris > > > > > On 16 Dec 2022, at 9:03 pm, Andrew Grant <agr...@confluent.io.invalid> > > wrote: > > > > > > Hey Chris, > > > > > > You'd need to do the same for all partitions. I just showed partition > 49 > > as > > > an example - I picked 49 because when I ran a describe it showed up at > > the > > > bottom of my terminal :) You could do all the partitions in the same > > > reassignment. In that JSON I just put partition 49 but you could add > all > > > the other partitions in it. > > > > > > Yeah I'm pretty sure you'd do basically the same for > > __transaction_state. I > > > havent tested that myself locally so might be worth doing so on your > end. > > > > > > Hope that helps a bit. > > > > > > Andrew > > > > > >> On Fri, Dec 16, 2022 at 11:41 AM Chris Peart <ch...@peart.me.uk> > wrote: > > >> > > >> Hi Andrew, > > >> > > >> Thanks for the speedy reply, so do I just need to do this for > partition > > >> 49? What about partitions 0-48, will these be covered by reassigning > > >> partition 49. > > >> > > >> Do I need to do this for the __transaction_state topics too? > > >> > > >> Many thanks, > > >> Chris > > >> > > >>> On 16 Dec 2022, at 4:17 pm, Andrew Grant <agr...@confluent.io.invalid > > > > >> wrote: > > >>> > > >>> Hey Chris, > > >>> I think you should be able to use the reassignment tool to add > > replicas. > > >>> You should be able to do something similar to migrate the partitions > > away > > >>> from the old brokers and onto the new ones and also increase the > > >>> replication factor at the same time. I tested just increasing the > > >>> replication factor with the following commands: > > >>> > > >>> kafka % ./bin/kafka-topics.sh --bootstrap-server localhost:9092 > --topic > > >>> __consumer_offsets --describe | grep 'Partition: 49' > > >>> Topic: __consumer_offsets Partition: 49 Leader: 1 Replicas: 1 Isr: 1 > > >>> Offline: > > >>> > > >>> kafka % cat reassignment.json > > >>> { > > >>> "version": 1, > > >>> "partitions": [ > > >>> { > > >>> "topic": "__consumer_offsets", > > >>> "partition": 49, > > >>> "replicas": [ 1, 0 ] > > >>> } > > >>> ] > > >>> } > > >>> > > >>> kafka % ./bin/kafka-reassign-partitions.sh --bootstrap-server > > >>> localhost:9092 --reassignment-json-file reassignment.json --execute > > >>> Current partition replica assignment > > >>> > > >>> > > >> > > > {"version":1,"partitions":[{"topic":"__consumer_offsets","partition":49,"replicas":[1],"log_dirs":["any"]}]} > > >>> > > >>> Save this to use as the --reassignment-json-file option during > rollback > > >>> Successfully started partition reassignment for __consumer_offsets-49 > > >>> kafka % ./bin/kafka-topics.sh --bootstrap-server localhost:9092 > --topic > > >>> __consumer_offsets --describe | grep 'Partition: 49' > > >>> Topic: __consumer_offsets Partition: 49 Leader: 1 Replicas: 1,0 Isr: > > 1,0 > > >>> Offline: > > >>> > > >>> > > >>> Andrew > > >>> > > >>> > > >>>> On Fri, Dec 16, 2022 at 9:46 AM Chris Peart <ch...@peart.me.uk> > > wrote: > > >>>> > > >>>> > > >>>> > > >>>> Hi, > > >>>> > > >>>> We have a kafka production cluster that was setup with the defaults > > for > > >>>> __consumer_offsets & __transaction_state topics. > > >>>> > > >>>> Is there a way to increase the replication factor from 1 to 3 using > > the > > >>>> kafka-reassign-partitions tool? > > >>>> > > >>>> We are also replacing our 4 brokers with new brokers, this has been > > >>>> completed so we now have an 8 broker cluster and have migrated all > the > > >>>> topics to the new brokers using the reassign tool except for the > > >>>> __consumer_offsets & __transaction_state topics. > > >>>> > > >>>> We stopped kafka on the old brokers today but all out consumers > failed > > >>>> due to the __consumer_offsets & __transaction_state topics residing > on > > >>>> the old brokers. > > >>>> > > >>>> I'm thinking we should could move the __consumer_offsets & > > >>>> __transaction_state topics to the now brokers using the reassign > > tool, i > > >>>> have done this on our dev platform and all is good. If you think > this > > is > > >>>> good idea we can then can stop kafka on the old nodes and then work > on > > >>>> the replication factoron the new nodes? > > >>>> > > >>>> The problem i have is how do we change the replication factor to 3 > > after > > >>>> we migrate the __consumer_offsets & __transaction_state topics? > > >>>> > > >>>> Thanks in advance. > > >>>> > > >>>> Chris > > >> > > >> > > > > >