forked from apache/pulsar
/
PersistentTopic.java
3263 lines (2895 loc) · 154 KB
/
PersistentTopic.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.persistent;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.commons.lang3.StringUtils.isBlank;
import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
import static org.apache.pulsar.common.events.EventsTopicNames.checkTopicIsEventsNames;
import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION;
import com.carrotsearch.hppc.ObjectObjectHashMap;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import io.netty.buffer.ByteBuf;
import io.netty.util.concurrent.FastThreadLocal;
import java.util.ArrayList;
import java.util.Collections;
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.CompletionException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCursorCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.OffloadCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenCursorCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.TerminateCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.UpdatePropertiesCallback;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerAlreadyClosedException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerFencedException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerTerminatedException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.bookkeeper.net.BookieId;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.broker.resources.NamespaceResources.PartitionedTopicResources;
import org.apache.pulsar.broker.service.AbstractTopic;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.BrokerServiceException.AlreadyRunningException;
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException;
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.ServerMetadataException;
import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException;
import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionNotFoundException;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicClosedException;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicFencedException;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicTerminatedException;
import org.apache.pulsar.broker.service.BrokerServiceException.UnsupportedVersionException;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.Dispatcher;
import org.apache.pulsar.broker.service.Producer;
import org.apache.pulsar.broker.service.Replicator;
import org.apache.pulsar.broker.service.StreamingStats;
import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.TopicPolicyListener;
import org.apache.pulsar.broker.service.TransportCnx;
import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type;
import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage;
import org.apache.pulsar.broker.stats.ClusterReplicationMetrics;
import org.apache.pulsar.broker.stats.NamespaceStats;
import org.apache.pulsar.broker.stats.ReplicationMetrics;
import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer;
import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferDisable;
import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore;
import org.apache.pulsar.client.admin.LongRunningProcessStatus;
import org.apache.pulsar.client.admin.OffloadProcessStatus;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.transaction.TxnID;
import org.apache.pulsar.client.impl.BatchMessageIdImpl;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.client.impl.MessageImpl;
import org.apache.pulsar.common.api.proto.CommandSubscribe;
import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition;
import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType;
import org.apache.pulsar.common.api.proto.KeySharedMeta;
import org.apache.pulsar.common.api.proto.MessageMetadata;
import org.apache.pulsar.common.api.proto.TxnAction;
import org.apache.pulsar.common.events.EventsTopicNames;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.BacklogQuota;
import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode;
import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.CursorStats;
import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.LedgerInfo;
import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.policies.data.RetentionPolicies;
import org.apache.pulsar.common.policies.data.TopicPolicies;
import org.apache.pulsar.common.policies.data.TransactionBufferStats;
import org.apache.pulsar.common.policies.data.TransactionInBufferStats;
import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats;
import org.apache.pulsar.common.policies.data.TransactionPendingAckStats;
import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl;
import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl;
import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl;
import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl;
import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.schema.SchemaData;
import org.apache.pulsar.common.protocol.schema.SchemaVersion;
import org.apache.pulsar.common.util.Codec;
import org.apache.pulsar.common.util.DateFormatter;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.apache.pulsar.compaction.CompactedTopic;
import org.apache.pulsar.compaction.CompactedTopicContext;
import org.apache.pulsar.compaction.CompactedTopicImpl;
import org.apache.pulsar.compaction.CompactionRecord;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.compaction.CompactorMXBean;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats;
import org.apache.pulsar.utils.StatsOutputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class PersistentTopic extends AbstractTopic
implements Topic, AddEntryCallback, TopicPolicyListener<TopicPolicies> {
// Managed ledger associated with the topic
protected final ManagedLedger ledger;
// Subscriptions to this topic
private final ConcurrentOpenHashMap<String, PersistentSubscription> subscriptions;
private final ConcurrentOpenHashMap<String, Replicator> replicators;
static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup";
private static final String TOPIC_EPOCH_PROPERTY_NAME = "pulsar.topic.epoch";
private static final double MESSAGE_EXPIRY_THRESHOLD = 1.5;
private static final long POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS = 60;
// topic has every published chunked message since topic is loaded
public boolean msgChunkPublished;
private Optional<DispatchRateLimiter> dispatchRateLimiter = Optional.empty();
private Optional<SubscribeRateLimiter> subscribeRateLimiter = Optional.empty();
public volatile long delayedDeliveryTickTimeMillis = 1000;
private final long backloggedCursorThresholdEntries;
public volatile boolean delayedDeliveryEnabled = false;
public static final int MESSAGE_RATE_BACKOFF_MS = 1000;
protected final MessageDeduplication messageDeduplication;
private static final long COMPACTION_NEVER_RUN = -0xfebecffeL;
private CompletableFuture<Long> currentCompaction = CompletableFuture.completedFuture(COMPACTION_NEVER_RUN);
private final CompactedTopic compactedTopic;
private CompletableFuture<MessageIdImpl> currentOffload = CompletableFuture.completedFuture(
(MessageIdImpl) MessageId.earliest);
private volatile Optional<ReplicatedSubscriptionsController> replicatedSubscriptionsController = Optional.empty();
private static final FastThreadLocal<TopicStatsHelper> threadLocalTopicStats =
new FastThreadLocal<TopicStatsHelper>() {
@Override
protected TopicStatsHelper initialValue() {
return new TopicStatsHelper();
}
};
private final AtomicLong pendingWriteOps = new AtomicLong(0);
private volatile double lastUpdatedAvgPublishRateInMsg = 0;
private volatile double lastUpdatedAvgPublishRateInByte = 0;
private volatile int maxUnackedMessagesOnSubscriptionApplied;
private volatile boolean isClosingOrDeleting = false;
private ScheduledFuture<?> fencedTopicMonitoringTask = null;
// this future is for publish txn message in order.
private volatile CompletableFuture<Void> transactionCompletableFuture;
protected final TransactionBuffer transactionBuffer;
private final LongAdder bytesOutFromRemovedSubscriptions = new LongAdder();
private final LongAdder msgOutFromRemovedSubscriptions = new LongAdder();
private static class TopicStatsHelper {
public double averageMsgSize;
public double aggMsgRateIn;
public double aggMsgThroughputIn;
public double aggMsgThrottlingFailure;
public double aggMsgRateOut;
public double aggMsgThroughputOut;
public final ObjectObjectHashMap<String, PublisherStatsImpl> remotePublishersStats;
public TopicStatsHelper() {
remotePublishersStats = new ObjectObjectHashMap<>();
reset();
}
public void reset() {
averageMsgSize = 0;
aggMsgRateIn = 0;
aggMsgThroughputIn = 0;
aggMsgRateOut = 0;
aggMsgThrottlingFailure = 0;
aggMsgThroughputOut = 0;
remotePublishersStats.clear();
}
}
public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerService) throws NamingException {
super(topic, brokerService);
this.ledger = ledger;
this.subscriptions = new ConcurrentOpenHashMap<>(16, 1);
this.replicators = new ConcurrentOpenHashMap<>(16, 1);
this.delayedDeliveryEnabled = brokerService.pulsar().getConfiguration().isDelayedDeliveryEnabled();
this.delayedDeliveryTickTimeMillis =
brokerService.pulsar().getConfiguration().getDelayedDeliveryTickTimeMillis();
this.backloggedCursorThresholdEntries =
brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold();
this.transactionCompletableFuture = new CompletableFuture<>();
initializeRateLimiterIfNeeded(Optional.empty());
registerTopicPolicyListener();
this.compactedTopic = new CompactedTopicImpl(brokerService.pulsar().getBookKeeperClient());
for (ManagedCursor cursor : ledger.getCursors()) {
if (cursor.getName().startsWith(replicatorPrefix)) {
String localCluster = brokerService.pulsar().getConfiguration().getClusterName();
String remoteCluster = PersistentReplicator.getRemoteCluster(cursor.getName());
boolean isReplicatorStarted = false;
try {
isReplicatorStarted = addReplicationCluster(remoteCluster, cursor, localCluster);
} catch (Exception e) {
log.warn("[{}] failed to start replication", topic, e);
}
if (!isReplicatorStarted) {
throw new NamingException(
PersistentTopic.this.getName() + " Failed to start replicator " + remoteCluster);
}
} else if (cursor.getName().equals(DEDUPLICATION_CURSOR_NAME)) {
// This is not a regular subscription, we are going to
// ignore it for now and let the message dedup logic to take care of it
} else {
final String subscriptionName = Codec.decode(cursor.getName());
subscriptions.put(subscriptionName, createPersistentSubscription(subscriptionName, cursor,
PersistentSubscription.isCursorFromReplicatedSubscription(cursor)));
// subscription-cursor gets activated by default: deactivate as there is no active subscription right
// now
subscriptions.get(subscriptionName).deactivateCursor();
}
}
this.messageDeduplication = new MessageDeduplication(brokerService.pulsar(), this, ledger);
if (ledger.getProperties().containsKey(TOPIC_EPOCH_PROPERTY_NAME)) {
topicEpoch = Optional.of(Long.parseLong(ledger.getProperties().get(TOPIC_EPOCH_PROPERTY_NAME)));
}
checkReplicatedSubscriptionControllerState();
TopicName topicName = TopicName.get(topic);
if (brokerService.getPulsar().getConfiguration().isTransactionCoordinatorEnabled()
&& !checkTopicIsEventsNames(topicName)) {
this.transactionBuffer = brokerService.getPulsar()
.getTransactionBufferProvider().newTransactionBuffer(this, transactionCompletableFuture);
} else {
this.transactionCompletableFuture.complete(null);
this.transactionBuffer = new TransactionBufferDisable();
}
transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry());
}
@Override
public CompletableFuture<Void> initialize() {
return brokerService.pulsar().getPulsarResources().getNamespaceResources()
.getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace()))
.thenAccept(optPolicies -> {
if (!optPolicies.isPresent()) {
isEncryptionRequired = false;
updateUnackedMessagesAppliedOnSubscription(null);
updateUnackedMessagesExceededOnConsumer(null);
return;
}
Policies policies = optPolicies.get();
this.isEncryptionRequired = policies.encryption_required;
setSchemaCompatibilityStrategy(policies);
isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema;
schemaValidationEnforced = policies.schema_validation_enforced;
if (policies.inactive_topic_policies != null) {
inactiveTopicPolicies = policies.inactive_topic_policies;
}
updateUnackedMessagesAppliedOnSubscription(policies);
updateUnackedMessagesExceededOnConsumer(policies);
}).exceptionally(ex -> {
log.warn("[{}] Error getting policies {} and isEncryptionRequired will be set to false",
topic, ex.getMessage());
isEncryptionRequired = false;
updateUnackedMessagesAppliedOnSubscription(null);
updateUnackedMessagesExceededOnConsumer(null);
return null;
});
}
// for testing purposes
@VisibleForTesting
PersistentTopic(String topic, BrokerService brokerService, ManagedLedger ledger,
MessageDeduplication messageDeduplication) {
super(topic, brokerService);
this.ledger = ledger;
this.messageDeduplication = messageDeduplication;
this.subscriptions = new ConcurrentOpenHashMap<>(16, 1);
this.replicators = new ConcurrentOpenHashMap<>(16, 1);
this.compactedTopic = new CompactedTopicImpl(brokerService.pulsar().getBookKeeperClient());
this.backloggedCursorThresholdEntries =
brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold();
this.transactionCompletableFuture = new CompletableFuture<>();
if (brokerService.pulsar().getConfiguration().isTransactionCoordinatorEnabled()) {
this.transactionBuffer = brokerService.getPulsar()
.getTransactionBufferProvider().newTransactionBuffer(this, transactionCompletableFuture);
} else {
this.transactionCompletableFuture.complete(null);
this.transactionBuffer = new TransactionBufferDisable();
}
}
private void initializeRateLimiterIfNeeded(Optional<Policies> policies) {
synchronized (dispatchRateLimiter) {
// dispatch rate limiter for topic
if (!dispatchRateLimiter.isPresent() && DispatchRateLimiter
.isDispatchRateNeeded(brokerService, policies, topic, Type.TOPIC)) {
this.dispatchRateLimiter = Optional.of(new DispatchRateLimiter(this, Type.TOPIC));
}
boolean isDispatchRateNeeded = SubscribeRateLimiter.isDispatchRateNeeded(brokerService, policies, topic);
if (!subscribeRateLimiter.isPresent() && isDispatchRateNeeded) {
this.subscribeRateLimiter = Optional.of(new SubscribeRateLimiter(this));
} else if (!isDispatchRateNeeded) {
this.subscribeRateLimiter = Optional.empty();
}
// dispatch rate limiter for each subscription
subscriptions.forEach((name, subscription) -> {
Dispatcher dispatcher = subscription.getDispatcher();
if (dispatcher != null) {
dispatcher.initializeDispatchRateLimiterIfNeeded(policies);
}
});
// dispatch rate limiter for each replicator
replicators.forEach((name, replicator) ->
replicator.initializeDispatchRateLimiterIfNeeded(policies));
}
}
private PersistentSubscription createPersistentSubscription(String subscriptionName, ManagedCursor cursor,
boolean replicated) {
checkNotNull(compactedTopic);
if (subscriptionName.equals(COMPACTION_SUBSCRIPTION)) {
return new CompactorSubscription(this, compactedTopic, subscriptionName, cursor);
} else {
return new PersistentSubscription(this, subscriptionName, cursor, replicated);
}
}
@Override
public void publishMessage(ByteBuf headersAndPayload, PublishContext publishContext) {
pendingWriteOps.incrementAndGet();
if (isFenced) {
publishContext.completed(new TopicFencedException("fenced"), -1, -1);
decrementPendingWriteOpsAndCheck();
return;
}
if (isExceedMaximumMessageSize(headersAndPayload.readableBytes())) {
publishContext.completed(new NotAllowedException("Exceed maximum message size")
, -1, -1);
decrementPendingWriteOpsAndCheck();
return;
}
MessageDeduplication.MessageDupStatus status =
messageDeduplication.isDuplicate(publishContext, headersAndPayload);
switch (status) {
case NotDup:
asyncAddEntry(headersAndPayload, publishContext);
break;
case Dup:
// Immediately acknowledge duplicated message
publishContext.completed(null, -1, -1);
decrementPendingWriteOpsAndCheck();
break;
default:
publishContext.completed(new MessageDeduplication.MessageDupUnknownException(), -1, -1);
decrementPendingWriteOpsAndCheck();
}
}
private void asyncAddEntry(ByteBuf headersAndPayload, PublishContext publishContext) {
if (brokerService.isBrokerEntryMetadataEnabled()) {
ledger.asyncAddEntry(headersAndPayload,
(int) publishContext.getNumberOfMessages(), this, publishContext);
} else {
ledger.asyncAddEntry(headersAndPayload, this, publishContext);
}
}
public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) {
if (ledger instanceof ManagedLedgerImpl) {
((ManagedLedgerImpl) ledger).asyncReadEntry(position, callback, ctx);
} else {
callback.readEntryFailed(new ManagedLedgerException(
"Unexpected managedledger implementation, doesn't support "
+ "direct read entry operation."), ctx);
}
}
public PositionImpl getPositionAfterN(PositionImpl startPosition, long n) throws ManagedLedgerException {
if (ledger instanceof ManagedLedgerImpl) {
return ((ManagedLedgerImpl) ledger).getPositionAfterN(startPosition, n,
ManagedLedgerImpl.PositionBound.startExcluded);
} else {
throw new ManagedLedgerException("Unexpected managedledger implementation, doesn't support "
+ "getPositionAfterN operation.");
}
}
public PositionImpl getFirstPosition() throws ManagedLedgerException {
if (ledger instanceof ManagedLedgerImpl) {
return ((ManagedLedgerImpl) ledger).getFirstPosition();
} else {
throw new ManagedLedgerException("Unexpected managedledger implementation, doesn't support "
+ "getFirstPosition operation.");
}
}
public long getNumberOfEntries() {
return ledger.getNumberOfEntries();
}
private void decrementPendingWriteOpsAndCheck() {
long pending = pendingWriteOps.decrementAndGet();
if (pending == 0 && isFenced && !isClosingOrDeleting) {
synchronized (this) {
if (isFenced && !isClosingOrDeleting) {
messageDeduplication.resetHighestSequenceIdPushed();
log.info("[{}] Un-fencing topic...", topic);
// signal to managed ledger that we are ready to resume by creating a new ledger
ledger.readyToCreateNewLedger();
unfence();
}
}
}
}
@Override
public void addComplete(Position pos, ByteBuf entryData, Object ctx) {
PublishContext publishContext = (PublishContext) ctx;
PositionImpl position = (PositionImpl) pos;
// Message has been successfully persisted
messageDeduplication.recordMessagePersisted(publishContext, position);
publishContext.setMetadataFromEntryData(entryData);
publishContext.completed(null, position.getLedgerId(), position.getEntryId());
// in order to sync the max position when cursor read entries
transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry());
decrementPendingWriteOpsAndCheck();
}
@Override
public synchronized void addFailed(ManagedLedgerException exception, Object ctx) {
if (exception instanceof ManagedLedgerFencedException) {
// If the managed ledger has been fenced, we cannot continue using it. We need to close and reopen
close();
} else {
// fence topic when failed to write a message to BK
fence();
// close all producers
List<CompletableFuture<Void>> futures = Lists.newArrayList();
producers.values().forEach(producer -> futures.add(producer.disconnect()));
FutureUtil.waitForAll(futures).handle((BiFunction<Void, Throwable, Void>) (aVoid, throwable) -> {
decrementPendingWriteOpsAndCheck();
return null;
});
PublishContext callback = (PublishContext) ctx;
if (exception instanceof ManagedLedgerAlreadyClosedException) {
if (log.isDebugEnabled()) {
log.debug("[{}] Failed to persist msg in store: {}", topic, exception.getMessage());
}
callback.completed(new TopicClosedException(exception), -1, -1);
return;
} else {
log.warn("[{}] Failed to persist msg in store: {}", topic, exception.getMessage());
}
if (exception instanceof ManagedLedgerTerminatedException) {
// Signal the producer that this topic is no longer available
callback.completed(new TopicTerminatedException(exception), -1, -1);
} else {
// Use generic persistence exception
callback.completed(new PersistenceException(exception), -1, -1);
}
}
}
@Override
public CompletableFuture<Optional<Long>> addProducer(Producer producer,
CompletableFuture<Void> producerQueuedFuture) {
return super.addProducer(producer, producerQueuedFuture).thenCompose(topicEpoch -> {
messageDeduplication.producerAdded(producer.getProducerName());
// Start replication producers if not already
return startReplProducers().thenApply(__ -> topicEpoch);
});
}
@Override
protected CompletableFuture<Long> incrementTopicEpoch(Optional<Long> currentEpoch) {
long newEpoch = currentEpoch.orElse(-1L) + 1;
return setTopicEpoch(newEpoch);
}
@Override
protected CompletableFuture<Long> setTopicEpoch(long newEpoch) {
CompletableFuture<Long> future = new CompletableFuture<>();
ledger.asyncSetProperty(TOPIC_EPOCH_PROPERTY_NAME, String.valueOf(newEpoch), new UpdatePropertiesCallback() {
@Override
public void updatePropertiesComplete(Map<String, String> properties, Object ctx) {
log.info("[{}] Updated topic epoch to {}", getName(), newEpoch);
future.complete(newEpoch);
}
@Override
public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) {
log.warn("[{}] Failed to update topic epoch to {}: {}", getName(), newEpoch, exception.getMessage());
future.completeExceptionally(exception);
}
}, null);
return future;
}
private boolean hasRemoteProducers() {
AtomicBoolean foundRemote = new AtomicBoolean(false);
producers.values().forEach(producer -> {
if (producer.isRemote()) {
foundRemote.set(true);
}
});
return foundRemote.get();
}
public CompletableFuture<Void> startReplProducers() {
// read repl-cluster from policies to avoid restart of replicator which are in process of disconnect and close
return brokerService.pulsar().getPulsarResources().getNamespaceResources()
.getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace()))
.thenAccept(optPolicies -> {
if (optPolicies.isPresent()) {
if (optPolicies.get().replication_clusters != null) {
Set<String> configuredClusters = Sets.newTreeSet(optPolicies.get().replication_clusters);
replicators.forEach((region, replicator) -> {
if (configuredClusters.contains(region)) {
replicator.startProducer();
}
});
}
} else {
replicators.forEach((region, replicator) -> replicator.startProducer());
}
}).exceptionally(ex -> {
if (log.isDebugEnabled()) {
log.debug("[{}] Error getting policies while starting repl-producers {}", topic, ex.getMessage());
}
replicators.forEach((region, replicator) -> replicator.startProducer());
return null;
});
}
public CompletableFuture<Void> stopReplProducers() {
List<CompletableFuture<Void>> closeFutures = Lists.newArrayList();
replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect()));
return FutureUtil.waitForAll(closeFutures);
}
private synchronized CompletableFuture<Void> closeReplProducersIfNoBacklog() {
List<CompletableFuture<Void>> closeFutures = Lists.newArrayList();
replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect(true)));
return FutureUtil.waitForAll(closeFutures);
}
@Override
protected void handleProducerRemoved(Producer producer) {
super.handleProducerRemoved(producer);
messageDeduplication.producerRemoved(producer.getProducerName());
}
@Override
public CompletableFuture<Consumer> subscribe(final TransportCnx cnx, String subscriptionName, long consumerId,
SubType subType, int priorityLevel, String consumerName,
boolean isDurable, MessageId startMessageId,
Map<String, String> metadata, boolean readCompacted,
InitialPosition initialPosition,
long startMessageRollbackDurationSec,
boolean replicatedSubscriptionStateArg,
KeySharedMeta keySharedMeta) {
if (readCompacted && !(subType == SubType.Failover || subType == SubType.Exclusive)) {
return FutureUtil.failedFuture(new NotAllowedException(
"readCompacted only allowed on failover or exclusive subscriptions"));
}
return brokerService.checkTopicNsOwnership(getName()).thenCompose(__ -> {
boolean replicatedSubscriptionState = replicatedSubscriptionStateArg;
if (replicatedSubscriptionState
&& !brokerService.pulsar().getConfiguration().isEnableReplicatedSubscriptions()) {
log.warn("[{}] Replicated Subscription is disabled by broker.", getName());
replicatedSubscriptionState = false;
}
if (subType == SubType.Key_Shared
&& !brokerService.pulsar().getConfiguration().isSubscriptionKeySharedEnable()) {
return FutureUtil.failedFuture(
new NotAllowedException("Key_Shared subscription is disabled by broker."));
}
try {
if (!topic.endsWith(EventsTopicNames.NAMESPACE_EVENTS_LOCAL_NAME)
&& !checkSubscriptionTypesEnable(subType)) {
return FutureUtil.failedFuture(
new NotAllowedException("Topic[{" + topic + "}] doesn't support "
+ subType.name() + " sub type!"));
}
} catch (Exception e) {
return FutureUtil.failedFuture(e);
}
if (isBlank(subscriptionName)) {
if (log.isDebugEnabled()) {
log.debug("[{}] Empty subscription name", topic);
}
return FutureUtil.failedFuture(new NamingException("Empty subscription name"));
}
if (hasBatchMessagePublished && !cnx.isBatchMessageCompatibleVersion()) {
if (log.isDebugEnabled()) {
log.debug("[{}] Consumer doesn't support batch-message {}", topic, subscriptionName);
}
return FutureUtil.failedFuture(
new UnsupportedVersionException("Consumer doesn't support batch-message"));
}
if (subscriptionName.startsWith(replicatorPrefix)
|| subscriptionName.equals(DEDUPLICATION_CURSOR_NAME)) {
log.warn("[{}] Failed to create subscription for {}", topic, subscriptionName);
return FutureUtil.failedFuture(
new NamingException("Subscription with reserved subscription name attempted"));
}
if (cnx.clientAddress() != null && cnx.clientAddress().toString().contains(":")) {
SubscribeRateLimiter.ConsumerIdentifier consumer = new SubscribeRateLimiter.ConsumerIdentifier(
cnx.clientAddress().toString().split(":")[0], consumerName, consumerId);
if (subscribeRateLimiter.isPresent() && (!subscribeRateLimiter.get().subscribeAvailable(consumer)
|| !subscribeRateLimiter.get().tryAcquire(consumer))) {
log.warn("[{}] Failed to create subscription for {} {} limited by {}, available {}",
topic, subscriptionName, consumer, subscribeRateLimiter.get().getSubscribeRate(),
subscribeRateLimiter.get().getAvailableSubscribeRateLimit(consumer));
return FutureUtil.failedFuture(
new NotAllowedException("Subscribe limited by subscribe rate limit per consumer."));
}
}
lock.readLock().lock();
try {
if (isFenced) {
log.warn("[{}] Attempting to subscribe to a fenced topic", topic);
return FutureUtil.failedFuture(new TopicFencedException("Topic is temporarily unavailable"));
}
handleConsumerAdded(subscriptionName, consumerName);
} finally {
lock.readLock().unlock();
}
CompletableFuture<? extends Subscription> subscriptionFuture = isDurable ? //
getDurableSubscription(subscriptionName, initialPosition, startMessageRollbackDurationSec,
replicatedSubscriptionState)
: getNonDurableSubscription(subscriptionName, startMessageId, initialPosition,
startMessageRollbackDurationSec);
int maxUnackedMessages = isDurable
? getMaxUnackedMessagesOnConsumer()
: 0;
CompletableFuture<Consumer> future = subscriptionFuture.thenCompose(subscription -> {
Consumer consumer = new Consumer(subscription, subType, topic, consumerId, priorityLevel,
consumerName, maxUnackedMessages, cnx, cnx.getAuthRole(), metadata,
readCompacted, initialPosition, keySharedMeta, startMessageId);
return addConsumerToSubscription(subscription, consumer).thenCompose(v -> {
checkBackloggedCursors();
if (!cnx.isActive()) {
try {
consumer.close();
} catch (BrokerServiceException e) {
if (e instanceof ConsumerBusyException) {
log.warn("[{}][{}] Consumer {} {} already connected",
topic, subscriptionName, consumerId, consumerName);
} else if (e instanceof SubscriptionBusyException) {
log.warn("[{}][{}] {}", topic, subscriptionName, e.getMessage());
}
decrementUsageCount();
return FutureUtil.failedFuture(e);
}
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] [{}] Subscribe failed -- count: {}", topic, subscriptionName,
consumer.consumerName(), currentUsageCount());
}
decrementUsageCount();
return FutureUtil.failedFuture(
new BrokerServiceException("Connection was closed while the opening the cursor "));
} else {
checkReplicatedSubscriptionControllerState();
log.info("[{}][{}] Created new subscription for {}", topic, subscriptionName, consumerId);
return CompletableFuture.completedFuture(consumer);
}
});
});
future.exceptionally(ex -> {
decrementUsageCount();
if (ex.getCause() instanceof ConsumerBusyException) {
log.warn("[{}][{}] Consumer {} {} already connected", topic, subscriptionName, consumerId,
consumerName);
} else if (ex.getCause() instanceof SubscriptionBusyException) {
log.warn("[{}][{}] {}", topic, subscriptionName, ex.getMessage());
} else {
log.error("[{}] Failed to create subscription: {} error: {}", topic, subscriptionName, ex);
}
return null;
});
return future;
});
}
public void updateUnackedMessagesAppliedOnSubscription(Policies policies) {
maxUnackedMessagesOnSubscriptionApplied = getTopicPolicies()
.map(TopicPolicies::getMaxUnackedMessagesOnSubscription)
.orElseGet(() ->
policies != null && policies.max_unacked_messages_per_subscription != null
? policies.max_unacked_messages_per_subscription
: brokerService.pulsar().getConfiguration().getMaxUnackedMessagesPerSubscription()
);
}
private void updateUnackedMessagesExceededOnConsumer(Policies data) {
maxUnackedMessagesOnConsumerAppilied = getTopicPolicies()
.map(TopicPolicies::getMaxUnackedMessagesOnConsumer)
.orElseGet(() -> data != null && data.max_unacked_messages_per_consumer != null
? data.max_unacked_messages_per_consumer
: brokerService.pulsar().getConfiguration().getMaxUnackedMessagesPerConsumer());
getSubscriptions().forEach((name, sub) -> {
if (sub != null) {
sub.getConsumers().forEach(consumer -> {
if (consumer.getMaxUnackedMessages() != maxUnackedMessagesOnConsumerAppilied) {
consumer.setMaxUnackedMessages(maxUnackedMessagesOnConsumerAppilied);
}
});
}
});
}
private CompletableFuture<Subscription> getDurableSubscription(String subscriptionName,
InitialPosition initialPosition, long startMessageRollbackDurationSec, boolean replicated) {
CompletableFuture<Subscription> subscriptionFuture = new CompletableFuture<>();
if (checkMaxSubscriptionsPerTopicExceed(subscriptionName)) {
subscriptionFuture.completeExceptionally(new NotAllowedException(
"Exceed the maximum number of subscriptions of the topic: " + topic));
return subscriptionFuture;
}
Map<String, Long> properties = PersistentSubscription.getBaseCursorProperties(replicated);
ledger.asyncOpenCursor(Codec.encode(subscriptionName), initialPosition, properties, new OpenCursorCallback() {
@Override
public void openCursorComplete(ManagedCursor cursor, Object ctx) {
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Opened cursor", topic, subscriptionName);
}
PersistentSubscription subscription = subscriptions.get(subscriptionName);
if (subscription == null) {
subscription = subscriptions.computeIfAbsent(subscriptionName,
name -> createPersistentSubscription(subscriptionName, cursor, replicated));
} else {
// if subscription exists, check if it's a non-durable subscription
if (subscription.getCursor() != null && !subscription.getCursor().isDurable()) {
subscriptionFuture.completeExceptionally(
new NotAllowedException("NonDurable subscription with the same name already exists."));
return;
}
}
if (replicated && !subscription.isReplicated()) {
// Flip the subscription state
subscription.setReplicated(replicated);
}
if (startMessageRollbackDurationSec > 0) {
resetSubscriptionCursor(subscription, subscriptionFuture, startMessageRollbackDurationSec);
} else {
subscriptionFuture.complete(subscription);
}
}
@Override
public void openCursorFailed(ManagedLedgerException exception, Object ctx) {
log.warn("[{}] Failed to create subscription for {}: {}", topic, subscriptionName,
exception.getMessage());
decrementUsageCount();
subscriptionFuture.completeExceptionally(new PersistenceException(exception));
if (exception instanceof ManagedLedgerFencedException) {
// If the managed ledger has been fenced, we cannot continue using it. We need to close and reopen
close();
}
}
}, null);
return subscriptionFuture;
}
private CompletableFuture<? extends Subscription> getNonDurableSubscription(String subscriptionName,
MessageId startMessageId, InitialPosition initialPosition, long startMessageRollbackDurationSec) {
log.info("[{}][{}] Creating non-durable subscription at msg id {}", topic, subscriptionName, startMessageId);
CompletableFuture<Subscription> subscriptionFuture = new CompletableFuture<>();
if (checkMaxSubscriptionsPerTopicExceed(subscriptionName)) {
subscriptionFuture.completeExceptionally(new NotAllowedException(
"Exceed the maximum number of subscriptions of the topic: " + topic));
return subscriptionFuture;
}
synchronized (ledger) {
// Create a new non-durable cursor only for the first consumer that connects
PersistentSubscription subscription = subscriptions.get(subscriptionName);
if (subscription == null) {
MessageIdImpl msgId = startMessageId != null ? (MessageIdImpl) startMessageId
: (MessageIdImpl) MessageId.latest;
long ledgerId = msgId.getLedgerId();
long entryId = msgId.getEntryId();
// Ensure that the start message id starts from a valid entry.
if (ledgerId >= 0 && entryId >= 0
&& msgId instanceof BatchMessageIdImpl) {
// When the start message is relative to a batch, we need to take one step back on the previous
// message,
// because the "batch" might not have been consumed in its entirety.
// The client will then be able to discard the first messages if needed.
entryId = msgId.getEntryId() - 1;
}
Position startPosition = new PositionImpl(ledgerId, entryId);
ManagedCursor cursor = null;
try {
cursor = ledger.newNonDurableCursor(startPosition, subscriptionName, initialPosition);
} catch (ManagedLedgerException e) {
return FutureUtil.failedFuture(e);
}
subscription = new PersistentSubscription(this, subscriptionName, cursor, false);
subscriptions.put(subscriptionName, subscription);
} else {
// if subscription exists, check if it's a durable subscription
if (subscription.getCursor() != null && subscription.getCursor().isDurable()) {
return FutureUtil.failedFuture(
new NotAllowedException("Durable subscription with the same name already exists."));
}
}
if (startMessageRollbackDurationSec > 0) {
resetSubscriptionCursor(subscription, subscriptionFuture, startMessageRollbackDurationSec);
return subscriptionFuture;
} else {
return CompletableFuture.completedFuture(subscription);
}
}
}
private void resetSubscriptionCursor(Subscription subscription, CompletableFuture<Subscription> subscriptionFuture,
long startMessageRollbackDurationSec) {
long timestamp = System.currentTimeMillis()
- TimeUnit.SECONDS.toMillis(startMessageRollbackDurationSec);
final Subscription finalSubscription = subscription;
subscription.resetCursor(timestamp).handle((s, ex) -> {
if (ex != null) {
log.warn("[{}] Failed to reset cursor {} position at timestamp {}, caused by {}", topic,
subscription.getName(), startMessageRollbackDurationSec, ex.getMessage());
}
subscriptionFuture.complete(finalSubscription);
return null;
});
}
@Override
public CompletableFuture<Subscription> createSubscription(String subscriptionName, InitialPosition initialPosition,
boolean replicateSubscriptionState) {
return getDurableSubscription(subscriptionName, initialPosition,
0 /*avoid reseting cursor*/, replicateSubscriptionState);
}
/**
* Delete the cursor ledger for a given subscription.
*
* @param subscriptionName Subscription for which the cursor ledger is to be deleted
* @return Completable future indicating completion of unsubscribe operation Completed exceptionally with:
* ManagedLedgerException if cursor ledger delete fails
*/
@Override
public CompletableFuture<Void> unsubscribe(String subscriptionName) {
CompletableFuture<Void> unsubscribeFuture = new CompletableFuture<>();
getBrokerService().getManagedLedgerFactory().asyncDelete(TopicName.get(MLPendingAckStore
.getTransactionPendingAckStoreSuffix(topic,
Codec.encode(subscriptionName))).getPersistenceNamingEncoding(),
new AsyncCallbacks.DeleteLedgerCallback() {
@Override
public void deleteLedgerComplete(Object ctx) {
asyncDeleteCursor(subscriptionName, unsubscribeFuture);
}
@Override
public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) {
if (exception instanceof MetadataNotFoundException) {
asyncDeleteCursor(subscriptionName, unsubscribeFuture);
return;
}
unsubscribeFuture.completeExceptionally(exception);
log.error("[{}][{}] Error deleting subscription pending ack store",
topic, subscriptionName, exception);
}
}, null);
return unsubscribeFuture;
}
private void asyncDeleteCursor(String subscriptionName, CompletableFuture<Void> unsubscribeFuture) {
ledger.asyncDeleteCursor(Codec.encode(subscriptionName), new DeleteCursorCallback() {
@Override
public void deleteCursorComplete(Object ctx) {
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Cursor deleted successfully", topic, subscriptionName);
}