forked from apache/pulsar
/
PulsarService.java
1679 lines (1457 loc) · 75.4 KB
/
PulsarService.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;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.commons.lang3.StringUtils.isBlank;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
import static org.apache.pulsar.broker.resourcegroup.ResourceUsageTransportManager.DISABLE_RESOURCE_USAGE_TRANSPORT_MANAGER;
import static org.apache.pulsar.common.naming.TopicName.TRANSACTION_COORDINATOR_LOG;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.socket.SocketChannel;
import io.netty.util.HashedWheelTimer;
import io.netty.util.concurrent.DefaultThreadFactory;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
import java.net.MalformedURLException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Consumer;
import java.util.function.Supplier;
import javax.servlet.ServletException;
import javax.websocket.DeploymentException;
import lombok.AccessLevel;
import lombok.Getter;
import lombok.Setter;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.common.util.OrderedExecutor;
import org.apache.bookkeeper.common.util.OrderedScheduler;
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.bookkeeper.mledger.LedgerOffloader;
import org.apache.bookkeeper.mledger.LedgerOffloaderFactory;
import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader;
import org.apache.bookkeeper.mledger.offload.Offloaders;
import org.apache.bookkeeper.mledger.offload.OffloadersCache;
import org.apache.commons.configuration.ConfigurationException;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
import org.apache.pulsar.PulsarVersion;
import org.apache.pulsar.broker.authentication.AuthenticationService;
import org.apache.pulsar.broker.authorization.AuthorizationService;
import org.apache.pulsar.broker.intercept.BrokerInterceptor;
import org.apache.pulsar.broker.intercept.BrokerInterceptors;
import org.apache.pulsar.broker.loadbalance.LeaderElectionService;
import org.apache.pulsar.broker.loadbalance.LoadManager;
import org.apache.pulsar.broker.loadbalance.LoadReportUpdaterTask;
import org.apache.pulsar.broker.loadbalance.LoadResourceQuotaUpdaterTask;
import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.broker.protocol.ProtocolHandlers;
import org.apache.pulsar.broker.resourcegroup.ResourceGroupService;
import org.apache.pulsar.broker.resourcegroup.ResourceUsageTopicTransportManager;
import org.apache.pulsar.broker.resourcegroup.ResourceUsageTransportManager;
import org.apache.pulsar.broker.resources.ClusterResources;
import org.apache.pulsar.broker.resources.PulsarResources;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.GracefulExecutorServicesShutdown;
import org.apache.pulsar.broker.service.SystemTopicBaseTxnBufferSnapshotService;
import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.TopicPoliciesService;
import org.apache.pulsar.broker.service.TransactionBufferSnapshotService;
import org.apache.pulsar.broker.service.schema.SchemaRegistryService;
import org.apache.pulsar.broker.stats.MetricsGenerator;
import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet;
import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider;
import org.apache.pulsar.broker.storage.ManagedLedgerStorage;
import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider;
import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl;
import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider;
import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore;
import org.apache.pulsar.broker.validator.MultipleListenerValidator;
import org.apache.pulsar.broker.web.WebService;
import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlet;
import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithClassLoader;
import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithPulsarService;
import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.PulsarAdminBuilder;
import org.apache.pulsar.client.api.AuthenticationFactory;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.transaction.TransactionBufferClient;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
import org.apache.pulsar.common.conf.InternalConfigurationData;
import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
import org.apache.pulsar.common.configuration.VipStatus;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterDataImpl;
import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl;
import org.apache.pulsar.common.protocol.schema.SchemaStorage;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.netty.EventLoopUtil;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.compaction.TwoPhaseCompactor;
import org.apache.pulsar.functions.worker.ErrorNotifier;
import org.apache.pulsar.functions.worker.WorkerConfig;
import org.apache.pulsar.functions.worker.WorkerService;
import org.apache.pulsar.metadata.api.MetadataStore;
import org.apache.pulsar.metadata.api.MetadataStoreConfig;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.metadata.api.MetadataStoreFactory;
import org.apache.pulsar.metadata.api.Notification;
import org.apache.pulsar.metadata.api.NotificationType;
import org.apache.pulsar.metadata.api.coordination.CoordinationService;
import org.apache.pulsar.metadata.api.coordination.LeaderElectionState;
import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
import org.apache.pulsar.metadata.api.extended.SessionEvent;
import org.apache.pulsar.metadata.coordination.impl.CoordinationServiceImpl;
import org.apache.pulsar.metadata.impl.ZKMetadataStore;
import org.apache.pulsar.packages.management.core.PackagesManagement;
import org.apache.pulsar.packages.management.core.PackagesStorage;
import org.apache.pulsar.packages.management.core.PackagesStorageProvider;
import org.apache.pulsar.packages.management.core.impl.DefaultPackagesStorageConfiguration;
import org.apache.pulsar.packages.management.core.impl.PackagesManagementImpl;
import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener;
import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreProvider;
import org.apache.pulsar.websocket.WebSocketConsumerServlet;
import org.apache.pulsar.websocket.WebSocketPingPongServlet;
import org.apache.pulsar.websocket.WebSocketProducerServlet;
import org.apache.pulsar.websocket.WebSocketReaderServlet;
import org.apache.pulsar.websocket.WebSocketService;
import org.apache.pulsar.zookeeper.ZooKeeperClientFactory;
import org.apache.pulsar.zookeeper.ZookeeperBkClientFactoryImpl;
import org.apache.zookeeper.ZooKeeper;
import org.eclipse.jetty.servlet.ServletHolder;
import org.eclipse.jetty.websocket.servlet.WebSocketServlet;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Main class for Pulsar broker service.
*/
@Getter(AccessLevel.PUBLIC)
@Setter(AccessLevel.PROTECTED)
public class PulsarService implements AutoCloseable, ShutdownService {
private static final Logger LOG = LoggerFactory.getLogger(PulsarService.class);
private static final double GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT = 0.5d;
private ServiceConfiguration config = null;
private NamespaceService nsService = null;
private ManagedLedgerStorage managedLedgerClientFactory = null;
private LeaderElectionService leaderElectionService = null;
private BrokerService brokerService = null;
private WebService webService = null;
private WebSocketService webSocketService = null;
private TopicPoliciesService topicPoliciesService = TopicPoliciesService.DISABLED;
private BookKeeperClientFactory bkClientFactory;
private Compactor compactor;
private ResourceUsageTransportManager resourceUsageTransportManager;
private ResourceGroupService resourceGroupServiceManager;
private final ScheduledExecutorService executor;
private final ScheduledExecutorService cacheExecutor;
private OrderedExecutor orderedExecutor;
private final ScheduledExecutorService loadManagerExecutor;
private ScheduledExecutorService compactorExecutor;
private OrderedScheduler offloaderScheduler;
private OffloadersCache offloadersCache = new OffloadersCache();
private LedgerOffloader defaultOffloader;
private Map<NamespaceName, LedgerOffloader> ledgerOffloaderMap = new ConcurrentHashMap<>();
private ScheduledFuture<?> loadReportTask = null;
private ScheduledFuture<?> loadSheddingTask = null;
private ScheduledFuture<?> loadResourceQuotaTask = null;
private final AtomicReference<LoadManager> loadManager = new AtomicReference<>();
private PulsarAdmin adminClient = null;
private PulsarClient client = null;
private ZooKeeperClientFactory zkClientFactory = null;
private final String bindAddress;
/**
* The host component of the broker's canonical name.
*/
private final String advertisedAddress;
private String webServiceAddress;
private String webServiceAddressTls;
private String brokerServiceUrl;
private String brokerServiceUrlTls;
private final String brokerVersion;
private SchemaStorage schemaStorage = null;
private SchemaRegistryService schemaRegistryService = null;
private final WorkerConfig workerConfig;
private final Optional<WorkerService> functionWorkerService;
private ProtocolHandlers protocolHandlers = null;
private final Consumer<Integer> processTerminator;
protected final EventLoopGroup ioEventLoopGroup;
private MetricsGenerator metricsGenerator;
private TransactionMetadataStoreService transactionMetadataStoreService;
private TransactionBufferProvider transactionBufferProvider;
private TransactionBufferClient transactionBufferClient;
private HashedWheelTimer transactionTimer;
private BrokerInterceptor brokerInterceptor;
private AdditionalServlets brokerAdditionalServlets;
// packages management service
private PackagesManagement packagesManagement;
private PrometheusMetricsServlet metricsServlet;
private List<PrometheusRawMetricsProvider> pendingMetricsProviders;
private MetadataStoreExtended localMetadataStore;
private CoordinationService coordinationService;
private TransactionBufferSnapshotService transactionBufferSnapshotService;
private MetadataStore configurationMetadataStore;
private boolean shouldShutdownConfigurationMetadataStore;
private PulsarResources pulsarResources;
private TransactionPendingAckStoreProvider transactionPendingAckStoreProvider;
private final ScheduledExecutorService transactionReplayExecutor;
public enum State {
Init, Started, Closing, Closed
}
private volatile State state;
private final ReentrantLock mutex = new ReentrantLock();
private final Condition isClosedCondition = mutex.newCondition();
private volatile CompletableFuture<Void> closeFuture;
// key is listener name , value is pulsar address and pulsar ssl address
private Map<String, AdvertisedListener> advertisedListeners;
public PulsarService(ServiceConfiguration config) {
this(config, Optional.empty(), (exitCode) -> {
LOG.info("Process termination requested with code {}. "
+ "Ignoring, as this constructor is intended for tests. ", exitCode);
});
}
public PulsarService(ServiceConfiguration config, Optional<WorkerService> functionWorkerService,
Consumer<Integer> processTerminator) {
this(config, new WorkerConfig(), functionWorkerService, processTerminator);
}
public PulsarService(ServiceConfiguration config,
WorkerConfig workerConfig,
Optional<WorkerService> functionWorkerService,
Consumer<Integer> processTerminator) {
state = State.Init;
// Validate correctness of configuration
PulsarConfigurationLoader.isComplete(config);
// validate `advertisedAddress`, `advertisedListeners`, `internalListenerName`
this.advertisedListeners = MultipleListenerValidator.validateAndAnalysisAdvertisedListener(config);
// the advertised address is defined as the host component of the broker's canonical name.
this.advertisedAddress = ServiceConfigurationUtils.getDefaultOrConfiguredAddress(config.getAdvertisedAddress());
// use `internalListenerName` listener as `advertisedAddress`
this.bindAddress = ServiceConfigurationUtils.getDefaultOrConfiguredAddress(config.getBindAddress());
this.brokerVersion = PulsarVersion.getVersion();
this.config = config;
this.processTerminator = processTerminator;
this.loadManagerExecutor = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-load-manager"));
this.workerConfig = workerConfig;
this.functionWorkerService = functionWorkerService;
this.executor = Executors.newScheduledThreadPool(config.getNumExecutorThreadPoolSize(),
new DefaultThreadFactory("pulsar"));
this.cacheExecutor = Executors.newScheduledThreadPool(config.getNumCacheExecutorThreadPoolSize(),
new DefaultThreadFactory("zk-cache-callback"));
if (config.isTransactionCoordinatorEnabled()) {
this.transactionReplayExecutor = Executors.newScheduledThreadPool(
config.getNumTransactionReplayThreadPoolSize(),
new DefaultThreadFactory("transaction-replay"));
} else {
this.transactionReplayExecutor = null;
}
this.ioEventLoopGroup = EventLoopUtil.newEventLoopGroup(config.getNumIOThreads(), config.isEnableBusyWait(),
new DefaultThreadFactory("pulsar-io"));
}
public MetadataStore createConfigurationMetadataStore() throws MetadataStoreException {
return MetadataStoreFactory.create(config.getConfigurationStoreServers(),
MetadataStoreConfig.builder()
.sessionTimeoutMillis((int) config.getZooKeeperSessionTimeoutMillis())
.allowReadOnlyOperations(false)
.build());
}
/**
* Close the session to the metadata service.
*
* This will immediately release all the resource locks held by this broker on the coordination service.
*
* @throws Exception if the close operation fails
*/
public void closeMetadataServiceSession() throws Exception {
localMetadataStore.close();
}
@Override
public void close() throws PulsarServerException {
try {
closeAsync().get();
} catch (ExecutionException e) {
Throwable cause = e.getCause();
if (cause instanceof PulsarServerException) {
throw (PulsarServerException) cause;
} else if (getConfiguration().getBrokerShutdownTimeoutMs() == 0
&& (cause instanceof TimeoutException || cause instanceof CancellationException)) {
// ignore shutdown timeout when timeout is 0, which is primarily used in tests
// to forcefully shutdown the broker
} else {
throw new PulsarServerException(cause);
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
/**
* Close the current pulsar service. All resources are released.
*/
public CompletableFuture<Void> closeAsync() {
mutex.lock();
try {
if (closeFuture != null) {
return closeFuture;
}
LOG.info("Closing PulsarService");
state = State.Closing;
// close the service in reverse order v.s. in which they are started
if (this.resourceUsageTransportManager != null) {
this.resourceUsageTransportManager.close();
this.resourceUsageTransportManager = null;
}
if (this.webService != null) {
try {
this.webService.close();
this.webService = null;
} catch (Exception e) {
LOG.error("Web service closing failed", e);
// Even if the web service fails to close, the graceful shutdown process continues
}
}
metricsServlet = null;
if (this.webSocketService != null) {
this.webSocketService.close();
}
if (brokerAdditionalServlets != null) {
brokerAdditionalServlets.close();
brokerAdditionalServlets = null;
}
GracefulExecutorServicesShutdown executorServicesShutdown =
GracefulExecutorServicesShutdown
.initiate()
.timeout(
Duration.ofMillis(
(long) (GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT
* getConfiguration()
.getBrokerShutdownTimeoutMs())));
// shutdown loadmanager before shutting down the broker
executorServicesShutdown.shutdown(loadManagerExecutor);
LoadManager loadManager = this.loadManager.get();
if (loadManager != null) {
loadManager.stop();
}
List<CompletableFuture<Void>> asyncCloseFutures = new ArrayList<>();
if (this.brokerService != null) {
asyncCloseFutures.add(this.brokerService.closeAsync());
this.brokerService = null;
}
if (this.managedLedgerClientFactory != null) {
this.managedLedgerClientFactory.close();
this.managedLedgerClientFactory = null;
}
if (bkClientFactory != null) {
this.bkClientFactory.close();
this.bkClientFactory = null;
}
if (this.leaderElectionService != null) {
this.leaderElectionService.close();
this.leaderElectionService = null;
}
if (adminClient != null) {
adminClient.close();
adminClient = null;
}
if (transactionBufferSnapshotService != null) {
transactionBufferSnapshotService.close();
transactionBufferSnapshotService = null;
}
if (client != null) {
client.close();
client = null;
}
if (nsService != null) {
nsService.close();
nsService = null;
}
executorServicesShutdown.shutdown(compactorExecutor);
executorServicesShutdown.shutdown(offloaderScheduler);
executorServicesShutdown.shutdown(executor);
executorServicesShutdown.shutdown(orderedExecutor);
executorServicesShutdown.shutdown(cacheExecutor);
if (schemaRegistryService != null) {
schemaRegistryService.close();
}
offloadersCache.close();
if (protocolHandlers != null) {
protocolHandlers.close();
protocolHandlers = null;
}
if (transactionBufferClient != null) {
transactionBufferClient.close();
}
if (coordinationService != null) {
coordinationService.close();
}
if (localMetadataStore != null) {
localMetadataStore.close();
}
if (configurationMetadataStore != null && shouldShutdownConfigurationMetadataStore) {
configurationMetadataStore.close();
}
if (transactionReplayExecutor != null) {
transactionReplayExecutor.shutdown();
}
ioEventLoopGroup.shutdownGracefully();
// add timeout handling for closing executors
asyncCloseFutures.add(executorServicesShutdown.handle());
closeFuture = addTimeoutHandling(FutureUtil.waitForAllAndSupportCancel(asyncCloseFutures));
closeFuture.handle((v, t) -> {
if (t == null) {
LOG.info("Closed");
} else if (t instanceof CancellationException) {
LOG.info("Closed (shutdown cancelled)");
} else if (t instanceof TimeoutException) {
LOG.info("Closed (shutdown timeout)");
} else {
LOG.warn("Closed with errors", t);
}
state = State.Closed;
isClosedCondition.signalAll();
return null;
});
return closeFuture;
} catch (Exception e) {
PulsarServerException pse;
if (e instanceof CompletionException && e.getCause() instanceof MetadataStoreException) {
pse = new PulsarServerException(MetadataStoreException.unwrap(e));
} else if (e.getCause() instanceof CompletionException
&& e.getCause().getCause() instanceof MetadataStoreException) {
pse = new PulsarServerException(MetadataStoreException.unwrap(e.getCause()));
} else {
pse = new PulsarServerException(e);
}
return FutureUtil.failedFuture(pse);
} finally {
mutex.unlock();
}
}
private CompletableFuture<Void> addTimeoutHandling(CompletableFuture<Void> future) {
ScheduledExecutorService shutdownExecutor = Executors.newSingleThreadScheduledExecutor(
new DefaultThreadFactory(getClass().getSimpleName() + "-shutdown"));
FutureUtil.addTimeoutHandling(future,
Duration.ofMillis(Math.max(1L, getConfiguration().getBrokerShutdownTimeoutMs())),
shutdownExecutor, () -> FutureUtil.createTimeoutException("Timeout in close", getClass(), "close"));
future.handle((v, t) -> {
// shutdown the shutdown executor
shutdownExecutor.shutdownNow();
return null;
});
return future;
}
/**
* Get the current service configuration.
*
* @return the current service configuration
*/
public ServiceConfiguration getConfiguration() {
return this.config;
}
/**
* Get the current function worker service configuration.
*
* @return the current function worker service configuration.
*/
public Optional<WorkerConfig> getWorkerConfig() {
return functionWorkerService.map(service -> workerConfig);
}
public Map<String, String> getProtocolDataToAdvertise() {
if (null == protocolHandlers) {
return Collections.emptyMap();
} else {
return protocolHandlers.getProtocolDataToAdvertise();
}
}
/**
* Start the pulsar service instance.
*/
public void start() throws PulsarServerException {
LOG.info("Starting Pulsar Broker service; version: '{}'",
(brokerVersion != null ? brokerVersion : "unknown"));
LOG.info("Git Revision {}", PulsarVersion.getGitSha());
LOG.info("Git Branch {}", PulsarVersion.getGitBranch());
LOG.info("Built by {} on {} at {}",
PulsarVersion.getBuildUser(),
PulsarVersion.getBuildHost(),
PulsarVersion.getBuildTime());
long startTimestamp = System.currentTimeMillis(); // start time mills
mutex.lock();
try {
if (state != State.Init) {
throw new PulsarServerException("Cannot start the service once it was stopped");
}
if (!config.getWebServicePort().isPresent() && !config.getWebServicePortTls().isPresent()) {
throw new IllegalArgumentException("webServicePort/webServicePortTls must be present");
}
if (config.isAuthorizationEnabled() && !config.isAuthenticationEnabled()) {
throw new IllegalStateException("Invalid broker configuration. Authentication must be enabled with "
+ "authenticationEnabled=true when authorization is enabled with authorizationEnabled=true.");
}
localMetadataStore = createLocalMetadataStore();
localMetadataStore.registerSessionListener(this::handleMetadataSessionEvent);
coordinationService = new CoordinationServiceImpl(localMetadataStore);
if (!StringUtils.equals(config.getConfigurationStoreServers(), config.getZookeeperServers())) {
configurationMetadataStore = createConfigurationMetadataStore();
shouldShutdownConfigurationMetadataStore = true;
} else {
configurationMetadataStore = localMetadataStore;
shouldShutdownConfigurationMetadataStore = false;
}
pulsarResources = new PulsarResources(localMetadataStore, configurationMetadataStore,
config.getZooKeeperOperationTimeoutSeconds());
pulsarResources.getClusterResources().getStore().registerListener(this::handleDeleteCluster);
orderedExecutor = OrderedExecutor.newBuilder()
.numThreads(config.getNumOrderedExecutorThreads())
.name("pulsar-ordered")
.build();
// Initialize the message protocol handlers
protocolHandlers = ProtocolHandlers.load(config);
protocolHandlers.initialize(config);
// Now we are ready to start services
this.bkClientFactory = newBookKeeperClientFactory();
managedLedgerClientFactory = ManagedLedgerStorage.create(
config, localMetadataStore, getZkClient(),
bkClientFactory, ioEventLoopGroup
);
this.brokerService = newBrokerService(this);
// Start load management service (even if load balancing is disabled)
this.loadManager.set(LoadManager.create(this));
// needs load management service and before start broker service,
this.startNamespaceService();
schemaStorage = createAndStartSchemaStorage();
schemaRegistryService = SchemaRegistryService.create(
schemaStorage, config.getSchemaRegistryCompatibilityCheckers());
this.defaultOffloader = createManagedLedgerOffloader(
OffloadPoliciesImpl.create(this.getConfiguration().getProperties()));
this.brokerInterceptor = BrokerInterceptors.load(config);
brokerService.setInterceptor(getBrokerInterceptor());
this.brokerInterceptor.initialize(this);
brokerService.start();
// Load additional servlets
this.brokerAdditionalServlets = AdditionalServlets.load(config);
this.webService = new WebService(this);
this.metricsServlet = new PrometheusMetricsServlet(
this, config.isExposeTopicLevelMetricsInPrometheus(),
config.isExposeConsumerLevelMetricsInPrometheus(),
config.isExposeProducerLevelMetricsInPrometheus(),
config.isSplitTopicAndPartitionLabelInPrometheus());
if (pendingMetricsProviders != null) {
pendingMetricsProviders.forEach(provider -> metricsServlet.addRawMetricsProvider(provider));
this.pendingMetricsProviders = null;
}
this.addWebServerHandlers(webService, metricsServlet, this.config);
this.webService.start();
// Refresh addresses and update configuration, since the port might have been dynamically assigned
if (config.getBrokerServicePort().equals(Optional.of(0))) {
config.setBrokerServicePort(brokerService.getListenPort());
}
if (config.getBrokerServicePortTls().equals(Optional.of(0))) {
config.setBrokerServicePortTls(brokerService.getListenPortTls());
}
this.webServiceAddress = webAddress(config);
this.webServiceAddressTls = webAddressTls(config);
this.brokerServiceUrl = brokerUrl(config);
this.brokerServiceUrlTls = brokerUrlTls(config);
if (null != this.webSocketService) {
ClusterDataImpl clusterData = ClusterDataImpl.builder()
.serviceUrl(webServiceAddress)
.serviceUrlTls(webServiceAddressTls)
.brokerServiceUrl(brokerServiceUrl)
.brokerServiceUrlTls(brokerServiceUrlTls)
.build();
this.webSocketService.setLocalCluster(clusterData);
}
// Initialize namespace service, after service url assigned. Should init zk and refresh self owner info.
this.nsService.initialize();
// Start topic level policies service
if (config.isTopicLevelPoliciesEnabled() && config.isSystemTopicEnabled()) {
this.topicPoliciesService = new SystemTopicBasedTopicPoliciesService(this);
}
this.topicPoliciesService.start();
// Start the leader election service
startLeaderElectionService();
// Register heartbeat and bootstrap namespaces.
this.nsService.registerBootstrapNamespaces();
// Register pulsar system namespaces and start transaction meta store service
if (config.isTransactionCoordinatorEnabled()) {
this.transactionBufferSnapshotService = new SystemTopicBaseTxnBufferSnapshotService(getClient());
this.transactionTimer =
new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer"));
transactionBufferClient = TransactionBufferClientImpl.create(getClient(), transactionTimer);
transactionMetadataStoreService = new TransactionMetadataStoreService(TransactionMetadataStoreProvider
.newProvider(config.getTransactionMetadataStoreProviderClassName()), this,
transactionBufferClient, transactionTimer);
transactionBufferProvider = TransactionBufferProvider
.newProvider(config.getTransactionBufferProviderClassName());
transactionPendingAckStoreProvider = TransactionPendingAckStoreProvider
.newProvider(config.getTransactionPendingAckStoreProviderClassName());
}
this.metricsGenerator = new MetricsGenerator(this);
// By starting the Load manager service, the broker will also become visible
// to the rest of the broker by creating the registration z-node. This needs
// to be done only when the broker is fully operative.
this.startLoadManagementService();
// Initialize the message protocol handlers.
// start the protocol handlers only after the broker is ready,
// so that the protocol handlers can access broker service properly.
this.protocolHandlers.start(brokerService);
Map<String, Map<InetSocketAddress, ChannelInitializer<SocketChannel>>> protocolHandlerChannelInitializers =
this.protocolHandlers.newChannelInitializers();
this.brokerService.startProtocolHandlers(protocolHandlerChannelInitializers);
acquireSLANamespace();
// start function worker service if necessary
this.startWorkerService(brokerService.getAuthenticationService(), brokerService.getAuthorizationService());
// start packages management service if necessary
if (config.isEnablePackagesManagement()) {
this.startPackagesManagementService();
}
// Start the task to publish resource usage, if necessary
this.resourceUsageTransportManager = DISABLE_RESOURCE_USAGE_TRANSPORT_MANAGER;
if (isNotBlank(config.getResourceUsageTransportClassName())) {
Class<?> clazz = Class.forName(config.getResourceUsageTransportClassName());
Constructor<?> ctor = clazz.getConstructor(PulsarService.class);
Object object = ctor.newInstance(new Object[]{this});
this.resourceUsageTransportManager = (ResourceUsageTopicTransportManager) object;
}
this.resourceGroupServiceManager = new ResourceGroupService(this);
long currentTimestamp = System.currentTimeMillis();
final long bootstrapTimeSeconds = TimeUnit.MILLISECONDS.toSeconds(currentTimestamp - startTimestamp);
final String bootstrapMessage = "bootstrap service "
+ (config.getWebServicePort().isPresent() ? "port = " + config.getWebServicePort().get() : "")
+ (config.getWebServicePortTls().isPresent() ? ", tls-port = " + config.getWebServicePortTls() : "")
+ (StringUtils.isNotEmpty(brokerServiceUrl) ? ", broker url= " + brokerServiceUrl : "")
+ (StringUtils.isNotEmpty(brokerServiceUrlTls) ? ", broker tls url= " + brokerServiceUrlTls : "");
LOG.info("messaging service is ready, bootstrap_seconds={}", bootstrapTimeSeconds);
LOG.info("messaging service is ready, {}, cluster={}, configs={}", bootstrapMessage,
config.getClusterName(), ReflectionToStringBuilder.toString(config));
state = State.Started;
} catch (Exception e) {
LOG.error("Failed to start Pulsar service: {}", e.getMessage(), e);
throw new PulsarServerException(e);
} finally {
mutex.unlock();
}
}
private void addWebServerHandlers(WebService webService,
PrometheusMetricsServlet metricsServlet,
ServiceConfiguration config)
throws PulsarServerException, PulsarClientException, MalformedURLException, ServletException,
DeploymentException {
Map<String, Object> attributeMap = Maps.newHashMap();
attributeMap.put(WebService.ATTRIBUTE_PULSAR_NAME, this);
Map<String, Object> vipAttributeMap = Maps.newHashMap();
vipAttributeMap.put(VipStatus.ATTRIBUTE_STATUS_FILE_PATH, config.getStatusFilePath());
vipAttributeMap.put(VipStatus.ATTRIBUTE_IS_READY_PROBE, (Supplier<Boolean>) () -> {
// Ensure the VIP status is only visible when the broker is fully initialized
return state == State.Started;
});
// Add admin rest resources
webService.addRestResources("/",
VipStatus.class.getPackage().getName(), false, vipAttributeMap);
webService.addRestResources("/",
"org.apache.pulsar.broker.web", false, attributeMap);
webService.addRestResources("/admin",
"org.apache.pulsar.broker.admin.v1", true, attributeMap);
webService.addRestResources("/admin/v2",
"org.apache.pulsar.broker.admin.v2", true, attributeMap);
webService.addRestResources("/admin/v3",
"org.apache.pulsar.broker.admin.v3", true, attributeMap);
webService.addRestResources("/lookup",
"org.apache.pulsar.broker.lookup", true, attributeMap);
webService.addRestResources("/topics",
"org.apache.pulsar.broker.rest", true, attributeMap);
// Add metrics servlet
webService.addServlet("/metrics",
new ServletHolder(metricsServlet),
false, attributeMap);
// Add websocket service
addWebSocketServiceHandler(webService, attributeMap, config);
if (LOG.isDebugEnabled()) {
LOG.debug("Attempting to add static directory");
}
// Add static resources
webService.addStaticResources("/static", "/static");
// Add broker additional servlets
addBrokerAdditionalServlets(webService, attributeMap, config);
}
private void handleMetadataSessionEvent(SessionEvent e) {
LOG.info("Received metadata service session event: {}", e);
if (e == SessionEvent.SessionLost
&& config.getZookeeperSessionExpiredPolicy() == MetadataSessionExpiredPolicy.shutdown) {
LOG.warn("The session with metadata service was lost. Shutting down.");
shutdownNow();
}
}
private void addBrokerAdditionalServlets(WebService webService,
Map<String, Object> attributeMap,
ServiceConfiguration config) {
if (this.getBrokerAdditionalServlets() != null) {
Collection<AdditionalServletWithClassLoader> additionalServletCollection =
this.getBrokerAdditionalServlets().getServlets().values();
for (AdditionalServletWithClassLoader servletWithClassLoader : additionalServletCollection) {
servletWithClassLoader.loadConfig(config);
AdditionalServlet additionalServlet = servletWithClassLoader.getServlet();
if (additionalServlet instanceof AdditionalServletWithPulsarService) {
((AdditionalServletWithPulsarService) additionalServlet).setPulsarService(this);
}
webService.addServlet(servletWithClassLoader.getBasePath(), servletWithClassLoader.getServletHolder(),
config.isAuthenticationEnabled(), attributeMap);
LOG.info("Broker add additional servlet basePath {} ", servletWithClassLoader.getBasePath());
}
}
}
private void addWebSocketServiceHandler(WebService webService,
Map<String, Object> attributeMap,
ServiceConfiguration config)
throws PulsarServerException, PulsarClientException, MalformedURLException, ServletException,
DeploymentException {
if (config.isWebSocketServiceEnabled()) {
// Use local broker address to avoid different IP address when using a VIP for service discovery
this.webSocketService = new WebSocketService(null, config);
this.webSocketService.start();
final WebSocketServlet producerWebSocketServlet = new WebSocketProducerServlet(webSocketService);
webService.addServlet(WebSocketProducerServlet.SERVLET_PATH,
new ServletHolder(producerWebSocketServlet), true, attributeMap);
webService.addServlet(WebSocketProducerServlet.SERVLET_PATH_V2,
new ServletHolder(producerWebSocketServlet), true, attributeMap);
final WebSocketServlet consumerWebSocketServlet = new WebSocketConsumerServlet(webSocketService);
webService.addServlet(WebSocketConsumerServlet.SERVLET_PATH,
new ServletHolder(consumerWebSocketServlet), true, attributeMap);
webService.addServlet(WebSocketConsumerServlet.SERVLET_PATH_V2,
new ServletHolder(consumerWebSocketServlet), true, attributeMap);
final WebSocketServlet readerWebSocketServlet = new WebSocketReaderServlet(webSocketService);
webService.addServlet(WebSocketReaderServlet.SERVLET_PATH,
new ServletHolder(readerWebSocketServlet), true, attributeMap);
webService.addServlet(WebSocketReaderServlet.SERVLET_PATH_V2,
new ServletHolder(readerWebSocketServlet), true, attributeMap);
final WebSocketServlet pingPongWebSocketServlet = new WebSocketPingPongServlet(webSocketService);
webService.addServlet(WebSocketPingPongServlet.SERVLET_PATH,
new ServletHolder(pingPongWebSocketServlet), true, attributeMap);
webService.addServlet(WebSocketPingPongServlet.SERVLET_PATH_V2,
new ServletHolder(pingPongWebSocketServlet), true, attributeMap);
}
}
private void handleDeleteCluster(Notification notification) {
if (ClusterResources.pathRepresentsClusterName(notification.getPath())
&& notification.getType() == NotificationType.Deleted) {
final String clusterName = ClusterResources.clusterNameFromPath(notification.getPath());
getBrokerService().closeAndRemoveReplicationClient(clusterName);
}
}
public MetadataStoreExtended createLocalMetadataStore() throws MetadataStoreException {
return MetadataStoreExtended.create(config.getZookeeperServers(),
MetadataStoreConfig.builder()
.sessionTimeoutMillis((int) config.getZooKeeperSessionTimeoutMillis())
.allowReadOnlyOperations(false)
.build());
}
protected void startLeaderElectionService() {
this.leaderElectionService = new LeaderElectionService(coordinationService, getSafeWebServiceAddress(),
state -> {
if (state == LeaderElectionState.Leading) {
LOG.info("This broker was elected leader");
if (getConfiguration().isLoadBalancerEnabled()) {
long loadSheddingInterval = TimeUnit.MINUTES
.toMillis(getConfiguration().getLoadBalancerSheddingIntervalMinutes());
long resourceQuotaUpdateInterval = TimeUnit.MINUTES
.toMillis(getConfiguration().getLoadBalancerResourceQuotaUpdateIntervalMinutes());
if (loadSheddingTask != null) {
loadSheddingTask.cancel(false);
}
if (loadResourceQuotaTask != null) {
loadResourceQuotaTask.cancel(false);
}
loadSheddingTask = loadManagerExecutor.scheduleAtFixedRate(
new LoadSheddingTask(loadManager),
loadSheddingInterval, loadSheddingInterval, TimeUnit.MILLISECONDS);
loadResourceQuotaTask = loadManagerExecutor.scheduleAtFixedRate(
new LoadResourceQuotaUpdaterTask(loadManager), resourceQuotaUpdateInterval,
resourceQuotaUpdateInterval, TimeUnit.MILLISECONDS);
}
} else {
if (leaderElectionService != null) {
LOG.info("This broker is a follower. Current leader is {}",
leaderElectionService.getCurrentLeader());
}
if (loadSheddingTask != null) {
loadSheddingTask.cancel(false);
loadSheddingTask = null;
}
if (loadResourceQuotaTask != null) {
loadResourceQuotaTask.cancel(false);
loadResourceQuotaTask = null;
}
}
});
leaderElectionService.start();
}
protected void acquireSLANamespace() {
try {
// Namespace not created hence no need to unload it
NamespaceName nsName = NamespaceService.getSLAMonitorNamespace(getAdvertisedAddress(), config);
if (!this.pulsarResources.getNamespaceResources().namespaceExists(nsName)) {
LOG.info("SLA Namespace = {} doesn't exist.", nsName);
return;
}
boolean acquiredSLANamespace;
try {
acquiredSLANamespace = nsService.registerSLANamespace();
LOG.info("Register SLA Namespace = {}, returned - {}.", nsName, acquiredSLANamespace);
} catch (PulsarServerException e) {
acquiredSLANamespace = false;
}
if (!acquiredSLANamespace) {
this.nsService.unloadSLANamespace();
}
} catch (Exception ex) {
LOG.warn(
"Exception while trying to unload the SLA namespace,"
+ " will try to unload the namespace again after 1 minute. Exception:",
ex);
executor.schedule(this::acquireSLANamespace, 1, TimeUnit.MINUTES);
} catch (Throwable ex) {
// To make sure SLA monitor doesn't interfere with the normal broker flow
LOG.warn(
"Exception while trying to unload the SLA namespace,"
+ " will not try to unload the namespace again. Exception:",
ex);
}
}
/**
* Block until the service is finally closed.
*/
public void waitUntilClosed() throws InterruptedException {
mutex.lock();