@@ -229,8 +229,7 @@ public class BrokerService implements Closeable {
229
229
private final OrderedExecutor topicOrderedExecutor ;
230
230
// offline topic backlog cache
231
231
private final ConcurrentOpenHashMap <TopicName , PersistentOfflineTopicStats > offlineTopicStatCache ;
232
- private final ConcurrentOpenHashMap <String , ConfigField > dynamicConfigurationMap =
233
- prepareDynamicConfigurationMap ();
232
+ private final ConcurrentOpenHashMap <String , ConfigField > dynamicConfigurationMap ;
234
233
private final ConcurrentOpenHashMap <String , Consumer <?>> configRegisteredListeners ;
235
234
236
235
private final ConcurrentLinkedQueue <TopicLoadingContext > pendingTopicLoadingQueue ;
@@ -313,6 +312,7 @@ public class BrokerService implements Closeable {
313
312
314
313
public BrokerService (PulsarService pulsar , EventLoopGroup eventLoopGroup ) throws Exception {
315
314
this .pulsar = pulsar ;
315
+ this .dynamicConfigurationMap = prepareDynamicConfigurationMap ();
316
316
this .brokerPublishRateLimiter = new PublishRateLimiterImpl (pulsar .getMonotonicSnapshotClock ());
317
317
this .preciseTopicPublishRateLimitingEnable =
318
318
pulsar .getConfiguration ().isPreciseTopicPublishRateLimiterEnable ();
@@ -2496,40 +2496,71 @@ private void handleDynamicConfigurationUpdates() {
2496
2496
2497
2497
if (dynamicConfigResources != null ) {
2498
2498
dynamicConfigResources .getDynamicConfigurationAsync ()
2499
- .thenAccept (optMap -> {
2500
- if (!optMap .isPresent ()) {
2501
- return ;
2499
+ .thenAccept (optMap -> {
2500
+ // Case some dynamic configs have been removed.
2501
+ dynamicConfigurationMap .forEach ((configKey , fieldWrapper ) -> {
2502
+ boolean configRemoved = optMap .isEmpty () || !optMap .get ().containsKey (configKey );
2503
+ if (fieldWrapper .lastDynamicValue != null && configRemoved ) {
2504
+ configValueChanged (configKey , null );
2502
2505
}
2503
- Map <String , String > data = optMap .get ();
2504
- data .forEach ((configKey , value ) -> {
2505
- ConfigField configFieldWrapper = dynamicConfigurationMap .get (configKey );
2506
- if (configFieldWrapper == null ) {
2507
- log .warn ("{} does not exist in dynamicConfigurationMap, skip this config." , configKey );
2508
- return ;
2509
- }
2510
- Field configField = configFieldWrapper .field ;
2511
- Consumer listener = configRegisteredListeners .get (configKey );
2512
- try {
2513
- final Object existingValue ;
2514
- final Object newValue ;
2515
- if (configField != null ) {
2516
- newValue = FieldParser .value (data .get (configKey ), configField );
2517
- existingValue = configField .get (pulsar .getConfiguration ());
2518
- configField .set (pulsar .getConfiguration (), newValue );
2519
- } else {
2520
- newValue = value ;
2521
- existingValue = configFieldWrapper .customValue ;
2522
- configFieldWrapper .customValue = newValue == null ? null : String .valueOf (newValue );
2523
- }
2524
- log .info ("Successfully updated configuration {}/{}" , configKey , data .get (configKey ));
2525
- if (listener != null && !Objects .equals (existingValue , newValue )) {
2526
- listener .accept (newValue );
2527
- }
2528
- } catch (Exception e ) {
2529
- log .error ("Failed to update config {}" , configKey , e );
2530
- }
2531
- });
2532
2506
});
2507
+ // Some configs have been changed.
2508
+ if (!optMap .isPresent ()) {
2509
+ return ;
2510
+ }
2511
+ Map <String , String > data = optMap .get ();
2512
+ data .forEach ((configKey , value ) -> {
2513
+ configValueChanged (configKey , value );
2514
+ });
2515
+ });
2516
+ }
2517
+ }
2518
+
2519
+ private void configValueChanged (String configKey , String newValueStr ) {
2520
+ ConfigField configFieldWrapper = dynamicConfigurationMap .get (configKey );
2521
+ if (configFieldWrapper == null ) {
2522
+ log .warn ("{} does not exist in dynamicConfigurationMap, skip this config." , configKey );
2523
+ return ;
2524
+ }
2525
+ Consumer listener = configRegisteredListeners .get (configKey );
2526
+ try {
2527
+ // Convert existingValue and newValue.
2528
+ final Object existingValue ;
2529
+ final Object newValue ;
2530
+ if (configFieldWrapper .field != null ) {
2531
+ if (StringUtils .isBlank (newValueStr )) {
2532
+ newValue = configFieldWrapper .defaultValue ;
2533
+ } else {
2534
+ newValue = FieldParser .value (newValueStr , configFieldWrapper .field );
2535
+ }
2536
+ existingValue = configFieldWrapper .field .get (pulsar .getConfiguration ());
2537
+ configFieldWrapper .field .set (pulsar .getConfiguration (), newValue );
2538
+ } else {
2539
+ // This case only occurs when it is a customized item.
2540
+ // See: https://github.com/apache/pulsar/blob/master/pip/pip-300.md.
2541
+ log .info ("Skip update customized dynamic configuration {}/{} in memory, only trigger an event"
2542
+ + " listeners." , configKey , newValueStr );
2543
+ existingValue = configFieldWrapper .lastDynamicValue ;
2544
+ newValue = newValueStr == null ? configFieldWrapper .defaultValue : newValueStr ;
2545
+ }
2546
+ // Record the latest dynamic config.
2547
+ configFieldWrapper .lastDynamicValue = newValueStr ;
2548
+
2549
+ if (newValueStr == null ) {
2550
+ log .info ("Successfully remove the dynamic configuration {}, and revert to the default value" ,
2551
+ configKey );
2552
+ } else {
2553
+ log .info ("Successfully updated configuration {}/{}" , configKey , newValueStr );
2554
+ }
2555
+
2556
+ if (listener != null && !Objects .equals (existingValue , newValue )) {
2557
+ // So far, all config items that related to configuration listeners, their default value is not null.
2558
+ // And the customized config can be null before.
2559
+ // So call "listener.accept(null)" is okay.
2560
+ listener .accept (newValue );
2561
+ }
2562
+ } catch (Exception e ) {
2563
+ log .error ("Failed to update config {}" , configKey , e );
2533
2564
}
2534
2565
}
2535
2566
@@ -2936,6 +2967,9 @@ private void updateManagedLedgerConfig() {
2936
2967
* On notification, listener should first check if config value has been changed and after taking appropriate
2937
2968
* action, listener should update config value with new value if it has been changed (so, next time listener can
2938
2969
* compare values on configMap change).
2970
+ *
2971
+ * Note: The new value that the {@param listener} may accept could be a null value.
2972
+ *
2939
2973
* @param <T>
2940
2974
*
2941
2975
* @param configKey
@@ -3057,16 +3091,23 @@ public boolean validateDynamicConfiguration(String key, String value) {
3057
3091
return true ;
3058
3092
}
3059
3093
3060
- private static ConcurrentOpenHashMap <String , ConfigField > prepareDynamicConfigurationMap () {
3094
+ private ConcurrentOpenHashMap <String , ConfigField > prepareDynamicConfigurationMap () {
3061
3095
ConcurrentOpenHashMap <String , ConfigField > dynamicConfigurationMap =
3062
3096
ConcurrentOpenHashMap .<String , ConfigField >newBuilder ().build ();
3063
- for (Field field : ServiceConfiguration .class .getDeclaredFields ()) {
3064
- if (field != null && field .isAnnotationPresent (FieldContext .class )) {
3065
- field .setAccessible (true );
3066
- if (field .getAnnotation (FieldContext .class ).dynamic ()) {
3067
- dynamicConfigurationMap .put (field .getName (), new ConfigField (field ));
3097
+ try {
3098
+ for (Field field : ServiceConfiguration .class .getDeclaredFields ()) {
3099
+ if (field != null && field .isAnnotationPresent (FieldContext .class )) {
3100
+ field .setAccessible (true );
3101
+ if (field .getAnnotation (FieldContext .class ).dynamic ()) {
3102
+ Object defaultValue = field .get (pulsar .getConfiguration ());
3103
+ dynamicConfigurationMap .put (field .getName (), new ConfigField (field , defaultValue ));
3104
+ }
3068
3105
}
3069
3106
}
3107
+ } catch (IllegalArgumentException | IllegalAccessException ex ) {
3108
+ // This error never occurs.
3109
+ log .error ("Failed to initialize dynamic configuration map" , ex );
3110
+ throw new RuntimeException (ex );
3070
3111
}
3071
3112
return dynamicConfigurationMap ;
3072
3113
}
@@ -3348,19 +3389,25 @@ private static class ConfigField {
3348
3389
// field holds the pulsar dynamic configuration.
3349
3390
final Field field ;
3350
3391
3351
- // customValue holds the external dynamic configuration.
3352
- volatile String customValue ;
3392
+ // It is the dynamic config value if set.
3393
+ // It is null if has does not set a dynamic config, even if the value of "pulsar.config" is present.
3394
+ volatile String lastDynamicValue ;
3395
+
3396
+ // The default value of "pulsar.config", which is initialized when the broker is starting.
3397
+ // After the dynamic config has been removed, revert the config to this default value.
3398
+ final Object defaultValue ;
3353
3399
3354
3400
Predicate <String > validator ;
3355
3401
3356
- public ConfigField (Field field ) {
3402
+ public ConfigField (Field field , Object defaultValue ) {
3357
3403
super ();
3358
3404
this .field = field ;
3405
+ this .defaultValue = defaultValue ;
3359
3406
}
3360
3407
3361
3408
public static ConfigField newCustomConfigField (String customValue ) {
3362
- ConfigField configField = new ConfigField (null );
3363
- configField .customValue = customValue ;
3409
+ ConfigField configField = new ConfigField (null , null );
3410
+ configField .lastDynamicValue = customValue ;
3364
3411
return configField ;
3365
3412
}
3366
3413
}
0 commit comments