-
Notifications
You must be signed in to change notification settings - Fork 350
/
Copy pathLoadRetriever.java
549 lines (498 loc) · 23.5 KB
/
LoadRetriever.java
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
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
/*
* Copyright 2024, AutoMQ HK Limited.
*
* The use of this file is governed by the Business Source License,
* as detailed in the file "/LICENSE.S3Stream" included in this repository.
*
* As of the Change Date specified in that file, in accordance with
* the Business Source License, use of this software will be governed
* by the Apache License, Version 2.0
*/
package kafka.autobalancer;
import kafka.autobalancer.common.AutoBalancerThreadFactory;
import kafka.autobalancer.common.Utils;
import kafka.autobalancer.common.types.MetricTypes;
import kafka.autobalancer.common.types.MetricVersion;
import kafka.autobalancer.config.AutoBalancerControllerConfig;
import kafka.autobalancer.config.StaticAutoBalancerConfig;
import kafka.autobalancer.config.StaticAutoBalancerConfigUtils;
import kafka.autobalancer.listeners.BrokerStatusListener;
import kafka.autobalancer.listeners.LeaderChangeListener;
import kafka.autobalancer.metricsreporter.metric.AutoBalancerMetrics;
import kafka.autobalancer.metricsreporter.metric.BrokerMetrics;
import kafka.autobalancer.metricsreporter.metric.MetricSerde;
import kafka.autobalancer.metricsreporter.metric.TopicPartitionMetrics;
import kafka.autobalancer.model.BrokerUpdater;
import kafka.autobalancer.model.ClusterModel;
import kafka.autobalancer.services.AbstractResumableService;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.internals.Topic;
import org.apache.kafka.common.message.CreatePartitionsRequestData;
import org.apache.kafka.common.message.CreatePartitionsResponseData;
import org.apache.kafka.common.message.CreateTopicsRequestData;
import org.apache.kafka.common.message.CreateTopicsResponseData;
import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.controller.Controller;
import org.apache.kafka.controller.ControllerRequestContext;
import org.apache.kafka.server.config.QuotaConfigs;
import com.automq.stream.utils.LogContext;
import java.time.Duration;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Properties;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
public class LoadRetriever extends AbstractResumableService implements BrokerStatusListener, LeaderChangeListener {
public static final Random RANDOM = new Random();
private final Map<Integer, BrokerEndpoints> bootstrapServerMap;
private volatile int metricReporterTopicPartition;
private final long metricReporterTopicRetentionTime;
protected volatile long consumerPollTimeout;
protected final String consumerClientIdPrefix;
protected final long consumerRetryBackOffMs;
protected final ClusterModel clusterModel;
private final Lock lock;
private final Condition cond;
private final Controller controller;
private final ScheduledExecutorService mainExecutorService;
private final Map<Integer, BrokerEndpoints> bootstrapServerMapInUse;
private final Set<TopicPartition> currentAssignment = new HashSet<>();
private final StaticAutoBalancerConfig staticConfig;
private final String listenerName;
private volatile boolean leaderEpochInitialized;
private volatile boolean isLeader;
private volatile Consumer<String, AutoBalancerMetrics> consumer;
public LoadRetriever(AutoBalancerControllerConfig config, Controller controller, ClusterModel clusterModel) {
this(config, controller, clusterModel, null);
}
public LoadRetriever(AutoBalancerControllerConfig config, Controller controller, ClusterModel clusterModel, LogContext logContext) {
super(logContext);
this.controller = controller;
this.clusterModel = clusterModel;
this.bootstrapServerMap = new HashMap<>();
this.bootstrapServerMapInUse = new HashMap<>();
this.lock = new ReentrantLock();
this.cond = lock.newCondition();
this.mainExecutorService = Executors.newSingleThreadScheduledExecutor(new AutoBalancerThreadFactory("load-retriever-main"));
leaderEpochInitialized = false;
staticConfig = new StaticAutoBalancerConfig(config.originals(), false);
listenerName = staticConfig.getString(StaticAutoBalancerConfig.AUTO_BALANCER_CLIENT_LISTENER_NAME_CONFIG);
metricReporterTopicPartition = config.getInt(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_METRICS_TOPIC_NUM_PARTITIONS_CONFIG);
metricReporterTopicRetentionTime = config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_METRICS_TOPIC_RETENTION_MS_CONFIG);
consumerPollTimeout = config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_CONSUMER_POLL_TIMEOUT);
consumerClientIdPrefix = QuotaConfigs.INTERNAL_CLIENT_ID_PREFIX + config.getString(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_CONSUMER_CLIENT_ID_PREFIX);
consumerRetryBackOffMs = config.getLong(AutoBalancerControllerConfig.AUTO_BALANCER_CONTROLLER_CONSUMER_RETRY_BACKOFF_MS);
}
@Override
protected void doRun() {
// seek to the latest offset if consumer exists
if (this.consumer != null) {
this.consumer.seekToEnd(Collections.emptyList());
}
scheduleRetrieve(this.epoch.get());
}
@Override
protected void doShutdown() {
this.mainExecutorService.shutdown();
try {
if (!mainExecutorService.awaitTermination(5, TimeUnit.SECONDS)) {
this.mainExecutorService.shutdownNow();
}
} catch (InterruptedException ignored) {
}
shutdownConsumer();
}
@Override
protected void doPause() {
}
private void scheduleRetrieve(int epoch) {
this.mainExecutorService.schedule(() -> retrieve(epoch), 0, TimeUnit.MILLISECONDS);
}
protected KafkaConsumer<String, AutoBalancerMetrics> createConsumer(String bootstrapServer) {
return new KafkaConsumer<>(buildConsumerProps(bootstrapServer));
}
protected Properties buildConsumerProps(String bootstrapServer) {
Properties consumerProps = new Properties();
long randomToken = RANDOM.nextLong();
consumerProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, consumerClientIdPrefix + randomToken);
consumerProps.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, Long.toString(consumerRetryBackOffMs));
consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest");
consumerProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, MetricSerde.class.getName());
StaticAutoBalancerConfigUtils.addSslConfigs(consumerProps, this.staticConfig);
return consumerProps;
}
public static class BrokerEndpoints {
private final int brokerId;
private Set<String> endpoints = new HashSet<>();
private boolean isFenced;
private boolean isOutdated = false;
public BrokerEndpoints(int brokerId) {
this.brokerId = brokerId;
}
public int brokerId() {
return this.brokerId;
}
public Set<String> getEndpoints() {
return this.endpoints;
}
public void setEndpoints(Set<String> endpoints) {
this.endpoints = new HashSet<>(endpoints);
}
public BrokerEndpoints setFenced(boolean isFenced) {
this.isFenced = isFenced;
return this;
}
public boolean isFenced() {
return this.isFenced;
}
public boolean isValid() {
return !this.isFenced && !this.endpoints.isEmpty();
}
public boolean isOutdated() {
return isOutdated;
}
public void setOutdated(boolean outdated) {
isOutdated = outdated;
}
}
@Override
public void onBrokerRegister(RegisterBrokerRecord record) {
lock.lock();
try {
Set<String> endpoints = new HashSet<>();
for (RegisterBrokerRecord.BrokerEndpoint endpoint : record.endPoints()) {
if ("CONTROLLER".equals(endpoint.name())) {
continue;
}
if (listenerName == null || listenerName.isEmpty() || Utils.checkListenerName(endpoint.name(), listenerName)) {
String url = endpoint.host() + ":" + endpoint.port();
endpoints.add(url);
}
}
if (endpoints.isEmpty()) {
logger.warn("No valid endpoint found for broker {} of name {}", record.brokerId(), listenerName);
}
BrokerEndpoints brokerEndpoints = new BrokerEndpoints(record.brokerId());
brokerEndpoints.setFenced(Utils.isBrokerFenced(record));
brokerEndpoints.setEndpoints(endpoints);
brokerEndpoints.setOutdated(false);
this.bootstrapServerMap.put(record.brokerId(), brokerEndpoints);
this.bootstrapServerMapInUse.computeIfPresent(record.brokerId(), (k, v) -> {
v.setOutdated(!v.getEndpoints().equals(endpoints));
v.setFenced(Utils.isBrokerFenced(record));
return v;
});
cond.signal();
} finally {
lock.unlock();
}
}
@Override
public void onBrokerUnregister(UnregisterBrokerRecord record) {
lock.lock();
try {
this.bootstrapServerMap.remove(record.brokerId());
} finally {
lock.unlock();
}
}
@Override
public void onBrokerRegistrationChanged(BrokerRegistrationChangeRecord record) {
Optional<Boolean> isBrokerFenced = Utils.isBrokerFenced(record);
isBrokerFenced.ifPresent(isFenced -> {
lock.lock();
try {
this.bootstrapServerMap.computeIfPresent(record.brokerId(), (k, v) -> {
v.setFenced(isFenced);
return v;
});
this.bootstrapServerMapInUse.computeIfPresent(record.brokerId(), (k, v) -> {
v.setFenced(isFenced);
return v;
});
cond.signal();
} finally {
lock.unlock();
}
});
}
boolean hasAvailableBrokerInUse() {
if (bootstrapServerMapInUse.isEmpty()) {
return false;
}
for (Map.Entry<Integer, BrokerEndpoints> entry : bootstrapServerMapInUse.entrySet()) {
int brokerId = entry.getKey();
BrokerEndpoints endpoints = entry.getValue();
if (bootstrapServerMap.containsKey(brokerId) && endpoints != null && endpoints.isValid() && !endpoints.isOutdated()) {
return true;
}
}
return false;
}
boolean hasAvailableBroker() {
if (this.bootstrapServerMap.isEmpty()) {
return false;
}
for (BrokerEndpoints brokerEndpoints : this.bootstrapServerMap.values()) {
if (brokerEndpoints.isValid()) {
return true;
}
}
return false;
}
public String buildBootstrapServer() {
Set<String> endpoints = new HashSet<>();
this.bootstrapServerMapInUse.clear();
for (BrokerEndpoints brokerEndpoints : this.bootstrapServerMap.values()) {
if (brokerEndpoints.isValid() && !brokerEndpoints.getEndpoints().isEmpty()) {
endpoints.add(brokerEndpoints.getEndpoints().iterator().next());
this.bootstrapServerMapInUse.put(brokerEndpoints.brokerId(), brokerEndpoints);
}
}
return String.join(",", endpoints);
}
void checkAndCreateConsumer(int epoch) {
this.lock.lock();
try {
if (!isRunnable(epoch)) {
return;
}
if (!hasAvailableBrokerInUse()) {
logger.info("No available broker in use, try to close current consumer");
shutdownConsumer();
while (isRunnable(epoch) && !hasAvailableBroker()) {
try {
this.cond.await();
} catch (InterruptedException ignored) {
}
}
if (!isRunnable(epoch)) {
return;
}
}
if (this.consumer == null) {
//TODO: fetch metadata from controller
String bootstrapServer = buildBootstrapServer();
this.consumer = createConsumer(bootstrapServer);
logger.info("Created consumer on {}", bootstrapServer);
}
} finally {
lock.unlock();
}
}
private void shutdownConsumer() {
this.lock.lock();
try {
if (this.consumer != null) {
try {
this.consumer.close(Duration.ofSeconds(5));
} catch (Exception e) {
logger.error("Exception when close consumer: {}", e.getMessage());
}
this.consumer = null;
this.currentAssignment.clear();
logger.info("Consumer closed");
}
} finally {
lock.unlock();
}
}
private void createTopic() {
if (!isValidState()) {
return;
}
CreateTopicsRequestData request = new CreateTopicsRequestData();
CreateTopicsRequestData.CreatableTopicCollection topicCollection = new CreateTopicsRequestData.CreatableTopicCollection();
CreateTopicsRequestData.CreatableTopic creatableTopic = new CreateTopicsRequestData.CreatableTopic();
creatableTopic.configs().add(new CreateTopicsRequestData.CreatableTopicConfig()
.setName(TopicConfig.RETENTION_MS_CONFIG)
.setValue(Long.toString(metricReporterTopicRetentionTime)));
creatableTopic.configs().add(new CreateTopicsRequestData.CreatableTopicConfig()
.setName(TopicConfig.CLEANUP_POLICY_CONFIG)
.setValue(TopicConfig.CLEANUP_POLICY_DELETE));
topicCollection.add(new CreateTopicsRequestData.CreatableTopic()
.setName(Topic.AUTO_BALANCER_METRICS_TOPIC_NAME)
.setNumPartitions(metricReporterTopicPartition)
.setReplicationFactor((short) 1)
.setConfigs(creatableTopic.configs()));
request.setTopics(topicCollection);
try {
CompletableFuture<CreateTopicsResponseData> future = this.controller.createTopics(
new ControllerRequestContext(null, null, OptionalLong.empty()),
request,
Collections.emptySet());
CreateTopicsResponseData rsp = future.get();
CreateTopicsResponseData.CreatableTopicResult result = rsp.topics().find(Topic.AUTO_BALANCER_METRICS_TOPIC_NAME);
if (result.errorCode() == Errors.NONE.code()) {
logger.info("Create metrics reporter topic {} succeed", Topic.AUTO_BALANCER_METRICS_TOPIC_NAME);
} else if (result.errorCode() != Errors.NONE.code() && result.errorCode() != Errors.TOPIC_ALREADY_EXISTS.code()) {
logger.warn("Create metrics reporter topic {} failed: {}", Topic.AUTO_BALANCER_METRICS_TOPIC_NAME, result.errorMessage());
}
} catch (Exception e) {
logger.error("Create metrics reporter topic {} exception", Topic.AUTO_BALANCER_METRICS_TOPIC_NAME, e);
}
}
private void createTopicPartitions() {
if (!isValidState()) {
return;
}
if (currentAssignment.size() > metricReporterTopicPartition) {
logger.info("Current partition number {} exceeds expected {}, skip alter topic partitions",
currentAssignment.size(), metricReporterTopicPartition);
return;
}
CreatePartitionsRequestData.CreatePartitionsTopic topic = new CreatePartitionsRequestData.CreatePartitionsTopic()
.setName(Topic.AUTO_BALANCER_METRICS_TOPIC_NAME)
.setCount(metricReporterTopicPartition)
.setAssignments(null);
try {
CompletableFuture<List<CreatePartitionsResponseData.CreatePartitionsTopicResult>> future =
this.controller.createPartitions(new ControllerRequestContext(null, null, OptionalLong.empty()),
List.of(topic), false);
List<CreatePartitionsResponseData.CreatePartitionsTopicResult> result = future.get();
for (CreatePartitionsResponseData.CreatePartitionsTopicResult r : result) {
if (r.errorCode() == Errors.NONE.code()) {
logger.info("Create metrics reporter topic {} with {} partitions succeed", Topic.AUTO_BALANCER_METRICS_TOPIC_NAME, metricReporterTopicPartition);
} else {
logger.warn("Create metrics reporter topic {} with {} partitions failed: {}", Topic.AUTO_BALANCER_METRICS_TOPIC_NAME,
metricReporterTopicPartition, r.errorMessage());
}
}
} catch (Exception e) {
logger.error("Create metrics reporter topic {} with {} partitions exception", Topic.AUTO_BALANCER_METRICS_TOPIC_NAME,
metricReporterTopicPartition, e);
}
}
private boolean isValidState() {
if (!leaderEpochInitialized || !isLeader) {
return false;
}
return hasAvailableBroker();
}
public void retrieve(int epoch) {
while (isRunnable(epoch)) {
try {
checkAndCreateConsumer(epoch);
if (!isRunnable(epoch)) {
return;
}
TopicAction action = refreshAssignment();
if (action != TopicAction.NONE) {
if (action == TopicAction.CREATE) {
createTopic();
} else {
createTopicPartitions();
}
shutdownConsumer();
this.mainExecutorService.schedule(() -> retrieve(epoch), 1, TimeUnit.SECONDS);
return;
}
ConsumerRecords<String, AutoBalancerMetrics> records = this.consumer.poll(Duration.ofMillis(consumerPollTimeout));
for (ConsumerRecord<String, AutoBalancerMetrics> record : records) {
if (record == null) {
// This means we cannot parse the metrics. It might happen when a newer type of metrics has been added and
// the current code is still old. We simply ignore that metric in this case.
logger.warn("Cannot parse record, maybe controller version is outdated.");
continue;
}
updateClusterModel(record.value());
}
if (logger.isDebugEnabled()) {
logger.debug("Finished consuming {} metrics from {}.", records.count(), Topic.AUTO_BALANCER_METRICS_TOPIC_NAME);
}
} catch (InvalidTopicException e) {
createTopic();
this.mainExecutorService.schedule(() -> retrieve(epoch), 1, TimeUnit.SECONDS);
return;
} catch (Exception e) {
logger.error("Consumer poll error", e);
this.mainExecutorService.schedule(() -> retrieve(epoch), 1, TimeUnit.SECONDS);
return;
} catch (Throwable t) {
logger.error("Consumer poll error and exit retrieve loop", t);
return;
}
}
}
private TopicAction refreshAssignment() {
List<PartitionInfo> partitionInfos = this.consumer.partitionsFor(Topic.AUTO_BALANCER_METRICS_TOPIC_NAME);
if (partitionInfos.isEmpty()) {
logger.info("No partitions found for topic {}, try to create topic", Topic.AUTO_BALANCER_METRICS_TOPIC_NAME);
return TopicAction.CREATE;
}
if (partitionInfos.size() != currentAssignment.size()) {
currentAssignment.clear();
for (PartitionInfo partitionInfo : partitionInfos) {
TopicPartition topicPartition = new TopicPartition(partitionInfo.topic(), partitionInfo.partition());
currentAssignment.add(topicPartition);
}
this.consumer.assign(currentAssignment);
logger.info("Partition changed for {}, assigned to {} partitions", Topic.AUTO_BALANCER_METRICS_TOPIC_NAME, currentAssignment.size());
}
if (partitionInfos.size() < metricReporterTopicPartition) {
logger.info("Partition num {} less than expected {}, try to alter partition number", partitionInfos.size(), metricReporterTopicPartition);
return TopicAction.ALTER;
}
return TopicAction.NONE;
}
public boolean isLeader() {
return isLeader;
}
@Override
public void onLeaderChanged(boolean isLeader) {
this.leaderEpochInitialized = true;
this.isLeader = isLeader;
}
protected void updateClusterModel(AutoBalancerMetrics metrics) {
switch (metrics.metricType()) {
case MetricTypes.TOPIC_PARTITION_METRIC:
TopicPartitionMetrics partitionMetrics = (TopicPartitionMetrics) metrics;
BrokerUpdater brokerUpdater = clusterModel.brokerUpdater(partitionMetrics.brokerId());
if (brokerUpdater != null && brokerUpdater.metricVersion() == MetricVersion.V0) {
clusterModel.updateBrokerMetrics(partitionMetrics.brokerId(), new HashMap<Byte, Double>().entrySet(), partitionMetrics.time());
}
clusterModel.updateTopicPartitionMetrics(partitionMetrics.brokerId(),
new TopicPartition(partitionMetrics.topic(), partitionMetrics.partition()),
partitionMetrics.getMetricValueMap().entrySet(), partitionMetrics.time());
break;
case MetricTypes.BROKER_METRIC:
BrokerMetrics brokerMetrics = (BrokerMetrics) metrics;
clusterModel.updateBrokerMetrics(brokerMetrics.brokerId(), brokerMetrics.getMetricValueMap().entrySet(), brokerMetrics.time());
break;
default:
logger.error("Not supported metrics version {}", metrics.metricType());
}
}
private enum TopicAction {
NONE,
CREATE,
ALTER
}
}