-
Notifications
You must be signed in to change notification settings - Fork 54
/
settings.go
529 lines (479 loc) · 11.4 KB
/
settings.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
519
520
521
522
523
524
525
526
527
528
529
package config
import (
"fmt"
"reflect"
"strconv"
"strings"
"time"
"github.com/hashicorp/go-multierror"
"github.com/segmentio/kafka-go"
"github.com/segmentio/topicctl/pkg/admin"
log "github.com/sirupsen/logrus"
)
type configValidator func(v string) bool
var keyValidators = map[string]configValidator{
"cleanup.policy": func(v string) bool {
subValues := strings.Split(v, ",")
if len(subValues) > 2 {
return false
}
for _, subValue := range subValues {
if !inValues(subValue, "compact", "delete") {
return false
}
}
return true
},
"compression.type": func(v string) bool {
return inValues(v, "uncompressed", "zstd", "lz4", "snappy", "gzip", "producer")
},
"delete.retention.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"file.delete.delay.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"flush.messages": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"flush.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"follower.replication.throttled.replicas": func(v string) bool {
subValues := strings.Split(v, ",")
for _, subValue := range subValues {
elements := strings.Split(subValue, ":")
if len(elements) != 2 {
return false
}
if _, err := strconv.ParseInt(elements[0], 10, 64); err != nil {
return false
}
if _, err := strconv.ParseInt(elements[1], 10, 64); err != nil {
return false
}
}
return true
},
"index.interval.bytes": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"leader.replication.throttled.replicas": func(v string) bool {
subValues := strings.Split(v, ",")
for _, subValue := range subValues {
elements := strings.Split(subValue, ":")
if len(elements) != 2 {
return false
}
if _, err := strconv.ParseInt(elements[0], 10, 64); err != nil {
return false
}
if _, err := strconv.ParseInt(elements[1], 10, 64); err != nil {
return false
}
}
return true
},
"max.compaction.lag.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 1
},
"max.message.bytes": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"message.format.version": func(v string) bool {
return inValues(
v,
"0.8.0",
"0.8.1",
"0.8.2",
"0.9.0",
"0.10.0-IV0",
"0.10.0-IV1",
"0.10.1-IV0",
"0.10.1-IV1",
"0.10.1-IV2",
"0.10.2-IV0",
"0.11.0-IV0",
"0.11.0-IV1",
"0.11.0-IV2",
"1.0-IV0",
"1.1-IV0",
"2.0-IV0",
"2.0-IV1",
"2.1-IV0",
"2.1-IV1",
"2.1-IV2",
"2.2-IV0",
"2.2-IV1",
"2.3-IV0",
"2.3-IV1",
"2.4-IV0",
"2.4-IV1",
"2.5-IV0",
"2.6-IV0",
)
},
"message.timestamp.difference.max.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"message.timestamp.type": func(v string) bool {
return inValues(v, "CreateTime", "LogAppendTime")
},
"min.cleanable.dirty.ratio": func(v string) bool {
floatVal, err := strconv.ParseFloat(v, 64)
if err != nil {
return false
}
return floatVal >= 0 && floatVal <= 1.0
},
"min.compaction.lag.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"min.insync.replicas": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 1
},
"preallocate": func(v string) bool {
_, err := strconv.ParseBool(v)
return err == nil
},
"retention.bytes": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= -1
},
"retention.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= -1
},
"segment.bytes": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 14
},
"segment.index.bytes": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"segment.jitter.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= 0
},
"segment.ms": func(v string) bool {
intVal, err := strconv.ParseInt(v, 10, 64)
if err != nil {
return false
}
return intVal >= -1
},
"unclean.leader.election.enable": func(v string) bool {
_, err := strconv.ParseBool(v)
return err == nil
},
}
// TopicSettings is a map of key/value pairs that correspond to Kafka
// topic config settings.
type TopicSettings map[string]interface{}
// Validate determines whether the given settings are valid. See
// https://kafka.apache.org/documentation/#topicconfigs for details.
func (t TopicSettings) Validate() error {
var validateErr error
for key, value := range t {
validator, ok := keyValidators[key]
if !ok {
validateErr = multierror.Append(
validateErr,
fmt.Errorf("Key %s is not recognized topic config setting", key),
)
continue
}
valueStr, err := interfaceToString(value)
if err != nil {
validateErr = multierror.Append(
validateErr,
fmt.Errorf(
"Could not convert value for key %s to string: %+v",
key,
err,
),
)
continue
}
if valueStr == "" {
continue
}
valid := validator(valueStr)
if !valid {
validateErr = multierror.Append(
validateErr,
fmt.Errorf("Invalid value for key %s: %s", key, valueStr),
)
}
}
return validateErr
}
// ToConfigEntries converts the argument keys in the current settings into a slice of
// kafka-go config entries. If keys is nil, then all fields are converted.
func (t TopicSettings) ToConfigEntries(keys []string) ([]kafka.ConfigEntry, error) {
entries := []kafka.ConfigEntry{}
if keys == nil {
for key, value := range t {
strValue, err := interfaceToString(value)
if err != nil {
return nil, fmt.Errorf("Error converting value for key %s: %+v", key, err)
}
entries = append(
entries,
kafka.ConfigEntry{
ConfigName: key,
ConfigValue: strValue,
},
)
}
} else {
for _, key := range keys {
value, ok := t[key]
if !ok {
return nil, fmt.Errorf("Key %s not found", key)
}
strValue, err := interfaceToString(value)
if err != nil {
return nil, fmt.Errorf("Error converting value for key %s: %+v", key, err)
}
entries = append(
entries,
kafka.ConfigEntry{
ConfigName: key,
ConfigValue: strValue,
},
)
}
}
return entries, nil
}
// HasKey returns whether the current settings instance contains the argument key.
func (t TopicSettings) HasKey(key string) bool {
_, ok := t[key]
return ok
}
// GetValueStr returns the string value for a key in this settings instance. It
// returns an error if the key is not found.
func (t TopicSettings) GetValueStr(key string) (string, error) {
value, ok := t[key]
if !ok {
return "", fmt.Errorf("Key %s not found", key)
}
return interfaceToString(value)
}
// ConfigMapDiffs compares these topic settings to a string map fetched from
// the cluster. It returns the keys that are set in the settings but different in
// the cluster and also the keys that are set in the cluster but not set in
// the settings.
func (t TopicSettings) ConfigMapDiffs(
configMap map[string]string,
) ([]string, []string, error) {
diffKeys := []string{}
missingKeys := []string{}
for key, value := range t {
strValue, err := interfaceToString(value)
if err != nil {
return nil, nil, err
}
configStrValue := configMap[key]
if strValue != configStrValue {
diffKeys = append(diffKeys, key)
}
}
for configKey := range configMap {
_, ok := t[configKey]
if !ok {
missingKeys = append(missingKeys, configKey)
}
}
return diffKeys, missingKeys, nil
}
func (t TopicSettings) ReduceRetentionDrop(
configMap map[string]string,
retentionDropStepDuration time.Duration,
) (bool, error) {
if retentionDropStepDuration <= 0 {
return false, nil
}
currRetentionMsStr, ok := configMap[admin.RetentionKey]
if !ok || currRetentionMsStr == "" {
// No retention currently set
return false, nil
}
currRetentionMs, err := strconv.ParseInt(currRetentionMsStr, 10, 64)
if err != nil {
// Parse error
return false, err
}
var setRetentionMsIface interface{}
for key, value := range t {
if key == admin.RetentionKey {
setRetentionMsIface = value
break
}
}
if setRetentionMsIface == nil {
// Retention not configured in topic settings
return false, nil
}
setRetentionMs, err := interfaceToInt64(setRetentionMsIface)
if err != nil {
// Parse error
return false, err
}
maxDropMs := retentionDropStepDuration.Milliseconds()
if currRetentionMs-setRetentionMs > maxDropMs {
// Reduce drop
log.Debugf(
"Updating retention from %d to %d ms",
setRetentionMs,
currRetentionMs-maxDropMs,
)
t[admin.RetentionKey] = currRetentionMs - maxDropMs
return true, nil
}
return false, nil
}
// Copy returns a shallow copy of this settings instance.
func (t TopicSettings) Copy() TopicSettings {
copy := TopicSettings{}
for key, value := range t {
copy[key] = value
}
return copy
}
// FromConfigMap converts a string map from a Kafka topic to a TopicSettings instance.
func FromConfigMap(configMap map[string]string) TopicSettings {
t := TopicSettings{}
for key, value := range configMap {
t[key] = value
}
return t
}
func interfaceToString(v interface{}) (string, error) {
if v == nil {
return "", nil
}
switch t := v.(type) {
case bool:
return strconv.FormatBool(t), nil
case float32:
if t == float32(int64(t)) {
// Treat this value as an int
return strconv.FormatInt(int64(t), 10), nil
}
return strconv.FormatFloat(float64(t), 'f', 2, 32), nil
case float64:
if t == float64(int64(t)) {
// Treat this value as an int
return strconv.FormatInt(int64(t), 10), nil
}
return strconv.FormatFloat(t, 'f', 2, 64), nil
case int:
return strconv.FormatInt(int64(t), 10), nil
case int64:
return strconv.FormatInt(t, 10), nil
case string:
return t, nil
case []string:
return strings.Join(t, ","), nil
case []interface{}:
strValues := []string{}
for _, item := range t {
itemStr, err := interfaceToString(item)
if err != nil {
return "", err
}
strValues = append(strValues, itemStr)
}
return strings.Join(strValues, ","), nil
}
return "", fmt.Errorf("Invalid setting value: %+v (%s)", v, reflect.TypeOf(v))
}
func interfaceToInt64(v interface{}) (int64, error) {
if v == nil {
return 0, nil
}
switch t := v.(type) {
case float32:
return int64(t), nil
case float64:
return int64(t), nil
case int:
return int64(t), nil
case int64:
return t, nil
case string:
return strconv.ParseInt(t, 10, 64)
default:
return 0, fmt.Errorf(
"Could not convert value %+v (type %+v) to int64",
v,
reflect.TypeOf(v),
)
}
}
func inValues(v string, values ...string) bool {
valuesMap := map[string]struct{}{}
for _, value := range values {
valuesMap[value] = struct{}{}
}
_, ok := valuesMap[v]
return ok
}