...
Note that the tool only updates the zookeeper path and exits. The controller reassign the replicas for the partitions asynchronously.
This tool is only available in the 0.8 branch for now.
How to use the tool?
WARNING: The tool was released in beta in 0.8 and has some bugs that can render the topic unusable. The tool is known to be stable in 0.8.1.
Code Block |
---|
bin/kafka-reassign-partitions.sh Option Description ------ ----------- --broker-list <brokerlist> The list of brokers to which the bootstrap-server <String: Server(s) the server(s) to use for to use for bootstrapping> bootstrapping. REQUIRED if an absolution path of the log directory is specified for any replica in the reassignment json file --broker-list <String: brokerlist> The list of brokers to which the partitions need to be reassigned in the form "0,1,2". This is required if --topics-to-move-json-file is used to generate reassignment configuration --disable-rack-aware Disable rack aware replica assignment --execute Kick off the reassignment as specified by the --reassignment-json-file option. --generate Generate a candidate partition reassignment configuration. Note that this only generates a candidate assignment, it does not execute it. --reassignment-json-file <String: The JSON file with the partition manual assignment json file path> reassignment configurationThe format to use is - {"partitions": [{"topic": "foo", "partition": 1, "replicas": [1,2,3], "log_dirs": ["dir1","dir2","dir3"] }], "version":1 } Note that "log_dirs" is optional. When it is specified, its length must equal the length of the replicas list. The value in this list can be either "any" or the absolution path of the log directory on the broker. If absolute log directory path is specified, it is currently required that the replica has not already been created on that broker. The replica will then be created in the specified log directory on the broker later. --throttle <Long: throttle> The movement of partitions will be throttled to this value (bytes/sec). Rerunning with this option, whilst a rebalance is in progress, will alter partitions needthe tothrottle bevalue. reassignedThe inthrottle rate theshould form "0,1,2". This is requiredbe at least 1 KB/s. for automatic topic reassignment.(default: -1) --execute [execute] timeout <Long: timeout> The maximum time This option does the actualin ms allowed to wait for partition reassignment. By default, the tool execution to doesbe asuccessfully dry run --manual-assignment-json-file <manualinitiated The JSON file with the list of manual assignment json file path> reassignmentsThis option or topics- (default: 10000) --topics-to-move-json-file <String: Generate a reassignment configuration topics to reassign json file path> to move the partitions of the to-move-json-file needs to be specified topics to the list of specified. The format to use is - brokers specified by the --broker- {"partitions": list option. The format to [{"topic": "foo",use is - {"partitiontopics": 1, [{"replicastopic": [1,2,3] "foo"},{"topic": "foo1"}], "version":1 } --topics-to-move-json-file <topics toverify The JSON file with the list of topics reassign json file path> Verify if the to reassign.This option or manual- reassignment completed as specified by the assignment--jsonreassignment-file needs to be json-file option. If there specified.is Thea format to use is - throttle engaged for the {"topics":replicas [{"topic": "foo"},{"topic": "foo1"}], specified, and the rebalance has "version":1 completed, the throttle will be } --zookeeper <urls> removed --zookeeper <String: urls> REQUIRED: The connection string for the zookeeper connection in the form host:port. Multiple URLS can be given to allow fail-over. |
...
The partition reassignment tool can be used to expand an existing 0.8 Kafka cluster. Cluster expansion involves including brokers with new broker ids in a Kafka 08 cluster. Typically, when you add new brokers to a cluster, they will not receive any data from existing topics until this tool is run to assign existing topics/partitions to the new brokers. The tool allows 2 options to make it easier to move some topics in bulk to the new brokers. These 2 options are a) topics to move b) list of newly added brokers. Using these 2 options, the tool automatically figures out the placements of partitions for the topics on the new brokers and generates new JSON data which can be used in the next step (with the --reassignment-json-file
option) to execute the move.
The following example moves 2 topics (foo1, foo2) to newly added brokers in a cluster (5,6,7)
Code Block |
---|
nnarkhed$$ ./bin/kafka-reassign-partitions.sh --topics-to-move-json-file topics-to-move.json --broker-list "5,6,7" --executegenerate nnarkhed$$ cat topics-to-move.json {"topics": [{"topic": "foo1"},{"topic": "foo2"}], "version":1 } |
...
The following example moves 1 partition (foo-1) from replicas 1,2,3 to 1,2,4
Code Block |
---|
nnarkhed$$ ./bin/kafka-reassign-partitions.sh --manual-assignmentreassignment-json-file partitions-to-move.json --execute nnarkhed$$ cat partitions-to-move.json {"partitions": [{"topic": "foo", "partition": 1, "replicas": [1,2,4] }], }], "version":1 } |
5. StateChangeLogMerger Tool
...