/
PersistentSubscription.java
1178 lines (1044 loc) · 52.9 KB
/
PersistentSubscription.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 org.apache.pulsar.common.events.EventsTopicNames.checkTopicIsEventsNames;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.MoreObjects;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.concurrent.atomic.LongAdder;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ClearBacklogCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback;
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.ConcurrentFindCursorPositionException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.InvalidCursorPositionException;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.commons.lang3.tuple.MutablePair;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.intercept.BrokerInterceptor;
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException;
import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException;
import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException;
import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionFencedException;
import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionInvalidCursorPosition;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.Dispatcher;
import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.transaction.pendingack.PendingAckHandle;
import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleDisabled;
import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl;
import org.apache.pulsar.client.api.transaction.TxnID;
import org.apache.pulsar.common.api.proto.CommandAck.AckType;
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.ReplicatedSubscriptionsSnapshot;
import org.apache.pulsar.common.api.proto.TxnAction;
import org.apache.pulsar.common.naming.TopicName;
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.SubscriptionStatsImpl;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.Markers;
import org.apache.pulsar.common.util.FutureUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class PersistentSubscription implements Subscription {
protected final PersistentTopic topic;
protected final ManagedCursor cursor;
protected volatile Dispatcher dispatcher;
protected final String topicName;
protected final String subName;
protected final String fullName;
private static final int FALSE = 0;
private static final int TRUE = 1;
private static final AtomicIntegerFieldUpdater<PersistentSubscription> IS_FENCED_UPDATER =
AtomicIntegerFieldUpdater.newUpdater(PersistentSubscription.class, "isFenced");
private volatile int isFenced = FALSE;
private PersistentMessageExpiryMonitor expiryMonitor;
private long lastExpireTimestamp = 0L;
private long lastConsumedFlowTimestamp = 0L;
private long lastMarkDeleteAdvancedTimestamp = 0L;
// for connected subscriptions, message expiry will be checked if the backlog is greater than this threshold
private static final int MINIMUM_BACKLOG_FOR_EXPIRY_CHECK = 1000;
private static final String REPLICATED_SUBSCRIPTION_PROPERTY = "pulsar.replicated.subscription";
// Map of properties that is used to mark this subscription as "replicated".
// Since this is the only field at this point, we can just keep a static
// instance of the map.
private static final Map<String, Long> REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES = new TreeMap<>();
private static final Map<String, Long> NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES = Collections.emptyMap();
private volatile ReplicatedSubscriptionSnapshotCache replicatedSubscriptionSnapshotCache;
private volatile Position lastMarkDeleteForTransactionMarker;
private volatile boolean isDeleteTransactionMarkerInProcess = false;
private final PendingAckHandle pendingAckHandle;
private final LongAdder bytesOutFromRemovedConsumers = new LongAdder();
private final LongAdder msgOutFromRemovedConsumer = new LongAdder();
static {
REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES.put(REPLICATED_SUBSCRIPTION_PROPERTY, 1L);
}
static Map<String, Long> getBaseCursorProperties(boolean isReplicated) {
return isReplicated ? REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES : NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES;
}
static boolean isCursorFromReplicatedSubscription(ManagedCursor cursor) {
return cursor.getProperties().containsKey(REPLICATED_SUBSCRIPTION_PROPERTY);
}
public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor,
boolean replicated) {
this.topic = topic;
this.cursor = cursor;
this.topicName = topic.getName();
this.subName = subscriptionName;
this.fullName = MoreObjects.toStringHelper(this).add("topic", topicName).add("name", subName).toString();
this.expiryMonitor = new PersistentMessageExpiryMonitor(topicName, subscriptionName, cursor, this);
this.setReplicated(replicated);
if (topic.getBrokerService().getPulsar().getConfig().isTransactionCoordinatorEnabled()
&& !checkTopicIsEventsNames(TopicName.get(topicName))) {
this.pendingAckHandle = new PendingAckHandleImpl(this);
} else {
this.pendingAckHandle = new PendingAckHandleDisabled();
}
IS_FENCED_UPDATER.set(this, FALSE);
}
public void updateLastMarkDeleteAdvancedTimestamp() {
this.lastMarkDeleteAdvancedTimestamp =
Math.max(this.lastMarkDeleteAdvancedTimestamp, System.currentTimeMillis());
}
@Override
public BrokerInterceptor interceptor() {
return topic.getBrokerService().getInterceptor();
}
@Override
public String getName() {
return this.subName;
}
@Override
public Topic getTopic() {
return topic;
}
@Override
public boolean isReplicated() {
return replicatedSubscriptionSnapshotCache != null;
}
public boolean setReplicated(boolean replicated) {
ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig();
if (!replicated || !config.isEnableReplicatedSubscriptions()) {
this.replicatedSubscriptionSnapshotCache = null;
} else if (this.replicatedSubscriptionSnapshotCache == null) {
this.replicatedSubscriptionSnapshotCache = new ReplicatedSubscriptionSnapshotCache(subName,
config.getReplicatedSubscriptionsSnapshotMaxCachedPerSubscription());
}
if (this.cursor != null) {
if (replicated) {
return this.cursor.putProperty(REPLICATED_SUBSCRIPTION_PROPERTY, 1L);
} else {
return this.cursor.removeProperty(REPLICATED_SUBSCRIPTION_PROPERTY);
}
}
return false;
}
@Override
public CompletableFuture<Void> addConsumer(Consumer consumer) {
return pendingAckHandle.pendingAckHandleFuture().thenCompose(future -> {
synchronized (PersistentSubscription.this) {
cursor.updateLastActive();
if (IS_FENCED_UPDATER.get(this) == TRUE) {
log.warn("Attempting to add consumer {} on a fenced subscription", consumer);
return FutureUtil.failedFuture(new SubscriptionFencedException("Subscription is fenced"));
}
if (dispatcher == null || !dispatcher.isConsumerConnected()) {
Dispatcher previousDispatcher = null;
boolean useStreamingDispatcher = topic.getBrokerService().getPulsar()
.getConfiguration().isStreamingDispatch();
switch (consumer.subType()) {
case Exclusive:
if (dispatcher == null || dispatcher.getType() != SubType.Exclusive) {
previousDispatcher = dispatcher;
dispatcher = useStreamingDispatcher
? new PersistentStreamingDispatcherSingleActiveConsumer(
cursor, SubType.Exclusive, 0, topic, this)
: new PersistentDispatcherSingleActiveConsumer(
cursor, SubType.Exclusive, 0, topic, this);
}
break;
case Shared:
if (dispatcher == null || dispatcher.getType() != SubType.Shared) {
previousDispatcher = dispatcher;
dispatcher = useStreamingDispatcher
? new PersistentStreamingDispatcherMultipleConsumers(
topic, cursor, this)
: new PersistentDispatcherMultipleConsumers(topic, cursor, this);
}
break;
case Failover:
int partitionIndex = TopicName.getPartitionIndex(topicName);
if (partitionIndex < 0) {
// For non partition topics, use a negative index so
// dispatcher won't sort consumers before picking
// an active consumer for the topic.
partitionIndex = -1;
}
if (dispatcher == null || dispatcher.getType() != SubType.Failover) {
previousDispatcher = dispatcher;
dispatcher = useStreamingDispatcher
? new PersistentStreamingDispatcherSingleActiveConsumer(
cursor, SubType.Failover, partitionIndex, topic, this) :
new PersistentDispatcherSingleActiveConsumer(cursor, SubType.Failover,
partitionIndex, topic, this);
}
break;
case Key_Shared:
if (dispatcher == null || dispatcher.getType() != SubType.Key_Shared) {
previousDispatcher = dispatcher;
KeySharedMeta ksm = consumer.getKeySharedMeta();
dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this,
topic.getBrokerService().getPulsar().getConfiguration(), ksm);
}
break;
default:
return FutureUtil.failedFuture(
new ServerMetadataException("Unsupported subscription type"));
}
if (previousDispatcher != null) {
previousDispatcher.close().thenRun(() -> {
log.info("[{}][{}] Successfully closed previous dispatcher", topicName, subName);
}).exceptionally(ex -> {
log.error("[{}][{}] Failed to close previous dispatcher", topicName, subName, ex);
return null;
});
}
} else {
if (consumer.subType() != dispatcher.getType()) {
return FutureUtil.failedFuture(
new SubscriptionBusyException("Subscription is of different type"));
}
}
try {
dispatcher.addConsumer(consumer);
return CompletableFuture.completedFuture(null);
} catch (BrokerServiceException brokerServiceException) {
return FutureUtil.failedFuture(brokerServiceException);
}
}
});
}
@Override
public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor) throws BrokerServiceException {
cursor.updateLastActive();
if (dispatcher != null) {
dispatcher.removeConsumer(consumer);
}
// preserve accumulative stats form removed consumer
ConsumerStatsImpl stats = consumer.getStats();
bytesOutFromRemovedConsumers.add(stats.bytesOutCounter);
msgOutFromRemovedConsumer.add(stats.msgOutCounter);
if (dispatcher != null && dispatcher.getConsumers().isEmpty()) {
deactivateCursor();
if (!cursor.isDurable()) {
// If cursor is not durable, we need to clean up the subscription as well
this.close().thenRun(() -> {
synchronized (this) {
if (dispatcher != null) {
dispatcher.close().thenRun(() -> {
log.info("[{}][{}] Successfully closed dispatcher for reader", topicName, subName);
}).exceptionally(ex -> {
log.error("[{}][{}] Failed to close dispatcher for reader", topicName, subName, ex);
return null;
});
}
}
}).exceptionally(exception -> {
log.error("[{}][{}] Failed to close subscription for reader", topicName, subName, exception);
return null;
});
// when topic closes: it iterates through concurrent-subscription map to close each subscription. so,
// topic.remove again try to access same map which creates deadlock. so, execute it in different thread.
topic.getBrokerService().pulsar().getExecutor().submit(() ->{
topic.removeSubscription(subName);
// Also need remove the cursor here, otherwise the data deletion will not work well.
// Because data deletion depends on the mark delete position of all cursors.
if (!isResetCursor) {
try {
topic.getManagedLedger().deleteCursor(cursor.getName());
} catch (InterruptedException | ManagedLedgerException e) {
log.warn("[{}] [{}] Failed to remove non durable cursor", topic.getName(), subName, e);
}
}
});
}
}
// invalid consumer remove will throw an exception
// decrement usage is triggered only for valid consumer close
topic.decrementUsageCount();
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] [{}] Removed consumer -- count: {}", topic.getName(), subName, consumer.consumerName(),
topic.currentUsageCount());
}
}
public void deactivateCursor() {
this.cursor.setInactive();
}
@Override
public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) {
this.lastConsumedFlowTimestamp = System.currentTimeMillis();
dispatcher.consumerFlow(consumer, additionalNumberOfMessages);
}
@Override
public void acknowledgeMessage(List<Position> positions, AckType ackType, Map<String, Long> properties) {
Position previousMarkDeletePosition = cursor.getMarkDeletedPosition();
if (ackType == AckType.Cumulative) {
if (positions.size() != 1) {
log.warn("[{}][{}] Invalid cumulative ack received with multiple message ids.", topicName, subName);
return;
}
Position position = positions.get(0);
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Cumulative ack on {}", topicName, subName, position);
}
cursor.asyncMarkDelete(position, mergeCursorProperties(properties),
markDeleteCallback, previousMarkDeletePosition);
} else {
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Individual acks on {}", topicName, subName, positions);
}
cursor.asyncDelete(positions, deleteCallback, previousMarkDeletePosition);
if (topic.getBrokerService().getPulsar().getConfig().isTransactionCoordinatorEnabled()) {
positions.forEach(position -> {
if (((ManagedCursorImpl) cursor).isMessageDeleted(position)) {
pendingAckHandle.clearIndividualPosition(position);
}
});
}
if (dispatcher != null) {
dispatcher.getRedeliveryTracker().removeBatch(positions);
}
}
if (!cursor.getMarkDeletedPosition().equals(previousMarkDeletePosition)) {
this.updateLastMarkDeleteAdvancedTimestamp();
// Mark delete position advance
ReplicatedSubscriptionSnapshotCache snapshotCache = this.replicatedSubscriptionSnapshotCache;
if (snapshotCache != null) {
ReplicatedSubscriptionsSnapshot snapshot = snapshotCache
.advancedMarkDeletePosition((PositionImpl) cursor.getMarkDeletedPosition());
if (snapshot != null) {
topic.getReplicatedSubscriptionController()
.ifPresent(c -> c.localSubscriptionUpdated(subName, snapshot));
}
}
}
if (topic.getBrokerService().getPulsar().getConfig().isTransactionCoordinatorEnabled()
&& this.pendingAckHandle.isTransactionAckPresent()) {
Position currentMarkDeletePosition = cursor.getMarkDeletedPosition();
if ((lastMarkDeleteForTransactionMarker == null
|| ((PositionImpl) lastMarkDeleteForTransactionMarker)
.compareTo((PositionImpl) currentMarkDeletePosition) < 0) && !isDeleteTransactionMarkerInProcess) {
isDeleteTransactionMarkerInProcess = true;
deleteTransactionMarker((PositionImpl) currentMarkDeletePosition, ackType, properties);
}
}
if (topic.getManagedLedger().isTerminated() && cursor.getNumberOfEntriesInBacklog(false) == 0) {
// Notify all consumer that the end of topic was reached
if (dispatcher != null) {
dispatcher.getConsumers().forEach(Consumer::reachedEndOfTopic);
}
}
}
private void deleteTransactionMarker(PositionImpl position, AckType ackType, Map<String, Long> properties) {
if (position != null) {
ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger());
PositionImpl nextPosition = managedLedger.getNextValidPosition(position);
if (nextPosition != null
&& nextPosition.compareTo((PositionImpl) managedLedger.getLastConfirmedEntry()) <= 0) {
managedLedger.asyncReadEntry(nextPosition, new ReadEntryCallback() {
@Override
public void readEntryComplete(Entry entry, Object ctx) {
try {
MessageMetadata messageMetadata = Commands.parseMessageMetadata(entry.getDataBuffer());
isDeleteTransactionMarkerInProcess = false;
if (Markers.isTxnCommitMarker(messageMetadata)
|| Markers.isTxnAbortMarker(messageMetadata)) {
lastMarkDeleteForTransactionMarker = position;
acknowledgeMessage(Collections.singletonList(nextPosition), ackType, properties);
}
} finally {
entry.release();
}
}
@Override
public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
isDeleteTransactionMarkerInProcess = false;
log.error("Fail to read transaction marker! Position : {}", position, exception);
}
}, null);
} else {
isDeleteTransactionMarkerInProcess = false;
}
} else {
isDeleteTransactionMarkerInProcess = false;
}
}
public CompletableFuture<Void> transactionIndividualAcknowledge(
TxnID txnId,
List<MutablePair<PositionImpl, Integer>> positions) {
return pendingAckHandle.individualAcknowledgeMessage(txnId, positions);
}
public CompletableFuture<Void> transactionCumulativeAcknowledge(TxnID txnId, List<PositionImpl> positions) {
return pendingAckHandle.cumulativeAcknowledgeMessage(txnId, positions);
}
private final MarkDeleteCallback markDeleteCallback = new MarkDeleteCallback() {
@Override
public void markDeleteComplete(Object ctx) {
PositionImpl oldMD = (PositionImpl) ctx;
PositionImpl newMD = (PositionImpl) cursor.getMarkDeletedPosition();
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Mark deleted messages to position {} from position {}",
topicName, subName, newMD, oldMD);
}
// Signal the dispatchers to give chance to take extra actions
notifyTheMarkDeletePositionMoveForwardIfNeeded(oldMD);
}
@Override
public void markDeleteFailed(ManagedLedgerException exception, Object ctx) {
// TODO: cut consumer connection on markDeleteFailed
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Failed to mark delete for position {}: {}", topicName, subName, ctx, exception);
}
}
};
private final DeleteCallback deleteCallback = new DeleteCallback() {
@Override
public void deleteComplete(Object position) {
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Deleted message at {}", topicName, subName, position);
}
// Signal the dispatchers to give chance to take extra actions
notifyTheMarkDeletePositionMoveForwardIfNeeded((PositionImpl) position);
}
@Override
public void deleteFailed(ManagedLedgerException exception, Object ctx) {
log.warn("[{}][{}] Failed to delete message at {}: {}", topicName, subName, ctx, exception);
}
};
private void notifyTheMarkDeletePositionMoveForwardIfNeeded(Position oldPosition) {
PositionImpl oldMD = (PositionImpl) oldPosition;
PositionImpl newMD = (PositionImpl) cursor.getMarkDeletedPosition();
if (dispatcher != null && newMD.compareTo(oldMD) > 0) {
dispatcher.markDeletePositionMoveForward();
}
}
@Override
public String toString() {
return fullName;
}
@Override
public String getTopicName() {
return this.topicName;
}
@Override
public SubType getType() {
return dispatcher != null ? dispatcher.getType() : null;
}
@Override
public String getTypeString() {
SubType type = getType();
if (type == null) {
return "None";
}
switch (type) {
case Exclusive:
return "Exclusive";
case Failover:
return "Failover";
case Shared:
return "Shared";
case Key_Shared:
return "Key_Shared";
}
return "Null";
}
@Override
public CompletableFuture<Void> clearBacklog() {
CompletableFuture<Void> future = new CompletableFuture<>();
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Backlog size before clearing: {}", topicName, subName,
cursor.getNumberOfEntriesInBacklog(false));
}
cursor.asyncClearBacklog(new ClearBacklogCallback() {
@Override
public void clearBacklogComplete(Object ctx) {
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Backlog size after clearing: {}", topicName, subName,
cursor.getNumberOfEntriesInBacklog(false));
}
if (dispatcher != null) {
dispatcher.clearDelayedMessages();
}
future.complete(null);
}
@Override
public void clearBacklogFailed(ManagedLedgerException exception, Object ctx) {
log.error("[{}][{}] Failed to clear backlog", topicName, subName, exception);
future.completeExceptionally(exception);
}
}, null);
return future;
}
@Override
public CompletableFuture<Void> skipMessages(int numMessagesToSkip) {
CompletableFuture<Void> future = new CompletableFuture<>();
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Skipping {} messages, current backlog {}", topicName, subName, numMessagesToSkip,
cursor.getNumberOfEntriesInBacklog(false));
}
cursor.asyncSkipEntries(numMessagesToSkip, IndividualDeletedEntries.Exclude,
new AsyncCallbacks.SkipEntriesCallback() {
@Override
public void skipEntriesComplete(Object ctx) {
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Skipped {} messages, new backlog {}", topicName, subName,
numMessagesToSkip, cursor.getNumberOfEntriesInBacklog(false));
}
future.complete(null);
}
@Override
public void skipEntriesFailed(ManagedLedgerException exception, Object ctx) {
log.error("[{}][{}] Failed to skip {} messages", topicName, subName, numMessagesToSkip,
exception);
future.completeExceptionally(exception);
}
}, null);
return future;
}
@Override
public CompletableFuture<Void> resetCursor(long timestamp) {
CompletableFuture<Void> future = new CompletableFuture<>();
PersistentMessageFinder persistentMessageFinder = new PersistentMessageFinder(topicName, cursor);
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Resetting subscription to timestamp {}", topicName, subName, timestamp);
}
persistentMessageFinder.findMessages(timestamp, new AsyncCallbacks.FindEntryCallback() {
@Override
public void findEntryComplete(Position position, Object ctx) {
final Position finalPosition;
if (position == null) {
// this should not happen ideally unless a reset is requested for a time
// that spans beyond the retention limits (time/size)
finalPosition = cursor.getFirstPosition();
if (finalPosition == null) {
log.warn("[{}][{}] Unable to find position for timestamp {}."
+ " Unable to reset cursor to first position",
topicName, subName, timestamp);
future.completeExceptionally(
new SubscriptionInvalidCursorPosition(
"Unable to find position for specified timestamp"));
return;
}
log.info(
"[{}][{}] Unable to find position for timestamp {}."
+ " Resetting cursor to first position {} in ledger",
topicName, subName, timestamp, finalPosition);
} else {
finalPosition = position.getNext();
}
resetCursor(finalPosition, future);
}
@Override
public void findEntryFailed(ManagedLedgerException exception,
Optional<Position> failedReadPosition, Object ctx) {
// todo - what can go wrong here that needs to be retried?
if (exception instanceof ConcurrentFindCursorPositionException) {
future.completeExceptionally(new SubscriptionBusyException(exception.getMessage()));
} else {
future.completeExceptionally(new BrokerServiceException(exception));
}
}
});
return future;
}
@Override
public CompletableFuture<Void> resetCursor(Position position) {
CompletableFuture<Void> future = new CompletableFuture<>();
resetCursor(position, future);
return future;
}
private void resetCursor(Position finalPosition, CompletableFuture<Void> future) {
if (!IS_FENCED_UPDATER.compareAndSet(PersistentSubscription.this, FALSE, TRUE)) {
future.completeExceptionally(new SubscriptionBusyException("Failed to fence subscription"));
return;
}
final CompletableFuture<Void> disconnectFuture;
// Lock the Subscription object before locking the Dispatcher object to avoid deadlocks
synchronized (this) {
if (dispatcher != null && dispatcher.isConsumerConnected()) {
disconnectFuture = dispatcher.disconnectActiveConsumers(true);
} else {
disconnectFuture = CompletableFuture.completedFuture(null);
}
}
disconnectFuture.whenComplete((aVoid, throwable) -> {
if (dispatcher != null) {
dispatcher.resetCloseFuture();
}
if (throwable != null) {
log.error("[{}][{}] Failed to disconnect consumer from subscription", topicName, subName, throwable);
IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE);
future.completeExceptionally(
new SubscriptionBusyException("Failed to disconnect consumers from subscription"));
return;
}
log.info("[{}][{}] Successfully disconnected consumers from subscription, proceeding with cursor reset",
topicName, subName);
try {
cursor.asyncResetCursor(finalPosition, new AsyncCallbacks.ResetCursorCallback() {
@Override
public void resetComplete(Object ctx) {
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Successfully reset subscription to position {}", topicName, subName,
finalPosition);
}
if (dispatcher != null) {
dispatcher.cursorIsReset();
}
IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE);
future.complete(null);
}
@Override
public void resetFailed(ManagedLedgerException exception, Object ctx) {
log.error("[{}][{}] Failed to reset subscription to position {}", topicName, subName,
finalPosition, exception);
IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE);
// todo - retry on InvalidCursorPositionException
// or should we just ask user to retry one more time?
if (exception instanceof InvalidCursorPositionException) {
future.completeExceptionally(new SubscriptionInvalidCursorPosition(exception.getMessage()));
} else if (exception instanceof ConcurrentFindCursorPositionException) {
future.completeExceptionally(new SubscriptionBusyException(exception.getMessage()));
} else {
future.completeExceptionally(new BrokerServiceException(exception));
}
}
});
} catch (Exception e) {
log.error("[{}][{}] Error while resetting cursor", topicName, subName, e);
IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE);
future.completeExceptionally(new BrokerServiceException(e));
}
});
}
@Override
public CompletableFuture<Entry> peekNthMessage(int messagePosition) {
CompletableFuture<Entry> future = new CompletableFuture<>();
if (log.isDebugEnabled()) {
log.debug("[{}][{}] Getting message at position {}", topicName, subName, messagePosition);
}
cursor.asyncGetNthEntry(messagePosition, IndividualDeletedEntries.Exclude, new ReadEntryCallback() {
@Override
public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
future.completeExceptionally(exception);
}
@Override
public void readEntryComplete(Entry entry, Object ctx) {
future.complete(entry);
}
}, null);
return future;
}
@Override
public long getNumberOfEntriesInBacklog(boolean getPreciseBacklog) {
return cursor.getNumberOfEntriesInBacklog(getPreciseBacklog);
}
@Override
public synchronized Dispatcher getDispatcher() {
return this.dispatcher;
}
public long getNumberOfEntriesSinceFirstNotAckedMessage() {
return cursor.getNumberOfEntriesSinceFirstNotAckedMessage();
}
public int getTotalNonContiguousDeletedMessagesRange() {
return cursor.getTotalNonContiguousDeletedMessagesRange();
}
/**
* Close the cursor ledger for this subscription. Requires that there are no active consumers on the dispatcher
*
* @return CompletableFuture indicating the completion of delete operation
*/
@Override
public CompletableFuture<Void> close() {
synchronized (this) {
if (dispatcher != null && dispatcher.isConsumerConnected()) {
return FutureUtil.failedFuture(new SubscriptionBusyException("Subscription has active consumers"));
}
return this.pendingAckHandle.close().thenAccept(v -> {
IS_FENCED_UPDATER.set(this, TRUE);
log.info("[{}][{}] Successfully closed subscription [{}]", topicName, subName, cursor);
});
}
}
/**
* Disconnect all consumers attached to the dispatcher and close this subscription.
*
* @return CompletableFuture indicating the completion of disconnect operation
*/
@Override
public synchronized CompletableFuture<Void> disconnect() {
CompletableFuture<Void> disconnectFuture = new CompletableFuture<>();
// block any further consumers on this subscription
IS_FENCED_UPDATER.set(this, TRUE);
(dispatcher != null ? dispatcher.close() : CompletableFuture.completedFuture(null))
.thenCompose(v -> close()).thenRun(() -> {
log.info("[{}][{}] Successfully disconnected and closed subscription", topicName, subName);
disconnectFuture.complete(null);
}).exceptionally(exception -> {
IS_FENCED_UPDATER.set(this, FALSE);
if (dispatcher != null) {
dispatcher.reset();
}
log.error("[{}][{}] Error disconnecting consumers from subscription", topicName, subName,
exception);
disconnectFuture.completeExceptionally(exception);
return null;
});
return disconnectFuture;
}
/**
* Delete the subscription by closing and deleting its managed cursor if no consumers are connected to it. Handle
* unsubscribe call from admin layer.
*
* @return CompletableFuture indicating the completion of delete operation
*/
@Override
public CompletableFuture<Void> delete() {
return delete(false);
}
/**
* Forcefully close all consumers and deletes the subscription.
* @return
*/
@Override
public CompletableFuture<Void> deleteForcefully() {
return delete(true);
}
/**
* Delete the subscription by closing and deleting its managed cursor. Handle unsubscribe call from admin layer.
*
* @param closeIfConsumersConnected
* Flag indicate whether explicitly close connected consumers before trying to delete subscription. If
* any consumer is connected to it and if this flag is disable then this operation fails.
* @return CompletableFuture indicating the completion of delete operation
*/
private CompletableFuture<Void> delete(boolean closeIfConsumersConnected) {
CompletableFuture<Void> deleteFuture = new CompletableFuture<>();
log.info("[{}][{}] Unsubscribing", topicName, subName);
CompletableFuture<Void> closeSubscriptionFuture = new CompletableFuture<>();
if (closeIfConsumersConnected) {
this.disconnect().thenRun(() -> {
closeSubscriptionFuture.complete(null);
}).exceptionally(ex -> {
log.error("[{}][{}] Error disconnecting and closing subscription", topicName, subName, ex);
closeSubscriptionFuture.completeExceptionally(ex);
return null;
});
} else {
this.close().thenRun(() -> {
closeSubscriptionFuture.complete(null);
}).exceptionally(exception -> {
log.error("[{}][{}] Error closing subscription", topicName, subName, exception);
closeSubscriptionFuture.completeExceptionally(exception);
return null;
});
}
// cursor close handles pending delete (ack) operations
closeSubscriptionFuture.thenCompose(v -> topic.unsubscribe(subName)).thenAccept(v -> {
synchronized (this) {
(dispatcher != null ? dispatcher.close() : CompletableFuture.completedFuture(null)).thenRun(() -> {
log.info("[{}][{}] Successfully deleted subscription", topicName, subName);
deleteFuture.complete(null);
}).exceptionally(ex -> {
IS_FENCED_UPDATER.set(this, FALSE);
if (dispatcher != null) {
dispatcher.reset();
}
log.error("[{}][{}] Error deleting subscription", topicName, subName, ex);
deleteFuture.completeExceptionally(ex);
return null;
});
}
}).exceptionally(exception -> {
IS_FENCED_UPDATER.set(this, FALSE);
log.error("[{}][{}] Error deleting subscription", topicName, subName, exception);
deleteFuture.completeExceptionally(exception);
return null;
});
return deleteFuture;
}
/**
* Handle unsubscribe command from the client API Check with the dispatcher is this consumer can proceed with
* unsubscribe.
*
* @param consumer consumer object that is initiating the unsubscribe operation
* @return CompletableFuture indicating the completion of unsubscribe operation
*/
@Override
public CompletableFuture<Void> doUnsubscribe(Consumer consumer) {
CompletableFuture<Void> future = new CompletableFuture<>();
try {
if (dispatcher.canUnsubscribe(consumer)) {
consumer.close();
return delete();
}
future.completeExceptionally(
new ServerMetadataException("Unconnected or shared consumer attempting to unsubscribe"));
} catch (BrokerServiceException e) {
log.warn("Error removing consumer {}", consumer);
future.completeExceptionally(e);
}
return future;
}
@Override
public List<Consumer> getConsumers() {
Dispatcher dispatcher = this.dispatcher;
if (dispatcher != null) {
return dispatcher.getConsumers();
} else {
return Collections.emptyList();
}
}
@Override
public boolean expireMessages(int messageTTLInSeconds) {
if ((getNumberOfEntriesInBacklog(false) == 0) || (dispatcher != null && dispatcher.isConsumerConnected()
&& getNumberOfEntriesInBacklog(false) < MINIMUM_BACKLOG_FOR_EXPIRY_CHECK
&& !topic.isOldestMessageExpired(cursor, messageTTLInSeconds))) {
// don't do anything for almost caught-up connected subscriptions
return false;
}
this.lastExpireTimestamp = System.currentTimeMillis();
return expiryMonitor.expireMessages(messageTTLInSeconds);
}
@Override
public boolean expireMessages(Position position) {
this.lastExpireTimestamp = System.currentTimeMillis();
return expiryMonitor.expireMessages(position);
}
public double getExpiredMessageRate() {
return expiryMonitor.getMessageExpiryRate();
}
public PersistentMessageExpiryMonitor getExpiryMonitor() {
return expiryMonitor;
}
public long estimateBacklogSize() {
return cursor.getEstimatedSizeSinceMarkDeletePosition();
}
public SubscriptionStatsImpl getStats(Boolean getPreciseBacklog, boolean subscriptionBacklogSize) {
SubscriptionStatsImpl subStats = new SubscriptionStatsImpl();
subStats.lastExpireTimestamp = lastExpireTimestamp;
subStats.lastConsumedFlowTimestamp = lastConsumedFlowTimestamp;
subStats.lastMarkDeleteAdvancedTimestamp = lastMarkDeleteAdvancedTimestamp;
subStats.bytesOutCounter = bytesOutFromRemovedConsumers.longValue();
subStats.msgOutCounter = msgOutFromRemovedConsumer.longValue();
Dispatcher dispatcher = this.dispatcher;
if (dispatcher != null) {
Map<String, List<String>> consumerKeyHashRanges = getType() == SubType.Key_Shared
? ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getConsumerKeyHashRanges() : null;
dispatcher.getConsumers().forEach(consumer -> {
ConsumerStatsImpl consumerStats = consumer.getStats();
subStats.consumers.add(consumerStats);
subStats.msgRateOut += consumerStats.msgRateOut;
subStats.msgThroughputOut += consumerStats.msgThroughputOut;
subStats.bytesOutCounter += consumerStats.bytesOutCounter;
subStats.msgOutCounter += consumerStats.msgOutCounter;
subStats.msgRateRedeliver += consumerStats.msgRateRedeliver;
subStats.chunkedMessageRate += consumerStats.chunkedMessageRate;
subStats.unackedMessages += consumerStats.unackedMessages;
subStats.lastConsumedTimestamp =
Math.max(subStats.lastConsumedTimestamp, consumerStats.lastConsumedTimestamp);