forked from apache/pulsar
/
BrokerService.java
2752 lines (2475 loc) · 136 KB
/
BrokerService.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
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.service;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun;
import static org.apache.commons.collections.CollectionUtils.isEmpty;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
import static org.apache.pulsar.broker.PulsarService.isTransactionSystemTopic;
import static org.apache.pulsar.common.events.EventsTopicNames.checkTopicIsEventsNames;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Queues;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.channel.AdaptiveRecvByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.socket.SocketChannel;
import io.netty.handler.ssl.SslContext;
import io.netty.util.concurrent.DefaultThreadFactory;
import java.io.Closeable;
import java.io.IOException;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
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.Set;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Consumer;
import java.util.function.Predicate;
import javax.ws.rs.core.Response;
import lombok.AccessLevel;
import lombok.Getter;
import lombok.Setter;
import org.apache.bookkeeper.common.util.OrderedExecutor;
import org.apache.bookkeeper.common.util.OrderedScheduler;
import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteLedgerCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback;
import org.apache.bookkeeper.mledger.LedgerOffloader;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerNotFoundException;
import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
import org.apache.bookkeeper.mledger.util.Futures;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.TransactionMetadataStoreService;
import org.apache.pulsar.broker.authentication.AuthenticationService;
import org.apache.pulsar.broker.authorization.AuthorizationService;
import org.apache.pulsar.broker.cache.BundlesQuotas;
import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory;
import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerLoader;
import org.apache.pulsar.broker.intercept.BrokerInterceptor;
import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl;
import org.apache.pulsar.broker.loadbalance.LoadManager;
import org.apache.pulsar.broker.resources.LocalPoliciesResources;
import org.apache.pulsar.broker.resources.NamespaceResources;
import org.apache.pulsar.broker.resources.NamespaceResources.PartitionedTopicResources;
import org.apache.pulsar.broker.service.BrokerServiceException.NamingException;
import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException;
import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException;
import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter;
import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.service.persistent.SystemTopic;
import org.apache.pulsar.broker.stats.ClusterReplicationMetrics;
import org.apache.pulsar.broker.stats.prometheus.metrics.ObserverGauge;
import org.apache.pulsar.broker.stats.prometheus.metrics.Summary;
import org.apache.pulsar.broker.systopic.SystemTopicClient;
import org.apache.pulsar.broker.validator.BindAddressValidator;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.PulsarAdminBuilder;
import org.apache.pulsar.client.api.ClientBuilder;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.impl.ClientBuilderImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.configuration.BindAddress;
import org.apache.pulsar.common.configuration.FieldContext;
import org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor;
import org.apache.pulsar.common.intercept.BrokerEntryMetadataInterceptor;
import org.apache.pulsar.common.intercept.BrokerEntryMetadataUtils;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicDomain;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride;
import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
import org.apache.pulsar.common.policies.data.BacklogQuota;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl;
import org.apache.pulsar.common.policies.data.PersistencePolicies;
import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.policies.data.PublishRate;
import org.apache.pulsar.common.policies.data.RetentionPolicies;
import org.apache.pulsar.common.policies.data.TopicPolicies;
import org.apache.pulsar.common.policies.data.TopicType;
import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl;
import org.apache.pulsar.common.protocol.schema.SchemaVersion;
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.common.util.FieldParser;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.RestException;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet;
import org.apache.pulsar.common.util.netty.ChannelFutures;
import org.apache.pulsar.common.util.netty.EventLoopUtil;
import org.apache.pulsar.common.util.netty.NettyFutureUtil;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.metadata.api.Notification;
import org.apache.pulsar.metadata.api.NotificationType;
import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats;
import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore;
import org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Getter(AccessLevel.PUBLIC)
@Setter(AccessLevel.PROTECTED)
public class BrokerService implements Closeable {
private static final Logger log = LoggerFactory.getLogger(BrokerService.class);
private static final Duration FUTURE_DEADLINE_TIMEOUT_DURATION = Duration.ofSeconds(60);
private static final TimeoutException FUTURE_DEADLINE_TIMEOUT_EXCEPTION =
FutureUtil.createTimeoutException("Future didn't finish within deadline", BrokerService.class,
"futureWithDeadline(...)");
private static final TimeoutException FAILED_TO_LOAD_TOPIC_TIMEOUT_EXCEPTION =
FutureUtil.createTimeoutException("Failed to load topic within timeout", BrokerService.class,
"futureWithDeadline(...)");
private static final long GRACEFUL_SHUTDOWN_QUIET_PERIOD_MAX_MS = 5000L;
private static final double GRACEFUL_SHUTDOWN_QUIET_PERIOD_RATIO_OF_TOTAL_TIMEOUT = 0.25d;
private static final double GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT = 0.5d;
private final PulsarService pulsar;
private final ManagedLedgerFactory managedLedgerFactory;
private final ConcurrentOpenHashMap<String, CompletableFuture<Optional<Topic>>> topics;
private final ConcurrentOpenHashMap<String, PulsarClient> replicationClients;
private final ConcurrentOpenHashMap<String, PulsarAdmin> clusterAdmins;
// Multi-layer topics map:
// Namespace --> Bundle --> topicName --> topic
private final ConcurrentOpenHashMap<String, ConcurrentOpenHashMap<String, ConcurrentOpenHashMap<String, Topic>>>
multiLayerTopicsMap;
// Keep track of topics and partitions served by this broker for fast lookup.
@Getter
private final ConcurrentOpenHashMap<String, ConcurrentOpenHashSet<Integer>> owningTopics;
private int numberOfNamespaceBundles = 0;
private final EventLoopGroup acceptorGroup;
private final EventLoopGroup workerGroup;
private final OrderedExecutor topicOrderedExecutor;
// offline topic backlog cache
private final ConcurrentOpenHashMap<TopicName, PersistentOfflineTopicStats> offlineTopicStatCache;
private static final ConcurrentOpenHashMap<String, ConfigField> dynamicConfigurationMap =
prepareDynamicConfigurationMap();
private final ConcurrentOpenHashMap<String, Consumer<?>> configRegisteredListeners;
private final ConcurrentLinkedQueue<Pair<String, CompletableFuture<Optional<Topic>>>> pendingTopicLoadingQueue;
private AuthorizationService authorizationService = null;
private final ScheduledExecutorService statsUpdater;
private final ScheduledExecutorService backlogQuotaChecker;
protected final AtomicReference<Semaphore> lookupRequestSemaphore;
protected final AtomicReference<Semaphore> topicLoadRequestSemaphore;
private final ObserverGauge pendingLookupRequests;
private final ObserverGauge pendingTopicLoadRequests;
private final ScheduledExecutorService inactivityMonitor;
private final ScheduledExecutorService messageExpiryMonitor;
private final ScheduledExecutorService compactionMonitor;
private final ScheduledExecutorService consumedLedgersMonitor;
private ScheduledExecutorService topicPublishRateLimiterMonitor;
private ScheduledExecutorService brokerPublishRateLimiterMonitor;
private ScheduledExecutorService deduplicationSnapshotMonitor;
protected volatile PublishRateLimiter brokerPublishRateLimiter = PublishRateLimiter.DISABLED_RATE_LIMITER;
private DistributedIdGenerator producerNameGenerator;
public static final String PRODUCER_NAME_GENERATOR_PATH = "/counters/producer-name";
private final BacklogQuotaManager backlogQuotaManager;
private final int keepAliveIntervalSeconds;
private final PulsarStats pulsarStats;
private final AuthenticationService authenticationService;
public static final String MANAGED_LEDGER_PATH_ZNODE = "/managed-ledgers";
private static final LongAdder totalUnackedMessages = new LongAdder();
private final int maxUnackedMessages;
public final int maxUnackedMsgsPerDispatcher;
private static final AtomicBoolean blockedDispatcherOnHighUnackedMsgs = new AtomicBoolean(false);
private final ConcurrentOpenHashSet<PersistentDispatcherMultipleConsumers> blockedDispatchers;
private final ReadWriteLock lock = new ReentrantReadWriteLock();
private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory;
private final ServerBootstrap defaultServerBootstrap;
@Getter
private final BundlesQuotas bundlesQuotas;
private PulsarChannelInitializer.Factory pulsarChannelInitFactory = PulsarChannelInitializer.DEFAULT_FACTORY;
private final List<Channel> listenChannels = new ArrayList<>(2);
private Channel listenChannel;
private Channel listenChannelTls;
private boolean preciseTopicPublishRateLimitingEnable;
private final LongAdder pausedConnections = new LongAdder();
private BrokerInterceptor interceptor;
private Set<BrokerEntryMetadataInterceptor> brokerEntryMetadataInterceptors;
public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws Exception {
this.pulsar = pulsar;
this.preciseTopicPublishRateLimitingEnable =
pulsar.getConfiguration().isPreciseTopicPublishRateLimiterEnable();
this.managedLedgerFactory = pulsar.getManagedLedgerFactory();
this.topics = new ConcurrentOpenHashMap<>();
this.replicationClients = new ConcurrentOpenHashMap<>();
this.clusterAdmins = new ConcurrentOpenHashMap<>();
this.keepAliveIntervalSeconds = pulsar.getConfiguration().getKeepAliveIntervalSeconds();
this.configRegisteredListeners = new ConcurrentOpenHashMap<>();
this.pendingTopicLoadingQueue = Queues.newConcurrentLinkedQueue();
this.multiLayerTopicsMap = new ConcurrentOpenHashMap<>();
this.owningTopics = new ConcurrentOpenHashMap<>();
this.pulsarStats = new PulsarStats(pulsar);
this.offlineTopicStatCache = new ConcurrentOpenHashMap<>();
this.topicOrderedExecutor = OrderedScheduler.newSchedulerBuilder()
.numThreads(pulsar.getConfiguration().getNumWorkerThreadsForNonPersistentTopic())
.name("broker-topic-workers").build();
final DefaultThreadFactory acceptorThreadFactory = new DefaultThreadFactory("pulsar-acceptor");
this.acceptorGroup = EventLoopUtil.newEventLoopGroup(
pulsar.getConfiguration().getNumAcceptorThreads(), false, acceptorThreadFactory);
this.workerGroup = eventLoopGroup;
this.statsUpdater = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
this.authorizationService = new AuthorizationService(
pulsar.getConfiguration(), pulsar().getPulsarResources());
pulsar.getLocalMetadataStore().registerListener(this::handleMetadataChanges);
pulsar.getConfigurationMetadataStore().registerListener(this::handleMetadataChanges);
this.inactivityMonitor = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-inactivity-monitor"));
this.messageExpiryMonitor = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-msg-expiry-monitor"));
this.compactionMonitor =
Executors.newSingleThreadScheduledExecutor(
new DefaultThreadFactory("pulsar-compaction-monitor"));
this.consumedLedgersMonitor = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("consumed-Ledgers-monitor"));
this.backlogQuotaManager = new BacklogQuotaManager(pulsar);
this.backlogQuotaChecker = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-backlog-quota-checker"));
this.authenticationService = new AuthenticationService(pulsar.getConfiguration());
this.blockedDispatchers = new ConcurrentOpenHashSet<>();
// update dynamic configuration and register-listener
updateConfigurationAndRegisterListeners();
this.lookupRequestSemaphore = new AtomicReference<Semaphore>(
new Semaphore(pulsar.getConfiguration().getMaxConcurrentLookupRequest(), false));
this.topicLoadRequestSemaphore = new AtomicReference<Semaphore>(
new Semaphore(pulsar.getConfiguration().getMaxConcurrentTopicLoadRequest(), false));
if (pulsar.getConfiguration().getMaxUnackedMessagesPerBroker() > 0
&& pulsar.getConfiguration().getMaxUnackedMessagesPerSubscriptionOnBrokerBlocked() > 0.0) {
this.maxUnackedMessages = pulsar.getConfiguration().getMaxUnackedMessagesPerBroker();
this.maxUnackedMsgsPerDispatcher = (int) ((maxUnackedMessages
* pulsar.getConfiguration().getMaxUnackedMessagesPerSubscriptionOnBrokerBlocked()) / 100);
log.info("Enabling per-broker unack-message limit {} and dispatcher-limit {} on blocked-broker",
maxUnackedMessages, maxUnackedMsgsPerDispatcher);
// block misbehaving dispatcher by checking periodically
pulsar.getExecutor().scheduleAtFixedRate(() -> checkUnAckMessageDispatching(),
600, 30, TimeUnit.SECONDS);
} else {
this.maxUnackedMessages = 0;
this.maxUnackedMsgsPerDispatcher = 0;
log.info(
"Disabling per broker unack-msg blocking due invalid"
+ " unAckMsgSubscriptionPercentageLimitOnBrokerBlocked {} ",
pulsar.getConfiguration().getMaxUnackedMessagesPerSubscriptionOnBrokerBlocked());
}
this.delayedDeliveryTrackerFactory = DelayedDeliveryTrackerLoader
.loadDelayedDeliveryTrackerFactory(pulsar.getConfiguration());
this.defaultServerBootstrap = defaultServerBootstrap();
this.pendingLookupRequests = ObserverGauge.build("pulsar_broker_lookup_pending_requests", "-")
.supplier(() -> pulsar.getConfig().getMaxConcurrentLookupRequest()
- lookupRequestSemaphore.get().availablePermits())
.register();
this.pendingTopicLoadRequests = ObserverGauge.build(
"pulsar_broker_topic_load_pending_requests", "-")
.supplier(() -> pulsar.getConfig().getMaxConcurrentTopicLoadRequest()
- topicLoadRequestSemaphore.get().availablePermits())
.register();
this.brokerEntryMetadataInterceptors = BrokerEntryMetadataUtils
.loadBrokerEntryMetadataInterceptors(pulsar.getConfiguration().getBrokerEntryMetadataInterceptors(),
BrokerService.class.getClassLoader());
this.bundlesQuotas = new BundlesQuotas(pulsar.getLocalMetadataStore());
}
// This call is used for starting additional protocol handlers
public void startProtocolHandlers(
Map<String, Map<InetSocketAddress, ChannelInitializer<SocketChannel>>> protocolHandlers) {
protocolHandlers.forEach((protocol, initializers) -> {
initializers.forEach((address, initializer) -> {
try {
startProtocolHandler(protocol, address, initializer);
} catch (IOException e) {
log.error("{}", e.getMessage(), e.getCause());
throw new RuntimeException(e.getMessage(), e.getCause());
}
});
});
}
private void startProtocolHandler(String protocol,
SocketAddress address,
ChannelInitializer<SocketChannel> initializer) throws IOException {
ServerBootstrap bootstrap = defaultServerBootstrap.clone();
bootstrap.childHandler(initializer);
try {
bootstrap.bind(address).sync();
} catch (Exception e) {
throw new IOException("Failed to bind protocol `" + protocol + "` on " + address, e);
}
log.info("Successfully bind protocol `{}` on {}", protocol, address);
}
private ServerBootstrap defaultServerBootstrap() {
ServerBootstrap bootstrap = new ServerBootstrap();
bootstrap.childOption(ChannelOption.ALLOCATOR, PulsarByteBufAllocator.DEFAULT);
bootstrap.group(acceptorGroup, workerGroup);
bootstrap.childOption(ChannelOption.TCP_NODELAY, true);
bootstrap.childOption(ChannelOption.RCVBUF_ALLOCATOR,
new AdaptiveRecvByteBufAllocator(1024, 16 * 1024, 1 * 1024 * 1024));
bootstrap.channel(EventLoopUtil.getServerSocketChannelClass(workerGroup));
EventLoopUtil.enableTriggeredMode(bootstrap);
return bootstrap;
}
public void start() throws Exception {
this.producerNameGenerator = new DistributedIdGenerator(pulsar.getCoordinationService(),
PRODUCER_NAME_GENERATOR_PATH, pulsar.getConfiguration().getClusterName());
ServiceConfiguration serviceConfig = pulsar.getConfiguration();
List<BindAddress> bindAddresses = BindAddressValidator.validateBindAddresses(serviceConfig,
Arrays.asList("pulsar", "pulsar+ssl"));
String internalListenerName = serviceConfig.getInternalListenerName();
// create a channel for each bind address
if (bindAddresses.size() == 0) {
throw new IllegalArgumentException("At least one broker bind address must be configured");
}
for (BindAddress a : bindAddresses) {
InetSocketAddress addr = new InetSocketAddress(a.getAddress().getHost(), a.getAddress().getPort());
boolean isTls = "pulsar+ssl".equals(a.getAddress().getScheme());
PulsarChannelInitializer.PulsarChannelOptions opts = PulsarChannelInitializer.PulsarChannelOptions.builder()
.enableTLS(isTls)
.listenerName(a.getListenerName()).build();
ServerBootstrap b = defaultServerBootstrap.clone();
b.childHandler(
pulsarChannelInitFactory.newPulsarChannelInitializer(pulsar, opts));
try {
Channel ch = b.bind(addr).sync().channel();
listenChannels.add(ch);
// identify the primary channel. Note that the legacy bindings appear first and have no listener.
if (StringUtils.isBlank(a.getListenerName())
|| StringUtils.equalsIgnoreCase(a.getListenerName(), internalListenerName)) {
if (this.listenChannel == null && !isTls) {
this.listenChannel = ch;
}
if (this.listenChannelTls == null && isTls) {
this.listenChannelTls = ch;
}
}
log.info("Started Pulsar Broker service on {}, TLS: {}, listener: {}",
ch.localAddress(),
isTls ? SslContext.defaultServerProvider().toString() : "(none)",
StringUtils.defaultString(a.getListenerName(), "(none)"));
} catch (Exception e) {
throw new IOException("Failed to bind Pulsar broker on " + addr, e);
}
}
// start other housekeeping functions
this.startStatsUpdater(
serviceConfig.getStatsUpdateInitialDelayInSecs(),
serviceConfig.getStatsUpdateFrequencyInSecs());
this.startInactivityMonitor();
this.startMessageExpiryMonitor();
this.startCompactionMonitor();
this.startConsumedLedgersMonitor();
this.startBacklogQuotaChecker();
this.updateBrokerPublisherThrottlingMaxRate();
this.startCheckReplicationPolicies();
this.startDeduplicationSnapshotMonitor();
}
protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpdateFrequencyInSecs) {
statsUpdater.scheduleAtFixedRate(safeRun(this::updateRates),
statsUpdateInitialDelayInSecs, statsUpdateFrequencyInSecs, TimeUnit.SECONDS);
// Ensure the broker starts up with initial stats
updateRates();
}
protected void startDeduplicationSnapshotMonitor() {
int interval = pulsar().getConfiguration().getBrokerDeduplicationSnapshotFrequencyInSeconds();
if (interval > 0 && pulsar().getConfiguration().isBrokerDeduplicationEnabled()) {
this.deduplicationSnapshotMonitor =
Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory(
"deduplication-snapshot-monitor"));
deduplicationSnapshotMonitor.scheduleAtFixedRate(safeRun(() -> forEachTopic(
Topic::checkDeduplicationSnapshot))
, interval, interval, TimeUnit.SECONDS);
}
}
protected void startInactivityMonitor() {
if (pulsar().getConfiguration().isBrokerDeleteInactiveTopicsEnabled()) {
int interval = pulsar().getConfiguration().getBrokerDeleteInactiveTopicsFrequencySeconds();
inactivityMonitor.scheduleAtFixedRate(safeRun(() -> checkGC()), interval, interval,
TimeUnit.SECONDS);
}
// Deduplication info checker
long duplicationCheckerIntervalInSeconds = TimeUnit.MINUTES
.toSeconds(pulsar().getConfiguration().getBrokerDeduplicationProducerInactivityTimeoutMinutes()) / 3;
inactivityMonitor.scheduleAtFixedRate(safeRun(this::checkMessageDeduplicationInfo),
duplicationCheckerIntervalInSeconds,
duplicationCheckerIntervalInSeconds, TimeUnit.SECONDS);
// Inactive subscriber checker
if (pulsar().getConfiguration().getSubscriptionExpiryCheckIntervalInMinutes() > 0) {
long subscriptionExpiryCheckIntervalInSeconds =
TimeUnit.MINUTES.toSeconds(pulsar().getConfiguration()
.getSubscriptionExpiryCheckIntervalInMinutes());
inactivityMonitor.scheduleAtFixedRate(safeRun(this::checkInactiveSubscriptions),
subscriptionExpiryCheckIntervalInSeconds,
subscriptionExpiryCheckIntervalInSeconds, TimeUnit.SECONDS);
}
}
protected void startMessageExpiryMonitor() {
int interval = pulsar().getConfiguration().getMessageExpiryCheckIntervalInMinutes();
messageExpiryMonitor.scheduleAtFixedRate(safeRun(this::checkMessageExpiry), interval, interval,
TimeUnit.MINUTES);
}
protected void startCheckReplicationPolicies() {
int interval = pulsar.getConfig().getReplicationPolicyCheckDurationSeconds();
if (interval > 0) {
messageExpiryMonitor.scheduleAtFixedRate(safeRun(this::checkReplicationPolicies), interval, interval,
TimeUnit.SECONDS);
}
}
protected void startCompactionMonitor() {
int interval = pulsar().getConfiguration().getBrokerServiceCompactionMonitorIntervalInSeconds();
if (interval > 0) {
compactionMonitor.scheduleAtFixedRate(safeRun(() -> checkCompaction()),
interval, interval, TimeUnit.SECONDS);
}
}
protected void startConsumedLedgersMonitor() {
int interval = pulsar().getConfiguration().getRetentionCheckIntervalInSeconds();
if (interval > 0) {
consumedLedgersMonitor.scheduleAtFixedRate(safeRun(this::checkConsumedLedgers),
interval, interval, TimeUnit.SECONDS);
}
}
protected void startBacklogQuotaChecker() {
if (pulsar().getConfiguration().isBacklogQuotaCheckEnabled()) {
final int interval = pulsar().getConfiguration().getBacklogQuotaCheckIntervalInSeconds();
log.info("Scheduling a thread to check backlog quota after [{}] seconds in background", interval);
backlogQuotaChecker.scheduleAtFixedRate(safeRun(this::monitorBacklogQuota), interval, interval,
TimeUnit.SECONDS);
} else {
log.info("Backlog quota check monitoring is disabled");
}
}
/**
* Schedules and monitors publish-throttling for all owned topics that has publish-throttling configured. It also
* disables and shutdowns publish-rate-limiter monitor task if broker disables it.
*/
public synchronized void setupTopicPublishRateLimiterMonitor() {
// set topic PublishRateLimiterMonitor
long topicTickTimeMs = pulsar().getConfiguration().getTopicPublisherThrottlingTickTimeMillis();
if (topicTickTimeMs > 0) {
if (this.topicPublishRateLimiterMonitor == null) {
this.topicPublishRateLimiterMonitor = Executors.newSingleThreadScheduledExecutor(
new DefaultThreadFactory("pulsar-topic-publish-rate-limiter-monitor"));
if (topicTickTimeMs > 0) {
// schedule task that sums up publish-rate across all cnx on a topic
topicPublishRateLimiterMonitor.scheduleAtFixedRate(safeRun(() -> checkTopicPublishThrottlingRate()),
topicTickTimeMs, topicTickTimeMs, TimeUnit.MILLISECONDS);
// schedule task that refreshes rate-limiting bucket
topicPublishRateLimiterMonitor.scheduleAtFixedRate(safeRun(() -> refreshTopicPublishRate()), 1, 1,
TimeUnit.SECONDS);
}
}
} else {
// disable publish-throttling for all topics
if (this.topicPublishRateLimiterMonitor != null) {
try {
this.topicPublishRateLimiterMonitor.awaitTermination(30, TimeUnit.SECONDS);
} catch (InterruptedException e) {
log.warn("failed to shutdown topicPublishRateLimiterMonitor", e);
}
// make sure topics are not being throttled
refreshTopicPublishRate();
this.topicPublishRateLimiterMonitor = null;
}
}
}
/**
* Schedules and monitors publish-throttling for broker that has publish-throttling configured. It also
* disables and shutdowns publish-rate-limiter monitor for broker task if broker disables it.
*/
public synchronized void setupBrokerPublishRateLimiterMonitor() {
// set broker PublishRateLimiterMonitor
long brokerTickTimeMs = pulsar().getConfiguration().getBrokerPublisherThrottlingTickTimeMillis();
if (brokerTickTimeMs > 0) {
if (this.brokerPublishRateLimiterMonitor == null) {
this.brokerPublishRateLimiterMonitor = Executors.newSingleThreadScheduledExecutor(
new DefaultThreadFactory("pulsar-broker-publish-rate-limiter-monitor"));
if (brokerTickTimeMs > 0) {
// schedule task that sums up publish-rate across all cnx on a topic,
// and check the rate limit exceeded or not.
brokerPublishRateLimiterMonitor.scheduleAtFixedRate(
safeRun(() -> checkBrokerPublishThrottlingRate()),
brokerTickTimeMs,
brokerTickTimeMs,
TimeUnit.MILLISECONDS);
// schedule task that refreshes rate-limiting bucket
brokerPublishRateLimiterMonitor.scheduleAtFixedRate(
safeRun(() -> refreshBrokerPublishRate()),
1,
1,
TimeUnit.SECONDS);
}
}
} else {
// disable publish-throttling for broker.
if (this.brokerPublishRateLimiterMonitor != null) {
try {
this.brokerPublishRateLimiterMonitor.awaitTermination(30, TimeUnit.SECONDS);
} catch (InterruptedException e) {
log.warn("failed to shutdown brokerPublishRateLimiterMonitor", e);
}
// make sure topics are not being throttled
refreshBrokerPublishRate();
this.brokerPublishRateLimiterMonitor = null;
}
}
}
public void close() throws IOException {
try {
closeAsync().get();
} catch (ExecutionException e) {
if (e.getCause() instanceof IOException) {
throw (IOException) e.getCause();
} else {
throw new PulsarServerException(e.getCause());
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
public CompletableFuture<Void> closeAndRemoveReplicationClient(String clusterName) {
List<CompletableFuture<Void>> futures = new ArrayList<>((int) topics.size());
topics.forEach((__, future) -> {
CompletableFuture<Void> f = new CompletableFuture<>();
futures.add(f);
future.whenComplete((ot, ex) -> {
if (ot.isPresent()) {
Replicator r = ot.get().getReplicators().get(clusterName);
if (r != null && r.isConnected()) {
r.disconnect(false).whenComplete((v, e) -> f.complete(null));
return;
}
}
f.complete(null);
});
});
return FutureUtil.waitForAll(futures).thenCompose(__ -> {
PulsarClient client = replicationClients.remove(clusterName);
if (client == null) {
return CompletableFuture.completedFuture(null);
}
return client.closeAsync();
});
}
public CompletableFuture<Void> closeAsync() {
try {
log.info("Shutting down Pulsar Broker service");
// unloads all namespaces gracefully without disrupting mutually
unloadNamespaceBundlesGracefully();
// close replication clients
replicationClients.forEach((cluster, client) -> {
try {
client.shutdown();
} catch (Exception e) {
log.warn("Error shutting down repl client for cluster {}", cluster, e);
}
});
// close replication admins
clusterAdmins.forEach((cluster, admin) -> {
try {
admin.close();
} catch (Exception e) {
log.warn("Error shutting down repl admin for cluster {}", cluster, e);
}
});
CompletableFuture<CompletableFuture<Void>> cancellableDownstreamFutureReference = new CompletableFuture<>();
log.info("Event loops shutting down gracefully...");
CompletableFuture<Void> shutdownFuture =
CompletableFuture.allOf(shutdownEventLoopGracefully(acceptorGroup),
shutdownEventLoopGracefully(workerGroup))
.handle((v, t) -> {
if (t != null) {
log.warn("Error shutting down event loops gracefully", t);
} else {
log.info("Event loops shutdown completed.");
}
return null;
})
.thenCompose(__ -> {
log.info("Continuing to second phase in shutdown.");
List<CompletableFuture<Void>> asyncCloseFutures = new ArrayList<>();
listenChannels.forEach(ch -> {
if (ch.isOpen()) {
asyncCloseFutures.add(closeChannel(ch));
}
});
if (interceptor != null) {
interceptor.close();
interceptor = null;
}
try {
authenticationService.close();
} catch (IOException e) {
log.warn("Error in closing authenticationService", e);
}
pulsarStats.close();
try {
delayedDeliveryTrackerFactory.close();
} catch (IOException e) {
log.warn("Error in closing delayedDeliveryTrackerFactory", e);
}
asyncCloseFutures.add(GracefulExecutorServicesShutdown
.initiate()
.timeout(
Duration.ofMillis(
(long) (GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT
* pulsar.getConfiguration()
.getBrokerShutdownTimeoutMs())))
.shutdown(
statsUpdater,
inactivityMonitor,
messageExpiryMonitor,
compactionMonitor,
consumedLedgersMonitor,
backlogQuotaChecker,
topicOrderedExecutor,
topicPublishRateLimiterMonitor,
brokerPublishRateLimiterMonitor,
deduplicationSnapshotMonitor)
.handle());
CompletableFuture<Void> combined =
FutureUtil.waitForAllAndSupportCancel(asyncCloseFutures);
cancellableDownstreamFutureReference.complete(combined);
combined.handle((v, t) -> {
if (t == null) {
log.info("Broker service completely shut down");
} else {
if (t instanceof CancellationException) {
log.warn("Broker service didn't complete gracefully. "
+ "Terminating Broker service.");
} else {
log.warn("Broker service shut down completed with exception", t);
}
}
return null;
});
return combined;
});
FutureUtil.whenCancelledOrTimedOut(shutdownFuture, () -> cancellableDownstreamFutureReference
.thenAccept(future -> future.cancel(false)));
return shutdownFuture;
} catch (Exception e) {
return FutureUtil.failedFuture(e);
}
}
CompletableFuture<Void> shutdownEventLoopGracefully(EventLoopGroup eventLoopGroup) {
long brokerShutdownTimeoutMs = pulsar.getConfiguration().getBrokerShutdownTimeoutMs();
long quietPeriod = Math.min((long) (
GRACEFUL_SHUTDOWN_QUIET_PERIOD_RATIO_OF_TOTAL_TIMEOUT * brokerShutdownTimeoutMs),
GRACEFUL_SHUTDOWN_QUIET_PERIOD_MAX_MS);
long timeout = (long) (GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT * brokerShutdownTimeoutMs);
return NettyFutureUtil.toCompletableFutureVoid(
eventLoopGroup.shutdownGracefully(quietPeriod,
timeout, TimeUnit.MILLISECONDS));
}
private CompletableFuture<Void> closeChannel(Channel channel) {
return ChannelFutures.toCompletableFuture(channel.close())
.handle((c, t) -> {
// log problem if closing of channel fails
// ignore RejectedExecutionException
if (t != null && !(t instanceof RejectedExecutionException)) {
log.warn("Cannot close channel {}", channel, t);
}
return null;
});
}
/**
* It unloads all owned namespacebundles gracefully.
* <ul>
* <li>First it makes current broker unavailable and isolates from the clusters so, it will not serve any new
* requests.</li>
* <li>Second it starts unloading namespace bundle one by one without closing the connection in order to avoid
* disruption for other namespacebundles which are sharing the same connection from the same client.</li>
* </ul>
*/
public void unloadNamespaceBundlesGracefully() {
try {
log.info("Unloading namespace-bundles...");
// make broker-node unavailable from the cluster
if (pulsar.getLoadManager() != null && pulsar.getLoadManager().get() != null) {
try {
pulsar.getLoadManager().get().disableBroker();
} catch (PulsarServerException.NotFoundException ne) {
log.warn("Broker load-manager znode doesn't exist ", ne);
// still continue and release bundle ownership as broker's registration node doesn't exist.
}
}
// unload all namespace-bundles gracefully
long closeTopicsStartTime = System.nanoTime();
Set<NamespaceBundle> serviceUnits = pulsar.getNamespaceService().getOwnedServiceUnits();
serviceUnits.forEach(su -> {
if (su instanceof NamespaceBundle) {
try {
pulsar.getNamespaceService().unloadNamespaceBundle(su, 1, TimeUnit.MINUTES).get();
} catch (Exception e) {
log.warn("Failed to unload namespace bundle {}", su, e);
}
}
});
double closeTopicsTimeSeconds = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - closeTopicsStartTime))
/ 1000.0;
log.info("Unloading {} namespace-bundles completed in {} seconds", serviceUnits.size(),
closeTopicsTimeSeconds);
} catch (Exception e) {
log.error("Failed to disable broker from loadbalancer list {}", e.getMessage(), e);
}
}
public CompletableFuture<Optional<Topic>> getTopicIfExists(final String topic) {
return getTopic(topic, false /* createIfMissing */);
}
public CompletableFuture<Topic> getOrCreateTopic(final String topic) {
return getTopic(topic, isAllowAutoTopicCreation(topic)).thenApply(Optional::get);
}
public CompletableFuture<Optional<Topic>> getTopic(final String topic, boolean createIfMissing) {
try {
CompletableFuture<Optional<Topic>> topicFuture = topics.get(topic);
if (topicFuture != null) {
if (topicFuture.isCompletedExceptionally()
|| (topicFuture.isDone() && !topicFuture.getNow(Optional.empty()).isPresent())) {
// Exceptional topics should be recreated.
topics.remove(topic, topicFuture);
} else {
// a non-existing topic in the cache shouldn't prevent creating a topic
if (createIfMissing) {
if (topicFuture.isDone() && topicFuture.getNow(Optional.empty()).isPresent()) {
return topicFuture;
} else {
return topicFuture.thenCompose(value -> {
if (!value.isPresent()) {
// retry and create topic
return getTopic(topic, createIfMissing);
} else {
// in-progress future completed successfully
return CompletableFuture.completedFuture(value);
}
});
}
} else {
return topicFuture;
}
}
}
final boolean isPersistentTopic = TopicName.get(topic).getDomain().equals(TopicDomain.persistent);
if (isPersistentTopic) {
return topics.computeIfAbsent(topic, (topicName) -> {
return this.loadOrCreatePersistentTopic(topicName, createIfMissing);
});
} else {
return topics.computeIfAbsent(topic, (name) -> {
final TopicName topicName = TopicName.get(name);
if (topicName.isPartitioned()) {
final TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName());
return this.fetchPartitionedTopicMetadataAsync(partitionedTopicName).thenCompose((metadata) -> {
if (topicName.getPartitionIndex() < metadata.partitions) {
return createNonPersistentTopic(name);
}
return CompletableFuture.completedFuture(Optional.empty());
});
} else if (createIfMissing) {
return createNonPersistentTopic(name);
} else {
return CompletableFuture.completedFuture(Optional.empty());
}
});
}
} catch (IllegalArgumentException e) {
log.warn("[{}] Illegalargument exception when loading topic", topic, e);
return FutureUtil.failedFuture(e);
} catch (RuntimeException e) {
Throwable cause = e.getCause();
if (cause instanceof ServiceUnitNotReadyException) {
log.warn("[{}] Service unit is not ready when loading the topic", topic);
} else {
log.warn("[{}] Unexpected exception when loading topic: {}", topic, e.getMessage(), e);
}
return FutureUtil.failedFuture(cause);
}
}
public CompletableFuture<SchemaVersion> deleteSchemaStorage(String topic) {
Optional<Topic> optTopic = getTopicReference(topic);
if (optTopic.isPresent()) {
return optTopic.get().deleteSchema();
} else {
return CompletableFuture.completedFuture(null);
}
}
public CompletableFuture<Void> deleteTopic(String topic, boolean forceDelete) {
return deleteTopic(topic, forceDelete, false);
}
public CompletableFuture<Void> deleteTopic(String topic, boolean forceDelete, boolean deleteSchema) {
Optional<Topic> optTopic = getTopicReference(topic);
if (optTopic.isPresent()) {
Topic t = optTopic.get();
if (forceDelete) {
if (deleteSchema) {
return t.deleteSchema().thenCompose(schemaVersion -> {
log.info("Successfully delete topic {}'s schema of version {}", t.getName(), schemaVersion);
return t.deleteForcefully();
});
} else {
return t.deleteForcefully();
}
}
// v2 topics have a global name so check if the topic is replicated.
if (t.isReplicated()) {
// Delete is disallowed on global topic
final List<String> clusters = t.getReplicators().keys();
log.error("Delete forbidden topic {} is replicated on clusters {}", topic, clusters);
return FutureUtil.failedFuture(
new IllegalStateException("Delete forbidden topic is replicated on clusters " + clusters));
}
if (deleteSchema) {
return t.deleteSchema().thenCompose(schemaVersion -> {
log.info("Successfully delete topic {}'s schema of version {}", t.getName(), schemaVersion);
return t.delete();
});
} else {
return t.delete();
}
}
if (log.isDebugEnabled()) {
log.debug("Topic {} is not loaded, try to delete from metadata", topic);
}
// Topic is not loaded, though we still might be able to delete from metadata
TopicName tn = TopicName.get(topic);
if (!tn.isPersistent()) {
// Nothing to do if it's not persistent
return CompletableFuture.completedFuture(null);
}
CompletableFuture<Void> future = new CompletableFuture<>();
CompletableFuture<Void> deleteTopicAuthenticationFuture = new CompletableFuture<>();
deleteTopicAuthenticationWithRetry(topic, deleteTopicAuthenticationFuture, 5);
deleteTopicAuthenticationFuture.whenComplete((v, ex) -> {
if (ex != null) {
future.completeExceptionally(ex);
return;
}
managedLedgerFactory.asyncDelete(tn.getPersistenceNamingEncoding(), new DeleteLedgerCallback() {
@Override
public void deleteLedgerComplete(Object ctx) {
future.complete(null);
}
@Override
public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) {