/
throttles.go
518 lines (446 loc) · 15.1 KB
/
throttles.go
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
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
package main
import (
"bytes"
"errors"
"fmt"
"log"
"math"
"strconv"
"time"
"github.com/DataDog/kafka-kit/kafkametrics"
"github.com/DataDog/kafka-kit/kafkazk"
)
// ReplicationThrottleMeta holds all types
// needed to call the updateReplicationThrottle func.
type ReplicationThrottleMeta struct {
topics []string
reassignments kafkazk.Reassignments
zk kafkazk.Handler
km kafkametrics.Handler
override string
events *EventGenerator
// Map of broker ID to last set throttle rate.
throttles map[int]float64
limits Limits
failureThreshold int
failures int
}
// Failure increments the failures count
// and returns true if the count exceeds
// the failures threshold.
func (r *ReplicationThrottleMeta) Failure() bool {
r.failures++
if r.failures > r.failureThreshold {
return true
}
return false
}
// ResetFailures resets the failures count.
func (r *ReplicationThrottleMeta) ResetFailures() {
r.failures = 0
}
// ReassigningBrokers is a list of brokers
// with a throttle applied for an ongoing
// reassignment.
type ReassigningBrokers struct {
Src []*kafkametrics.Broker
Dst []*kafkametrics.Broker
}
// highestSrcNetTX takes a ReassigningBrokers and returns
// the leader with the highest outbound network throughput.
func (t ReassigningBrokers) highestSrcNetTX() *kafkametrics.Broker {
hwm := 0.00
var broker *kafkametrics.Broker
for _, b := range t.Src {
if b.NetTX > hwm {
hwm = b.NetTX
broker = b
}
}
return broker
}
// updateReplicationThrottle takes a ReplicationThrottleMeta
// that holds topics being replicated, any clients, throttle override params,
// and other required metadata.
// Metrics for brokers participating in any ongoing replication
// are fetched to determine replication headroom.
// The replication throttle is then adjusted accordingly.
// If a non-empty override is provided, that static value is used instead
// of a dynamically determined value.
func updateReplicationThrottle(params *ReplicationThrottleMeta) error {
// Get the maps of brokers handling
// reassignments.
bmaps, err := mapsFromReassigments(params.reassignments, params.zk)
if err != nil {
return err
}
// Creates lists from maps.
srcBrokers, dstBrokers, allBrokers := bmaps.lists()
log.Printf("Source brokers participating in replication: %v\n", srcBrokers)
log.Printf("Destination brokers participating in replication: %v\n", dstBrokers)
/************************
Determine throttle rates.
************************/
// Use the throttle override if set.
// Otherwise, make a calculation
// using broker metrics and known
// capacity values.
var replicationCapacity float64
var currThrottle float64
var useMetrics bool
var brokerMetrics kafkametrics.BrokerMetrics
var metricErrs []error
var inFailureMode bool
if params.override != "" {
log.Printf("A throttle override is set: %sMB/s\n", params.override)
o, _ := strconv.Atoi(params.override)
replicationCapacity = float64(o)
} else {
useMetrics = true
// Get broker metrics.
brokerMetrics, metricErrs = params.km.GetMetrics()
// Even if errors are returned, we can still
// proceed as long as we have complete metrics
// data for all target brokers. If we have broker
// metrics for all target brokers, we can ignore
// any errors.
if metricErrs != nil {
if brokerMetrics == nil || incompleteBrokerMetrics(allBrokers, brokerMetrics) {
inFailureMode = true
}
}
// If we cannot proceed normally due to missing/partial
// metrics data, check what failure iteration we're in.
// If we're above the threshold, revert to the minimum
// rate, otherwise retain the previous rate.
if inFailureMode {
log.Printf("Errors fetching metrics: %s\n", metricErrs)
// Check our failures against the
// configured threshold.
over := params.Failure()
// Over threshold. Set replicationCapacity which will be
// applied in the apply throttles stage.
if over {
log.Printf("Metrics fetch failure count %d exceeds threshold %d, reverting to min-rate %.2fMB/s\n",
params.failures, params.failureThreshold, params.limits["minimum"])
replicationCapacity = params.limits["minimum"]
// Not over threshold. Return and retain previous throttle.
} else {
log.Printf("Metrics fetch failure count %d doesn't exceed threshold %d, retaining previous throttle\n",
params.failures, params.failureThreshold)
return nil
}
} else {
// Reset the failure counter
// in case it was incremented
// in previous iterations.
params.ResetFailures()
}
}
// If we're using metrics and successfully
// fetched them, determine a tvalue based on
// the most-utilized path.
if useMetrics && !inFailureMode {
var e string
replicationCapacity, currThrottle, e, err = repCapacityByMetrics(params, bmaps, brokerMetrics)
if err != nil {
return err
}
log.Println(e)
log.Printf("Replication capacity (based on a %.0f%% max free capacity utilization): %0.2fMB/s\n",
params.limits["maximum"], replicationCapacity)
// Check if the delta between the newly calculated
// throttle and the previous throttle exceeds the
// ChangeThreshold param.
d := math.Abs((currThrottle - replicationCapacity) / currThrottle * 100)
if d < Config.ChangeThreshold {
log.Printf("Proposed throttle is within %.2f%% of the previous throttle "+
"(below %.2f%% threshold), skipping throttle update\n",
d, Config.ChangeThreshold)
return nil
}
}
// Get a rate string based on the final tvalue.
rateString := fmt.Sprintf("%.0f", replicationCapacity*1000000.00)
/**************************
Set topic throttle configs.
**************************/
errs := applyTopicThrottles(bmaps.throttled, params.zk)
for _, e := range errs {
log.Println(e)
}
/***************************
Set broker throttle configs.
***************************/
errs = applyBrokerThrottles(bmaps.all,
rateString,
replicationCapacity,
params.throttles,
params.zk)
for _, e := range errs {
log.Println(e)
}
/***********
Log success.
***********/
// Write event.
var b bytes.Buffer
b.WriteString(fmt.Sprintf("Replication throttle of %0.2fMB/s set on the following brokers: %v\n",
replicationCapacity, allBrokers))
b.WriteString(fmt.Sprintf("Topics currently undergoing replication: %v", params.topics))
params.events.Write("Broker replication throttle set", b.String())
return nil
}
// mapsFromReassigments takes a kafakzk.Reassignments and returns
// a bmapBundle, which includes a broker list for source, destination,
// and all brokers handling any ongoing reassignments. Additionally, a map
// of throttled replicas by topic is included.
func mapsFromReassigments(r kafkazk.Reassignments, zk kafkazk.Handler) (bmapBundle, error) {
lb := bmapBundle{
// Maps of src and dst brokers
// used as sets.
src: map[int]struct{}{},
dst: map[int]struct{}{},
all: map[int]struct{}{},
// A map for each topic with a list throttled
// leaders and followers. This is used to write
// the topic config throttled brokers lists. E.g.:
// map[topic]map[leaders]["0:1001", "1:1002"]
// map[topic]map[followers]["2:1003", "3:1004"]
throttled: map[string]map[string][]string{},
}
// Get topic data for each topic
// undergoing a reassignment.
for t := range r {
lb.throttled[t] = make(map[string][]string)
lb.throttled[t]["leaders"] = []string{}
lb.throttled[t]["followers"] = []string{}
tstate, err := zk.GetTopicStateISR(t)
if err != nil {
return lb, fmt.Errorf("Error fetching topic data: %s\n", err.Error())
}
// For each partition, compare the current
// ISR leader to the brokers being assigned
// in the reassignments. The current leaders
// will be sources, new brokers in the assignment
// list will be destinations.
for p := range tstate {
part, _ := strconv.Atoi(p)
if reassigning, exists := r[t][part]; exists {
// Source brokers.
leader := tstate[p].Leader
lb.src[leader] = struct{}{}
// Append to the throttle list.
lb.throttled[t]["leaders"] = append(lb.throttled[t]["leaders"], fmt.Sprintf("%d:%d", part, leader))
// Dest brokers.
for _, b := range reassigning {
if b != leader {
lb.dst[b] = struct{}{}
lb.throttled[t]["followers"] = append(lb.throttled[t]["followers"], fmt.Sprintf("%d:%d", part, b))
}
}
}
}
}
lb.all = mergeMaps(lb.src, lb.dst)
return lb, nil
}
// repCapacityByMetrics finds the most constrained src broker and returns
// a calculated replication capacity, the currently applied throttle, a slice
// of event strings and any errors if encountered.
func repCapacityByMetrics(rtm *ReplicationThrottleMeta, bmb bmapBundle, bm kafkametrics.BrokerMetrics) (float64, float64, string, error) {
// Map src/dst broker IDs to a *ReassigningBrokers.
participatingBrokers := &ReassigningBrokers{}
var event string
// Source brokers.
for b := range bmb.src {
if broker, exists := bm[b]; exists {
participatingBrokers.Src = append(participatingBrokers.Src, broker)
} else {
return 0.00, 0.00, event, fmt.Errorf("Broker %d not found in broker metrics", b)
}
}
// Destination brokers.
for b := range bmb.dst {
if broker, exists := bm[b]; exists {
participatingBrokers.Dst = append(participatingBrokers.Dst, broker)
} else {
return 0.00, 0.00, event, fmt.Errorf("Broker %d not found in broker metrics", b)
}
}
// Get the most constrained src broker and
// its current throttle, if applied.
constrainingSrc := participatingBrokers.highestSrcNetTX()
currThrottle, exists := rtm.throttles[constrainingSrc.ID]
if !exists {
currThrottle = 0.00
}
replicationCapacity, err := rtm.limits.headroom(constrainingSrc, currThrottle)
if err != nil {
return 0.00, 0.00, event, err
}
event = fmt.Sprintf("Most utilized source broker: "+
"[%d] net tx of %.2fMB/s (over %ds) with an existing throttle rate of %.2fMB/s",
constrainingSrc.ID, constrainingSrc.NetTX, Config.MetricsWindow, currThrottle)
return replicationCapacity, currThrottle, event, nil
}
// applyTopicThrottles updates the throttled brokers list for
// all topics undergoing replication.
// XXX we need to avoid continously resetting this to reduce writes
// to ZK and subsequent config propagations to all brokers.
// We can either:
// - Ensure throttle lists are sorted so that if we provide the
// same list each iteration that it results in a no-op in the backend.
// - Keep track of topics that have already had a throttle list
// written and assume that it's not going to change
// (a throttle list is applied) when a topic is initially set
// for reassignment and cleared by autothrottle as soon as
// the reassignment is done).
func applyTopicThrottles(throttled map[string]map[string][]string, zk kafkazk.Handler) []string {
var errs []string
for t := range throttled {
// Generate config.
config := kafkazk.KafkaConfig{
Type: "topic",
Name: t,
Configs: [][2]string{},
}
leaderList := sliceToString(throttled[t]["leaders"])
if leaderList != "" {
c := [2]string{"leader.replication.throttled.replicas", leaderList}
config.Configs = append(config.Configs, c)
}
followerList := sliceToString(throttled[t]["followers"])
if followerList != "" {
c := [2]string{"follower.replication.throttled.replicas", followerList}
config.Configs = append(config.Configs, c)
}
// Write the config.
_, err := zk.UpdateKafkaConfig(config)
if err != nil {
errs = append(errs, fmt.Sprintf("Error setting throttle list on topic %s: %s\n", t, err))
}
}
return errs
}
// applyBrokerThrottles take a list of brokers, a replication throttle rate string,
// rate, map of applied throttles, and zk kafkazk.Handler zookeeper client.
// For each broker, the throttle rate is applied and if successful, the rate
// is stored in the throttles map for future reference.
func applyBrokerThrottles(bs map[int]struct{}, ratestr string, r float64, ts map[int]float64, zk kafkazk.Handler) []string {
var errs []string
// Generate a broker throttle config.
for b := range bs {
config := kafkazk.KafkaConfig{
Type: "broker",
Name: strconv.Itoa(b),
Configs: [][2]string{
[2]string{"leader.replication.throttled.rate", ratestr},
[2]string{"follower.replication.throttled.rate", ratestr},
},
}
// Write the throttle config.
changed, err := zk.UpdateKafkaConfig(config)
if err != nil {
errs = append(errs, fmt.Sprintf("Error setting throttle on broker %d: %s\n", b, err))
}
if changed {
// Store the configured rate.
ts[b] = r
log.Printf("Updated throttle to %0.2fMB/s on broker %d\n", r, b)
}
// Hard coded sleep to reduce
// ZK load.
time.Sleep(250 * time.Millisecond)
}
return errs
}
// removeAllThrottles removes all topic and
// broker throttle configs.
func removeAllThrottles(zk kafkazk.Handler, params *ReplicationThrottleMeta) error {
/****************************
Clear topic throttle configs.
****************************/
// Get all topics.
topics, err := zk.GetTopics(topicsRegex)
if err != nil {
return err
}
for _, topic := range topics {
config := kafkazk.KafkaConfig{
Type: "topic",
Name: topic,
Configs: [][2]string{
[2]string{"leader.replication.throttled.replicas", ""},
[2]string{"follower.replication.throttled.replicas", ""},
},
}
// Update the config.
_, err := zk.UpdateKafkaConfig(config)
if err != nil {
log.Printf("Error removing throttle config on topic %s: %s\n", topic, err)
}
// Hardcoded sleep to reduce
// ZK load.
time.Sleep(250 * time.Millisecond)
}
/**********************
Clear broker throttles.
**********************/
// Fetch brokers.
brokers, errs := zk.GetAllBrokerMeta(false)
if errs != nil {
return errs[0]
}
var unthrottledBrokers []int
// Unset throttles.
for b := range brokers {
config := kafkazk.KafkaConfig{
Type: "broker",
Name: strconv.Itoa(b),
Configs: [][2]string{
[2]string{"leader.replication.throttled.rate", ""},
[2]string{"follower.replication.throttled.rate", ""},
},
}
changed, err := zk.UpdateKafkaConfig(config)
switch err.(type) {
case nil:
case kafkazk.ErrNoNode:
// We'd get an ErrNoNode here only if
// the parent path for dynamic broker
// configs (/config/brokers) if it doesn't
// exist, which can happen in new clusters
// that have never had dynamic configs applied.
// Rather than creating that znode, we'll just
// ignore errors here; if the znodes don't exist,
// there's not even config to remove.
default:
log.Printf("Error removing throttle on broker %d: %s\n", b, err)
}
if changed {
unthrottledBrokers = append(unthrottledBrokers, b)
log.Printf("Throttle removed on broker %d\n", b)
}
// Hardcoded sleep to reduce
// ZK load.
time.Sleep(250 * time.Millisecond)
}
// Write event.
if len(unthrottledBrokers) > 0 {
m := fmt.Sprintf("Replication throttle removed on the following brokers: %v",
unthrottledBrokers)
params.events.Write("Broker replication throttle removed", m)
}
// Lazily check if any
// errors were encountered,
// return a generic error.
if err != nil {
return errors.New("one or more throttles were not cleared")
}
// Unset all stored throttle rates.
for b := range params.throttles {
params.throttles[b] = 0.0
}
return nil
}