-
Notifications
You must be signed in to change notification settings - Fork 0
/
kafka-reassign-tool.rb
381 lines (306 loc) · 10.6 KB
/
kafka-reassign-tool.rb
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
#!/usr/bin/ruby
require 'yaml'
require 'json'
require 'open3'
require 'tempfile'
require 'optparse'
DEFAULT_KAFKA_ROOT='/usr'
DEFAULT_KAFKA_CONFIG='/config/server.properties'
$kafka_root = nil
$zookeeper_url = nil
def run(command, args = [], opts = {})
opts = {
:raise_on_exitstatus => true,
:raise_on_err => true,
:input => ''
}.merge(opts)
out = nil
err = nil
exit_status = nil
Open3::popen3(command, *args) do |stdin, stdout, stderr, wait_thr|
stdin << opts[:input]
stdin.close
out = stdout.read
err = stderr.read
exit_status = wait_thr.value
end
if (opts[:raise_on_exitstatus] && exit_status.exitstatus != 0) || (opts[:raise_on_err] && !err.empty?) then
puts "#{command} failed. exitstatus=#{exit_status.exitstatus}, stderr:"
puts err
raise "#{command} failed"
end
{:out => out, :err => err, :exit_status => exit_status}
end
def get_kafka_root()
return $kafka_root if $kafka_root
DEFAULT_KAFKA_ROOT
end
def get_zk_url()
return $zookeeper_url if $zookeeper_url
config_file = get_kafka_root + DEFAULT_KAFKA_CONFIG
if File.file? config_file then
puts "Reading #{config_file}"
File.open(config_file, 'r') do |file|
file.each_line do |line|
if line =~ /^zookeeper.connect=(.*)/ then
$zookeeper_url = $1.strip
end
end
end
else
puts "Config file #{config_file} does not exist"
end
raise "No zookeeper URL given" unless $zookeeper_url
$zookeeper_url
end
def get_brokers()
result = run(get_kafka_root + '/bin/zookeeper-shell.sh', [get_zk_url], {:input => 'ls /brokers/ids'})
# Look for a line
# [1003, 1001, 1002]
#
brokers = nil
result[:out].lines.each {|line|
if line =~ /^ \[ ([^\]]+) \] /x then
brokers = $1.split(',').collect(&:strip).collect(&:to_i)
end
}
brokers
end
def get_topics()
result = run(get_kafka_root + '/bin/kafka-topics.sh', [
'--zookeeper', get_zk_url,
'--list'
])
result[:out].lines.collect(&:strip)
end
def topics_json(topics)
# {
# "topics": [
# {"topic": "topicname1"},
# {"topic": "topicname2"}
# ],
# "version":1
# }
data = {
:version => 1,
:topics => topics.collect{|t| { :topic => t }}
}
data.to_json
end
def get_current_assignments(topics, brokers)
topics_file = Tempfile.new(File.basename(__FILE__) + '-topics')
topics_file << topics_json(topics)
topics_file.close
result = run(get_kafka_root + '/bin/kafka-reassign-partitions.sh', [
'--zookeeper', get_zk_url,
'--topics-to-move-json-file', topics_file.path,
'--broker-list', brokers.join(','),
'--generate'
])
# Current partition replica assignment
#
# {"version":1,"partitions":[]}
# Proposed partition reassignment configuration
#
# {"version":1,"partitions":[]}
unless result[:out] =~ /\ACurrent partition replica assignment(.*)Proposed partition reassignment configuration(.*)\z/m
puts "kafka-reassign-partitions output:"
puts result[:out]
raise "Cannot parse assignment data"
end
JSON.parse($1)
end
# Returns a hash
# {
# 'topic_name' => {
# 'broker_id' => {
# :replica => 0, // for how many partitions this broker has a replica
# :leader => 0, // for how many partitions this broker is the leader
# },
# ...
# },
# ...
# }
#
def get_broker_stats(assignments)
stats = {}
assignments['partitions'].each {|item|
topic = item['topic']
replicas = item['replicas']
stats[topic] ||= {}
topic_stats = stats[topic]
leader = nil
replicas.each {|replica|
leader = replica if leader.nil?
topic_stats[replica] ||= {:replica => 0, :leader => 0}
topic_stats[replica][:replica] += 1
}
topic_stats[leader][:leader] += 1 if leader
}
stats
end
def set_replication(assignments, brokers, replication_factor)
stats = get_broker_stats(assignments)
# Make sure we have an entry for each broker even if given topic never uses it
stats.each {|topic, topic_data|
brokers.each {|broker|
topic_data[broker] ||= {:replica => 0, :leader => 0}
}
}
result = assignments.clone()
new_assignments = []
result['partitions'] = new_assignments
puts
puts "Initial stats:"
puts stats
puts
assignments['partitions'].each {|item|
topic = item['topic']
partition = item['partition']
replicas = item['replicas']
topic_stats = stats[topic]
# Calculate for how many partitions a broker should be a leader
# I calculate number of partition as a sum of how many partitions each broker leads
# this kinda sucks, it would be easier if get_broker_stats returned that info but
# that will require changing its result structure once again. So next time.
partition_count = topic_stats.values.collect {|data| data[:leader]}.reduce(0, :+)
leader_target = partition_count / brokers.size
# Remember what replicas we had before so we can see later if anything has changed
orig_replicas = replicas.clone()
# Remove replicas that are not in the given list of allowed brokers
old_leader = replicas[0]
(replicas - brokers).each {|broker|
replicas.delete(broker)
topic_stats[broker][:replica] -= 1
}
while replicas.size > replication_factor do
# Remove the most used (for this topic) broker
broker = topic_stats.select{|broker, data| replicas.include? broker} .max_by{|broker, data| data[:replica]}[0]
topic_stats[broker][:replica] -= 1
replicas.delete(broker)
end
while replicas.size < replication_factor do
# Use least used (for this topic) allowed broker as a new replica
broker = topic_stats.select{|broker, data| (brokers - replicas).include? broker} .min_by{|broker, data| data[:replica]}[0]
# Make a new broker ordinary replica, the code below will promote it to leader if needed
replicas.push broker
topic_stats[broker][:replica] += 1
end
# If removing brokers that were not allowed or adding/removing replicas changed the leader, update stats too
# Need to have up-to-date stats before using them to select a new leader
new_leader = replicas[0]
if new_leader != old_leader then
topic_stats[old_leader][:leader] -= 1 unless old_leader.nil?
topic_stats[new_leader][:leader] += 1 unless new_leader.nil?
end
old_leader = new_leader
# Select the best leader for this partition (based on which broker is a leader for fewest partitions)
candidate_leader_pos = nil
candidate_leader_leads = leader_target
replicas.each_with_index{|broker, pos|
# For how many partitions this broker is currently the leader
leader_for = topic_stats[broker][:leader]
if leader_for < candidate_leader_leads then
candidate_leader_pos = pos
candidate_leader_leads = leader_for
end
}
# Make our leader the first element in the list
replicas.unshift(replicas.delete_at(candidate_leader_pos)) if candidate_leader_pos
# Update leader stats again if leader has changed
new_leader = replicas[0]
if new_leader != old_leader then
topic_stats[old_leader][:leader] -= 1 unless old_leader.nil?
topic_stats[new_leader][:leader] += 1 unless new_leader.nil?
end
next if replicas == orig_replicas
puts " #{topic}-#{partition} : #{orig_replicas} => #{replicas}"
item = item.clone()
item['replicas'] = replicas
new_assignments << item
}
puts
puts "Final stats:"
puts stats
puts
result
end
topics = []
brokers = []
replication_factor = nil
optparse = OptionParser.new do |opts|
opts.on("--kafka-home <dir>",
"Root directory of the Kafka installation",
" (standard Kafka scripts must be under bin/ directory there)",
" Default: #{DEFAULT_KAFKA_ROOT}") { |v| $kafka_root = v }
opts.on("--zookeeper <url>",
"The connection string for the zookeeper connection",
" If not specified, and attempt to read it from Kafka config file is made") { |v| $zookeeper_url = v }
opts.on("--topic <topic1>",
"Can be specified multiple times to list one or more topic to apply operation to.",
" If option is not used, operation will apply to all existing topics") { |v| topics << v }
opts.on("--brokers <broker1,broker2,...>",
"Coma-separated list of brokers that partitions will be reassigned to",
" If option is not used, all brokers of the cluster will be used") { |v| brokers = v.split(',').collect(&:strip).collect(&:to_i) }
opts.on("--replication-factor <factor>",
"Target replication factor. Required") { |v| replication_factor = v.to_i }
end
begin
optparse.parse!
raise OptionParser::MissingArgument.new('replication-factor') if replication_factor.nil?
rescue OptionParser::InvalidOption, OptionParser::MissingArgument
puts $!.to_s
puts optparse
exit -1
end
zk_url = get_zk_url()
puts "Using zookeeper URL: #{zk_url}"
puts "Reading list of brokers..."
known_brokers = get_brokers()
if brokers.empty? then
brokers = known_brokers
else
brokers = brokers.sort.uniq
invalid = brokers - known_brokers
unless invalid.empty?
raise "Unknown brokers: #{invalid}"
end
end
puts "Reading list of topics..."
known_topics = get_topics()
if topics.empty? then
topics = known_topics
else
topics = topics.sort.uniq
invalid = topics - known_topics
unless invalid.empty?
raise "Unknown topics: #{invalid}"
end
end
puts "------------------------"
puts "Brokers:"
brokers.each {|broker| puts " #{broker}" }
puts "Topics:"
topics.each {|topic| puts " #{topic}" }
puts "------------------------"
if brokers.size < replication_factor then
raise "Cannot achieve replication factor of #{replication_factor} with #{brokers.size} broker(s)"
end
puts "Getting current assignments..."
assignments = get_current_assignments(topics, known_brokers)
puts "Building new assignments..."
new_assignments = set_replication(assignments, brokers, replication_factor)
if new_assignments['partitions'].empty? then
puts "No changes needed"
else
puts "Saving new assignments into new-assignments.json..."
File.open('new-assignments.json', 'w') { |file|
file.write(new_assignments.to_json)
}
puts "Done"
puts "To apply, run:"
puts " #{get_kafka_root + '/bin/kafka-reassign-partitions.sh'} --zookeeper #{zk_url} --reassignment-json-file new-assignments.json --execute --throttle XXXXXXXX"
puts "Then verify with:"
puts " #{get_kafka_root + '/bin/kafka-reassign-partitions.sh'} --zookeeper #{zk_url} --reassignment-json-file new-assignments.json --verify"
puts
end