/
ManagedCursorImpl.java
3075 lines (2672 loc) · 134 KB
/
ManagedCursorImpl.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.bookkeeper.mledger.impl;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.bookkeeper.mledger.ManagedLedgerException.getManagedLedgerException;
import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.DEFAULT_LEDGER_DELETE_BACKOFF_TIME_SEC;
import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.DEFAULT_LEDGER_DELETE_RETRIES;
import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException;
import static org.apache.bookkeeper.mledger.util.Errors.isNoSuchLedgerExistsException;
import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.MoreObjects;
import com.google.common.base.Predicate;
import com.google.common.collect.Collections2;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Range;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.RateLimiter;
import com.google.protobuf.InvalidProtocolBufferException;
import io.netty.util.concurrent.FastThreadLocal;
import java.time.Clock;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
import org.apache.bookkeeper.client.AsyncCallback.DeleteCallback;
import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.api.BKException.Code;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ClearBacklogCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.FindEntryCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.SkipEntriesCallback;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.CursorAlreadyClosedException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException;
import org.apache.bookkeeper.mledger.ManagedCursorMXBean;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound;
import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback;
import org.apache.bookkeeper.mledger.proto.MLDataFormats;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.common.util.collections.BitSetRecyclable;
import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet;
import org.apache.pulsar.common.util.collections.LongPairRangeSet;
import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer;
import org.apache.pulsar.metadata.api.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings("checkstyle:javadoctype")
public class ManagedCursorImpl implements ManagedCursor {
protected final BookKeeper bookkeeper;
protected final ManagedLedgerConfig config;
protected final ManagedLedgerImpl ledger;
private final String name;
private final BookKeeper.DigestType digestType;
protected volatile PositionImpl markDeletePosition;
// this position is have persistent mark delete position
protected volatile PositionImpl persistentMarkDeletePosition;
protected static final AtomicReferenceFieldUpdater<ManagedCursorImpl, PositionImpl> READ_POSITION_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, PositionImpl.class, "readPosition");
protected volatile PositionImpl readPosition;
// keeps sample of last read-position for validation and monitoring if read-position is not moving forward.
protected volatile PositionImpl statsLastReadPosition;
protected static final AtomicReferenceFieldUpdater<ManagedCursorImpl, MarkDeleteEntry> LAST_MARK_DELETE_ENTRY_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, MarkDeleteEntry.class, "lastMarkDeleteEntry");
protected volatile MarkDeleteEntry lastMarkDeleteEntry;
protected static final AtomicReferenceFieldUpdater<ManagedCursorImpl, OpReadEntry> WAITING_READ_OP_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, OpReadEntry.class, "waitingReadOp");
@SuppressWarnings("unused")
private volatile OpReadEntry waitingReadOp = null;
public static final int FALSE = 0;
public static final int TRUE = 1;
private static final AtomicIntegerFieldUpdater<ManagedCursorImpl> RESET_CURSOR_IN_PROGRESS_UPDATER =
AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "resetCursorInProgress");
@SuppressWarnings("unused")
private volatile int resetCursorInProgress = FALSE;
private static final AtomicIntegerFieldUpdater<ManagedCursorImpl> PENDING_READ_OPS_UPDATER =
AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "pendingReadOps");
@SuppressWarnings("unused")
private volatile int pendingReadOps = 0;
private static final AtomicLongFieldUpdater<ManagedCursorImpl> MSG_CONSUMED_COUNTER_UPDATER =
AtomicLongFieldUpdater.newUpdater(ManagedCursorImpl.class, "messagesConsumedCounter");
// This counters are used to compute the numberOfEntries and numberOfEntriesInBacklog values, without having to look
// at the list of ledgers in the ml. They are initialized to (-backlog) at opening, and will be incremented each
// time a message is read or deleted.
protected volatile long messagesConsumedCounter;
// Current ledger used to append the mark-delete position
private volatile LedgerHandle cursorLedger;
// Wether the current cursorLedger is read-only or writable
private boolean isCursorLedgerReadOnly = true;
// Stat of the cursor z-node
private volatile Stat cursorLedgerStat;
private static final LongPairConsumer<PositionImpl> positionRangeConverter = PositionImpl::new;
private static final LongPairConsumer<PositionImplRecyclable> recyclePositionRangeConverter = (key, value) -> {
PositionImplRecyclable position = PositionImplRecyclable.create();
position.ledgerId = key;
position.entryId = value;
position.ackSet = null;
return position;
};
private final LongPairRangeSet<PositionImpl> individualDeletedMessages;
// Maintain the deletion status for batch messages
// (ledgerId, entryId) -> deletion indexes
private final ConcurrentSkipListMap<PositionImpl, BitSetRecyclable> batchDeletedIndexes;
private final ReadWriteLock lock = new ReentrantReadWriteLock();
private RateLimiter markDeleteLimiter;
// The cursor is considered "dirty" when there are mark-delete updates that are only applied in memory,
// because of the rate limiting.
private volatile boolean isDirty = false;
private boolean alwaysInactive = false;
/** used temporary variables to {@link #getNumIndividualDeletedEntriesToSkip(long)} **/
private static final FastThreadLocal<Long> tempTotalEntriesToSkip = new FastThreadLocal<>();
private static final FastThreadLocal<Long> tempDeletedMessages = new FastThreadLocal<>();
private static final FastThreadLocal<PositionImpl> tempStartPosition = new FastThreadLocal<>();
private static final FastThreadLocal<PositionImpl> tempEndPosition = new FastThreadLocal<>();
private static final long NO_MAX_SIZE_LIMIT = -1L;
private long entriesReadCount;
private long entriesReadSize;
private int individualDeletedMessagesSerializedSize;
class MarkDeleteEntry {
final PositionImpl newPosition;
final MarkDeleteCallback callback;
final Object ctx;
final Map<String, Long> properties;
// If the callbackGroup is set, it means this mark-delete request was done on behalf of a group of request (just
// persist the last one in the chain). In this case we need to trigger the callbacks for every request in the
// group.
List<MarkDeleteEntry> callbackGroup;
public MarkDeleteEntry(PositionImpl newPosition, Map<String, Long> properties,
MarkDeleteCallback callback, Object ctx) {
this.newPosition = newPosition;
this.properties = properties;
this.callback = callback;
this.ctx = ctx;
}
}
protected final ArrayDeque<MarkDeleteEntry> pendingMarkDeleteOps = new ArrayDeque<>();
private static final AtomicIntegerFieldUpdater<ManagedCursorImpl> PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER =
AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "pendingMarkDeletedSubmittedCount");
@SuppressWarnings("unused")
private volatile int pendingMarkDeletedSubmittedCount = 0;
private long lastLedgerSwitchTimestamp;
private final Clock clock;
// The last active time (Unix time, milliseconds) of the cursor
private long lastActive;
enum State {
Uninitialized, // Cursor is being initialized
NoLedger, // There is no metadata ledger open for writing
Open, // Metadata ledger is ready
SwitchingLedger, // The metadata ledger is being switched
Closing, // The managed cursor is closing
Closed // The managed cursor has been closed
}
private static final AtomicReferenceFieldUpdater<ManagedCursorImpl, State> STATE_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, State.class, "state");
protected volatile State state = null;
protected final ManagedCursorMXBean mbean;
@SuppressWarnings("checkstyle:javadoctype")
public interface VoidCallback {
void operationComplete();
void operationFailed(ManagedLedgerException exception);
}
ManagedCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, ManagedLedgerImpl ledger, String cursorName) {
this.bookkeeper = bookkeeper;
this.config = config;
this.ledger = ledger;
this.name = cursorName;
this.individualDeletedMessages = config.isUnackedRangesOpenCacheSetEnabled()
? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter)
: new LongPairRangeSet.DefaultRangeSet<>(positionRangeConverter);
if (config.isDeletionAtBatchIndexLevelEnabled()) {
this.batchDeletedIndexes = new ConcurrentSkipListMap<>();
} else {
this.batchDeletedIndexes = null;
}
this.digestType = BookKeeper.DigestType.fromApiDigestType(config.getDigestType());
STATE_UPDATER.set(this, State.Uninitialized);
PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.set(this, 0);
PENDING_READ_OPS_UPDATER.set(this, 0);
RESET_CURSOR_IN_PROGRESS_UPDATER.set(this, FALSE);
WAITING_READ_OP_UPDATER.set(this, null);
this.clock = config.getClock();
this.lastActive = this.clock.millis();
this.lastLedgerSwitchTimestamp = this.clock.millis();
if (config.getThrottleMarkDelete() > 0.0) {
markDeleteLimiter = RateLimiter.create(config.getThrottleMarkDelete());
} else {
// Disable mark-delete rate limiter
markDeleteLimiter = null;
}
this.mbean = new ManagedCursorMXBeanImpl(this);
}
@Override
public Map<String, Long> getProperties() {
return lastMarkDeleteEntry != null ? lastMarkDeleteEntry.properties : Collections.emptyMap();
}
@Override
public boolean putProperty(String key, Long value) {
if (lastMarkDeleteEntry != null) {
LAST_MARK_DELETE_ENTRY_UPDATER.updateAndGet(this, last -> {
Map<String, Long> properties = last.properties;
Map<String, Long> newProperties = properties == null ? Maps.newHashMap() : Maps.newHashMap(properties);
newProperties.put(key, value);
MarkDeleteEntry newLastMarkDeleteEntry = new MarkDeleteEntry(last.newPosition, newProperties,
last.callback, last.ctx);
newLastMarkDeleteEntry.callbackGroup = last.callbackGroup;
return newLastMarkDeleteEntry;
});
return true;
}
return false;
}
@Override
public boolean removeProperty(String key) {
if (lastMarkDeleteEntry != null) {
LAST_MARK_DELETE_ENTRY_UPDATER.updateAndGet(this, last -> {
Map<String, Long> properties = last.properties;
if (properties != null && properties.containsKey(key)) {
properties.remove(key);
}
return last;
});
return true;
}
return false;
}
/**
* Performs the initial recovery, reading the mark-deleted position from the ledger and then calling initialize to
* have a new opened ledger.
*/
void recover(final VoidCallback callback) {
// Read the meta-data ledgerId from the store
log.info("[{}] Recovering from bookkeeper ledger cursor: {}", ledger.getName(), name);
ledger.getStore().asyncGetCursorInfo(ledger.getName(), name, new MetaStoreCallback<ManagedCursorInfo>() {
@Override
public void operationComplete(ManagedCursorInfo info, Stat stat) {
cursorLedgerStat = stat;
lastActive = info.getLastActive() != 0 ? info.getLastActive() : lastActive;
if (info.getCursorsLedgerId() == -1L) {
// There is no cursor ledger to read the last position from. It means the cursor has been properly
// closed and the last mark-delete position is stored in the ManagedCursorInfo itself.
PositionImpl recoveredPosition = new PositionImpl(info.getMarkDeleteLedgerId(),
info.getMarkDeleteEntryId());
if (info.getIndividualDeletedMessagesCount() > 0) {
recoverIndividualDeletedMessages(info.getIndividualDeletedMessagesList());
}
Map<String, Long> recoveredProperties = Collections.emptyMap();
if (info.getPropertiesCount() > 0) {
// Recover properties map
recoveredProperties = Maps.newHashMap();
for (int i = 0; i < info.getPropertiesCount(); i++) {
LongProperty property = info.getProperties(i);
recoveredProperties.put(property.getName(), property.getValue());
}
}
recoveredCursor(recoveredPosition, recoveredProperties, null);
callback.operationComplete();
} else {
// Need to proceed and read the last entry in the specified ledger to find out the last position
log.info("[{}] Consumer {} meta-data recover from ledger {}", ledger.getName(), name,
info.getCursorsLedgerId());
recoverFromLedger(info, callback);
}
}
@Override
public void operationFailed(MetaStoreException e) {
callback.operationFailed(e);
}
});
}
protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallback callback) {
// Read the acknowledged position from the metadata ledger, then create
// a new ledger and write the position into it
ledger.mbean.startCursorLedgerOpenOp();
long ledgerId = info.getCursorsLedgerId();
OpenCallback openCallback = (rc, lh, ctx) -> {
if (log.isInfoEnabled()) {
log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc);
}
if (isBkErrorNotRecoverable(rc)) {
log.error("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name,
BKException.getMessage(rc));
// Rewind to oldest entry available
initialize(getRollbackPosition(info), Collections.emptyMap(), callback);
return;
} else if (rc != BKException.Code.OK) {
log.warn("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name,
BKException.getMessage(rc));
callback.operationFailed(new ManagedLedgerException(BKException.getMessage(rc)));
return;
}
// Read the last entry in the ledger
long lastEntryInLedger = lh.getLastAddConfirmed();
if (lastEntryInLedger < 0) {
log.warn("[{}] Error reading from metadata ledger {} for consumer {}: No entries in ledger",
ledger.getName(), ledgerId, name);
// Rewind to last cursor snapshot available
initialize(getRollbackPosition(info), Collections.emptyMap(), callback);
return;
}
lh.asyncReadEntries(lastEntryInLedger, lastEntryInLedger, (rc1, lh1, seq, ctx1) -> {
if (log.isDebugEnabled()) {
log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed());
}
if (isBkErrorNotRecoverable(rc1)) {
log.error("[{}] Error reading from metadata ledger {} for consumer {}: {}", ledger.getName(),
ledgerId, name, BKException.getMessage(rc1));
// Rewind to oldest entry available
initialize(getRollbackPosition(info), Collections.emptyMap(), callback);
return;
} else if (rc1 != BKException.Code.OK) {
log.warn("[{}] Error reading from metadata ledger {} for consumer {}: {}", ledger.getName(),
ledgerId, name, BKException.getMessage(rc1));
callback.operationFailed(createManagedLedgerException(rc1));
return;
}
LedgerEntry entry = seq.nextElement();
PositionInfo positionInfo;
try {
positionInfo = PositionInfo.parseFrom(entry.getEntry());
} catch (InvalidProtocolBufferException e) {
callback.operationFailed(new ManagedLedgerException(e));
return;
}
Map<String, Long> recoveredProperties = Collections.emptyMap();
if (positionInfo.getPropertiesCount() > 0) {
// Recover properties map
recoveredProperties = Maps.newHashMap();
for (int i = 0; i < positionInfo.getPropertiesCount(); i++) {
LongProperty property = positionInfo.getProperties(i);
recoveredProperties.put(property.getName(), property.getValue());
}
}
PositionImpl position = new PositionImpl(positionInfo);
if (positionInfo.getIndividualDeletedMessagesCount() > 0) {
recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList());
}
if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null
&& positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) {
recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList());
}
recoveredCursor(position, recoveredProperties, lh);
callback.operationComplete();
}, null);
};
try {
bookkeeper.asyncOpenLedger(ledgerId, digestType, config.getPassword(), openCallback, null);
} catch (Throwable t) {
log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}",
ledger.getName(), ledgerId, name, t);
openCallback.openComplete(BKException.Code.UnexpectedConditionException, null, null);
}
}
private void recoverIndividualDeletedMessages(List<MLDataFormats.MessageRange> individualDeletedMessagesList) {
lock.writeLock().lock();
try {
individualDeletedMessages.clear();
individualDeletedMessagesList.forEach(messageRange -> {
MLDataFormats.NestedPositionInfo lowerEndpoint = messageRange.getLowerEndpoint();
MLDataFormats.NestedPositionInfo upperEndpoint = messageRange.getUpperEndpoint();
if (lowerEndpoint.getLedgerId() == upperEndpoint.getLedgerId()) {
individualDeletedMessages.addOpenClosed(lowerEndpoint.getLedgerId(), lowerEndpoint.getEntryId(),
upperEndpoint.getLedgerId(), upperEndpoint.getEntryId());
} else {
// Store message ranges after splitting them by ledger ID
LedgerInfo lowerEndpointLedgerInfo = ledger.getLedgersInfo().get(lowerEndpoint.getLedgerId());
if (lowerEndpointLedgerInfo != null) {
individualDeletedMessages.addOpenClosed(lowerEndpoint.getLedgerId(), lowerEndpoint.getEntryId(),
lowerEndpoint.getLedgerId(), lowerEndpointLedgerInfo.getEntries() - 1);
} else {
log.warn("[{}][{}] No ledger info of lower endpoint {}:{}", ledger.getName(), name,
lowerEndpoint.getLedgerId(), lowerEndpoint.getEntryId());
}
for (LedgerInfo li : ledger.getLedgersInfo()
.subMap(lowerEndpoint.getLedgerId(), false, upperEndpoint.getLedgerId(), false).values()) {
individualDeletedMessages.addOpenClosed(li.getLedgerId(), -1, li.getLedgerId(),
li.getEntries() - 1);
}
individualDeletedMessages.addOpenClosed(upperEndpoint.getLedgerId(), -1,
upperEndpoint.getLedgerId(), upperEndpoint.getEntryId());
}
});
} finally {
lock.writeLock().unlock();
}
}
private void recoverBatchDeletedIndexes (List<MLDataFormats.BatchedEntryDeletionIndexInfo> batchDeletedIndexInfoList) {
lock.writeLock().lock();
try {
this.batchDeletedIndexes.clear();
batchDeletedIndexInfoList.forEach(batchDeletedIndexInfo -> {
if (batchDeletedIndexInfo.getDeleteSetCount() > 0) {
long[] array = new long[batchDeletedIndexInfo.getDeleteSetCount()];
for (int i = 0; i < batchDeletedIndexInfo.getDeleteSetList().size(); i++) {
array[i] = batchDeletedIndexInfo.getDeleteSetList().get(i);
}
this.batchDeletedIndexes.put(PositionImpl.get(batchDeletedIndexInfo.getPosition().getLedgerId(),
batchDeletedIndexInfo.getPosition().getEntryId()), BitSetRecyclable.create().resetWords(array));
}
});
} finally {
lock.writeLock().unlock();
}
}
private void recoveredCursor(PositionImpl position, Map<String, Long> properties,
LedgerHandle recoveredFromCursorLedger) {
// if the position was at a ledger that didn't exist (since it will be deleted if it was previously empty),
// we need to move to the next existing ledger
if (!ledger.ledgerExists(position.getLedgerId())) {
Long nextExistingLedger = ledger.getNextValidLedger(position.getLedgerId());
if (nextExistingLedger == null) {
log.info("[{}] [{}] Couldn't find next next valid ledger for recovery {}", ledger.getName(), name,
position);
}
position = nextExistingLedger != null ? PositionImpl.get(nextExistingLedger, -1) : position;
}
if (position.compareTo(ledger.getLastPosition()) > 0) {
log.warn("[{}] [{}] Current position {} is ahead of last position {}", ledger.getName(), name, position,
ledger.getLastPosition());
position = ledger.getLastPosition();
}
log.info("[{}] Cursor {} recovered to position {}", ledger.getName(), name, position);
messagesConsumedCounter = -getNumberOfEntries(Range.openClosed(position, ledger.getLastPosition()));
markDeletePosition = position;
persistentMarkDeletePosition = position;
readPosition = ledger.getNextValidPosition(position);
lastMarkDeleteEntry = new MarkDeleteEntry(markDeletePosition, properties, null, null);
// assign cursor-ledger so, it can be deleted when new ledger will be switched
this.cursorLedger = recoveredFromCursorLedger;
this.isCursorLedgerReadOnly = true;
STATE_UPDATER.set(this, State.NoLedger);
}
void initialize(PositionImpl position, Map<String, Long> properties, final VoidCallback callback) {
recoveredCursor(position, properties, null);
if (log.isDebugEnabled()) {
log.debug("[{}] Consumer {} cursor initialized with counters: consumed {} mdPos {} rdPos {}",
ledger.getName(), name, messagesConsumedCounter, markDeletePosition, readPosition);
}
createNewMetadataLedger(new VoidCallback() {
@Override
public void operationComplete() {
STATE_UPDATER.set(ManagedCursorImpl.this, State.Open);
callback.operationComplete();
}
@Override
public void operationFailed(ManagedLedgerException exception) {
callback.operationFailed(exception);
}
});
}
@Override
public List<Entry> readEntries(int numberOfEntriesToRead) throws InterruptedException, ManagedLedgerException {
checkArgument(numberOfEntriesToRead > 0);
final CountDownLatch counter = new CountDownLatch(1);
class Result {
ManagedLedgerException exception = null;
List<Entry> entries = null;
}
final Result result = new Result();
asyncReadEntries(numberOfEntriesToRead, new ReadEntriesCallback() {
@Override
public void readEntriesComplete(List<Entry> entries, Object ctx) {
result.entries = entries;
counter.countDown();
}
@Override
public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
result.exception = exception;
counter.countDown();
}
}, null, PositionImpl.latest);
counter.await();
if (result.exception != null) {
throw result.exception;
}
return result.entries;
}
@Override
public void asyncReadEntries(final int numberOfEntriesToRead, final ReadEntriesCallback callback,
final Object ctx, PositionImpl maxPosition) {
asyncReadEntries(numberOfEntriesToRead, NO_MAX_SIZE_LIMIT, callback, ctx, maxPosition);
}
@Override
public void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback,
Object ctx, PositionImpl maxPosition) {
checkArgument(numberOfEntriesToRead > 0);
if (isClosed()) {
callback.readEntriesFailed(new ManagedLedgerException
.CursorAlreadyClosedException("Cursor was already closed"), ctx);
return;
}
int numOfEntriesToRead = applyMaxSizeCap(numberOfEntriesToRead, maxSizeBytes);
PENDING_READ_OPS_UPDATER.incrementAndGet(this);
OpReadEntry op = OpReadEntry.create(this, readPosition, numOfEntriesToRead, callback, ctx, maxPosition);
ledger.asyncReadEntries(op);
}
@Override
public Entry getNthEntry(int n, IndividualDeletedEntries deletedEntries)
throws InterruptedException, ManagedLedgerException {
final CountDownLatch counter = new CountDownLatch(1);
class Result {
ManagedLedgerException exception = null;
Entry entry = null;
}
final Result result = new Result();
asyncGetNthEntry(n, deletedEntries, new ReadEntryCallback() {
@Override
public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
result.exception = exception;
counter.countDown();
}
@Override
public void readEntryComplete(Entry entry, Object ctx) {
result.entry = entry;
counter.countDown();
}
}, null);
counter.await(ledger.getConfig().getMetadataOperationsTimeoutSeconds(), TimeUnit.SECONDS);
if (result.exception != null) {
throw result.exception;
}
return result.entry;
}
@Override
public void asyncGetNthEntry(int n, IndividualDeletedEntries deletedEntries, ReadEntryCallback callback,
Object ctx) {
checkArgument(n > 0);
if (isClosed()) {
callback.readEntryFailed(new ManagedLedgerException
.CursorAlreadyClosedException("Cursor was already closed"), ctx);
return;
}
PositionImpl startPosition = ledger.getNextValidPosition(markDeletePosition);
PositionImpl endPosition = ledger.getLastPosition();
if (startPosition.compareTo(endPosition) <= 0) {
long numOfEntries = getNumberOfEntries(Range.closed(startPosition, endPosition));
if (numOfEntries >= n) {
long deletedMessages = 0;
if (deletedEntries == IndividualDeletedEntries.Exclude) {
deletedMessages = getNumIndividualDeletedEntriesToSkip(n);
}
PositionImpl positionAfterN = ledger.getPositionAfterN(markDeletePosition, n + deletedMessages,
PositionBound.startExcluded);
ledger.asyncReadEntry(positionAfterN, callback, ctx);
} else {
callback.readEntryComplete(null, ctx);
}
} else {
callback.readEntryComplete(null, ctx);
}
}
@Override
public List<Entry> readEntriesOrWait(int numberOfEntriesToRead)
throws InterruptedException, ManagedLedgerException {
return readEntriesOrWait(numberOfEntriesToRead, NO_MAX_SIZE_LIMIT);
}
@Override
public List<Entry> readEntriesOrWait(int numberOfEntriesToRead, long maxSizeBytes)
throws InterruptedException, ManagedLedgerException {
checkArgument(numberOfEntriesToRead > 0);
final CountDownLatch counter = new CountDownLatch(1);
class Result {
ManagedLedgerException exception = null;
List<Entry> entries = null;
}
final Result result = new Result();
asyncReadEntriesOrWait(numberOfEntriesToRead, maxSizeBytes, new ReadEntriesCallback() {
@Override
public void readEntriesComplete(List<Entry> entries, Object ctx) {
result.entries = entries;
counter.countDown();
}
@Override
public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
result.exception = exception;
counter.countDown();
}
}, null, PositionImpl.latest);
counter.await();
if (result.exception != null) {
throw result.exception;
}
return result.entries;
}
@Override
public void asyncReadEntriesOrWait(int numberOfEntriesToRead, ReadEntriesCallback callback, Object ctx,
PositionImpl maxPosition) {
asyncReadEntriesOrWait(numberOfEntriesToRead, NO_MAX_SIZE_LIMIT, callback, ctx, maxPosition);
}
@Override
public void asyncReadEntriesOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallback callback, Object ctx,
PositionImpl maxPosition) {
checkArgument(maxEntries > 0);
if (isClosed()) {
callback.readEntriesFailed(new CursorAlreadyClosedException("Cursor was already closed"), ctx);
return;
}
int numberOfEntriesToRead = applyMaxSizeCap(maxEntries, maxSizeBytes);
if (hasMoreEntries()) {
// If we have available entries, we can read them immediately
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Read entries immediately", ledger.getName(), name);
}
asyncReadEntries(numberOfEntriesToRead, callback, ctx, maxPosition);
} else {
OpReadEntry op = OpReadEntry.create(this, readPosition, numberOfEntriesToRead, callback,
ctx, maxPosition);
if (!WAITING_READ_OP_UPDATER.compareAndSet(this, null, op)) {
callback.readEntriesFailed(new ManagedLedgerException("We can only have a single waiting callback"),
ctx);
return;
}
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Deferring retry of read at position {}", ledger.getName(), name, op.readPosition);
}
// Check again for new entries after the configured time, then if still no entries are available register
// to be notified
if (config.getNewEntriesCheckDelayInMillis() > 0) {
ledger.getScheduledExecutor()
.schedule(() -> checkForNewEntries(op, callback, ctx),
config.getNewEntriesCheckDelayInMillis(), TimeUnit.MILLISECONDS);
} else {
// If there's no delay, check directly from the same thread
checkForNewEntries(op, callback, ctx);
}
}
}
private void checkForNewEntries(OpReadEntry op, ReadEntriesCallback callback, Object ctx) {
try {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Re-trying the read at position {}", ledger.getName(), name, op.readPosition);
}
if (!hasMoreEntries()) {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Still no entries available. Register for notification", ledger.getName(),
name);
}
// Let the managed ledger know we want to be notified whenever a new entry is published
ledger.waitingCursors.add(this);
} else {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Skip notification registering since we do have entries available",
ledger.getName(), name);
}
}
// Check again the entries count, since an entry could have been written between the time we
// checked and the time we've asked to be notified by managed ledger
if (hasMoreEntries()) {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Found more entries", ledger.getName(), name);
}
// Try to cancel the notification request
if (WAITING_READ_OP_UPDATER.compareAndSet(this, op, null)) {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Cancelled notification and scheduled read at {}", ledger.getName(),
name, op.readPosition);
}
PENDING_READ_OPS_UPDATER.incrementAndGet(this);
ledger.asyncReadEntries(op);
} else {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] notification was already cancelled", ledger.getName(), name);
}
}
} else if (ledger.isTerminated()) {
// At this point we registered for notification and still there were no more available
// entries.
// If the managed ledger was indeed terminated, we need to notify the cursor
callback.readEntriesFailed(new NoMoreEntriesToReadException("Topic was terminated"), ctx);
}
} catch (Throwable t) {
callback.readEntriesFailed(new ManagedLedgerException(t), ctx);
}
}
public boolean isClosed() {
return state == State.Closed || state == State.Closing;
}
@Override
public boolean cancelPendingReadRequest() {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Cancel pending read request", ledger.getName(), name);
}
return WAITING_READ_OP_UPDATER.getAndSet(this, null) != null;
}
public boolean hasPendingReadRequest() {
return WAITING_READ_OP_UPDATER.get(this) != null;
}
@Override
public boolean hasMoreEntries() {
// If writer and reader are on the same ledger, we just need to compare the entry id to know if we have more
// entries.
// If they are on different ledgers we have 2 cases :
// * Writer pointing to valid entry --> should return true since we have available entries
// * Writer pointing to "invalid" entry -1 (meaning no entries in that ledger) --> Need to check if the reader
// is
// at the last entry in the previous ledger
PositionImpl writerPosition = ledger.getLastPosition();
if (writerPosition.getEntryId() != -1) {
return readPosition.compareTo(writerPosition) <= 0;
} else {
// Fall back to checking the number of entries to ensure we are at the last entry in ledger and no ledgers
// are in the middle
return getNumberOfEntries() > 0;
}
}
@Override
public long getNumberOfEntries() {
if (readPosition.compareTo(ledger.getLastPosition().getNext()) > 0) {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Read position {} is ahead of last position {}. There are no entries to read",
ledger.getName(), name, readPosition, ledger.getLastPosition());
}
return 0;
} else {
return getNumberOfEntries(Range.closedOpen(readPosition, ledger.getLastPosition().getNext()));
}
}
@Override
public long getNumberOfEntriesSinceFirstNotAckedMessage() {
// sometimes for already caught up consumer: due to race condition markDeletePosition > readPosition. so,
// validate it before preparing range
PositionImpl markDeletePosition = this.markDeletePosition;
PositionImpl readPosition = this.readPosition;
return (markDeletePosition != null && readPosition != null && markDeletePosition.compareTo(readPosition) < 0)
? ledger.getNumberOfEntries(Range.openClosed(markDeletePosition, readPosition))
: 0;
}
@Override
public int getTotalNonContiguousDeletedMessagesRange() {
return individualDeletedMessages.size();
}
@Override
public int getNonContiguousDeletedMessagesRangeSerializedSize() {
return this.individualDeletedMessagesSerializedSize;
}
@Override
public long getEstimatedSizeSinceMarkDeletePosition() {
return ledger.estimateBacklogFromPosition(markDeletePosition);
}
@Override
public long getNumberOfEntriesInBacklog(boolean isPrecise) {
if (log.isDebugEnabled()) {
log.debug("[{}] Consumer {} cursor ml-entries: {} -- deleted-counter: {} other counters: mdPos {} rdPos {}",
ledger.getName(), name, ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER.get(ledger),
messagesConsumedCounter, markDeletePosition, readPosition);
}
if (isPrecise) {
return getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition()));
}
long backlog = ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER.get(ledger) - messagesConsumedCounter;
if (backlog < 0) {
// In some case the counters get incorrect values, fall back to the precise backlog count
backlog = getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition()));
}
return backlog;
}
public long getNumberOfEntriesInStorage() {
return ledger.getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition().getNext()));
}
@Override
public Position findNewestMatching(Predicate<Entry> condition) throws InterruptedException, ManagedLedgerException {
return findNewestMatching(FindPositionConstraint.SearchActiveEntries, condition);
}
@Override
public Position findNewestMatching(FindPositionConstraint constraint, Predicate<Entry> condition) throws InterruptedException, ManagedLedgerException {
final CountDownLatch counter = new CountDownLatch(1);
class Result {
ManagedLedgerException exception = null;
Position position = null;
}
final Result result = new Result();
asyncFindNewestMatching(constraint, condition, new FindEntryCallback() {
@Override
public void findEntryComplete(Position position, Object ctx) {
result.position = position;
counter.countDown();
}
@Override
public void findEntryFailed(ManagedLedgerException exception, Optional<Position> failedReadPosition,
Object ctx) {
result.exception = exception;
counter.countDown();
}
}, null);
counter.await();
if (result.exception != null) {
throw result.exception;
}
return result.position;
}
@Override
public void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate<Entry> condition,
FindEntryCallback callback, Object ctx) {
OpFindNewest op;
PositionImpl startPosition = null;
long max = 0;
switch (constraint) {
case SearchAllAvailableEntries:
startPosition = (PositionImpl) getFirstPosition();
max = ledger.getNumberOfEntries() - 1;
break;
case SearchActiveEntries:
startPosition = ledger.getNextValidPosition(markDeletePosition);
max = getNumberOfEntriesInStorage();
break;
default:
callback.findEntryFailed(new ManagedLedgerException("Unknown position constraint"), Optional.empty(), ctx);
return;
}
if (startPosition == null) {
callback.findEntryFailed(new ManagedLedgerException("Couldn't find start position"),
Optional.empty(), ctx);
return;
}
op = new OpFindNewest(this, startPosition, condition, max, callback, ctx);
op.find();
}