-
Notifications
You must be signed in to change notification settings - Fork 3.5k
/
ManagedLedgerFactoryImpl.java
1083 lines (959 loc) · 50.8 KB
/
ManagedLedgerFactoryImpl.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 org.apache.bookkeeper.mledger.ManagedLedgerException.getManagedLedgerException;
import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.NULL_OFFLOAD_PROMISE;
import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables;
import com.google.common.base.Predicates;
import com.google.common.collect.Maps;
import io.netty.util.concurrent.DefaultThreadFactory;
import io.opentelemetry.api.OpenTelemetry;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import lombok.Getter;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.common.util.OrderedScheduler;
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteLedgerCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.ManagedLedgerInfoCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenReadOnlyCursorCallback;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException;
import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig;
import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean;
import org.apache.bookkeeper.mledger.ManagedLedgerInfo;
import org.apache.bookkeeper.mledger.ManagedLedgerInfo.CursorInfo;
import org.apache.bookkeeper.mledger.ManagedLedgerInfo.LedgerInfo;
import org.apache.bookkeeper.mledger.ManagedLedgerInfo.MessageRangeInfo;
import org.apache.bookkeeper.mledger.ManagedLedgerInfo.PositionInfo;
import org.apache.bookkeeper.mledger.MetadataCompressionConfig;
import org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.ReadOnlyCursor;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ManagedLedgerInitializeLedgerCallback;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.State;
import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback;
import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager;
import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl;
import org.apache.bookkeeper.mledger.offload.OffloadUtils;
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.MessageRange;
import org.apache.bookkeeper.mledger.util.Futures;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig;
import org.apache.pulsar.common.util.DateFormatter;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.Runnables;
import org.apache.pulsar.metadata.api.MetadataStore;
import org.apache.pulsar.metadata.api.Stat;
import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
import org.apache.pulsar.metadata.api.extended.SessionEvent;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ManagedLedgerFactoryImpl implements ManagedLedgerFactory {
private final MetaStore store;
private final BookkeeperFactoryForCustomEnsemblePlacementPolicy bookkeeperFactory;
private final boolean isBookkeeperManaged;
private final ManagedLedgerFactoryConfig config;
@Getter
protected final OrderedScheduler scheduledExecutor;
private final ScheduledExecutorService cacheEvictionExecutor;
@Getter
protected final ManagedLedgerFactoryMBeanImpl mbean;
protected final ConcurrentHashMap<String, CompletableFuture<ManagedLedgerImpl>> ledgers = new ConcurrentHashMap<>();
protected final ConcurrentHashMap<String, PendingInitializeManagedLedger> pendingInitializeLedgers =
new ConcurrentHashMap<>();
private final EntryCacheManager entryCacheManager;
private long lastStatTimestamp = System.nanoTime();
private final ScheduledFuture<?> statsTask;
private final ScheduledFuture<?> flushCursorsTask;
private volatile long cacheEvictionTimeThresholdNanos;
private final MetadataStore metadataStore;
private final OpenTelemetryManagedLedgerCacheStats openTelemetryCacheStats;
//indicate whether shutdown() is called.
private volatile boolean closed;
/**
* Keep a flag to indicate whether we're currently connected to the metadata service.
*/
@Getter
private boolean metadataServiceAvailable;
private static class PendingInitializeManagedLedger {
private final ManagedLedgerImpl ledger;
private final long createTimeMs;
PendingInitializeManagedLedger(ManagedLedgerImpl ledger) {
this.ledger = ledger;
this.createTimeMs = System.currentTimeMillis();
}
}
public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, ClientConfiguration bkClientConfiguration)
throws Exception {
this(metadataStore, bkClientConfiguration, new ManagedLedgerFactoryConfig());
}
@SuppressWarnings("deprecation")
public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, ClientConfiguration bkClientConfiguration,
ManagedLedgerFactoryConfig config)
throws Exception {
this(metadataStore, new DefaultBkFactory(bkClientConfiguration),
true /* isBookkeeperManaged */, config, NullStatsLogger.INSTANCE, OpenTelemetry.noop());
}
public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper bookKeeper)
throws Exception {
this(metadataStore, bookKeeper, new ManagedLedgerFactoryConfig());
}
public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper bookKeeper,
ManagedLedgerFactoryConfig config)
throws Exception {
this(metadataStore, (policyConfig) -> CompletableFuture.completedFuture(bookKeeper), config);
}
public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore,
BookkeeperFactoryForCustomEnsemblePlacementPolicy bookKeeperGroupFactory,
ManagedLedgerFactoryConfig config)
throws Exception {
this(metadataStore, bookKeeperGroupFactory, false /* isBookkeeperManaged */,
config, NullStatsLogger.INSTANCE, OpenTelemetry.noop());
}
public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore,
BookkeeperFactoryForCustomEnsemblePlacementPolicy bookKeeperGroupFactory,
ManagedLedgerFactoryConfig config, StatsLogger statsLogger,
OpenTelemetry openTelemetry)
throws Exception {
this(metadataStore, bookKeeperGroupFactory, false /* isBookkeeperManaged */,
config, statsLogger, openTelemetry);
}
private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore,
BookkeeperFactoryForCustomEnsemblePlacementPolicy bookKeeperGroupFactory,
boolean isBookkeeperManaged,
ManagedLedgerFactoryConfig config,
StatsLogger statsLogger,
OpenTelemetry openTelemetry) throws Exception {
MetadataCompressionConfig compressionConfigForManagedLedgerInfo =
config.getCompressionConfigForManagedLedgerInfo();
MetadataCompressionConfig compressionConfigForManagedCursorInfo =
config.getCompressionConfigForManagedCursorInfo();
scheduledExecutor = OrderedScheduler.newSchedulerBuilder()
.numThreads(config.getNumManagedLedgerSchedulerThreads())
.statsLogger(statsLogger)
.traceTaskExecution(config.isTraceTaskExecution())
.name("bookkeeper-ml-scheduler")
.build();
cacheEvictionExecutor = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("bookkeeper-ml-cache-eviction"));
this.metadataServiceAvailable = true;
this.bookkeeperFactory = bookKeeperGroupFactory;
this.isBookkeeperManaged = isBookkeeperManaged;
this.metadataStore = metadataStore;
this.store = new MetaStoreImpl(metadataStore, scheduledExecutor,
compressionConfigForManagedLedgerInfo,
compressionConfigForManagedCursorInfo);
this.config = config;
this.mbean = new ManagedLedgerFactoryMBeanImpl(this);
this.entryCacheManager = new RangeEntryCacheManagerImpl(this);
this.statsTask = scheduledExecutor.scheduleWithFixedDelay(catchingAndLoggingThrowables(this::refreshStats),
0, config.getStatsPeriodSeconds(), TimeUnit.SECONDS);
this.flushCursorsTask = scheduledExecutor.scheduleAtFixedRate(catchingAndLoggingThrowables(this::flushCursors),
config.getCursorPositionFlushSeconds(), config.getCursorPositionFlushSeconds(), TimeUnit.SECONDS);
this.cacheEvictionTimeThresholdNanos = TimeUnit.MILLISECONDS
.toNanos(config.getCacheEvictionTimeThresholdMillis());
long evictionTaskInterval = config.getCacheEvictionIntervalMs();
cacheEvictionExecutor.scheduleWithFixedDelay(Runnables.catchingAndLoggingThrowables(this::doCacheEviction),
evictionTaskInterval, evictionTaskInterval, TimeUnit.MILLISECONDS);
closed = false;
metadataStore.registerSessionListener(this::handleMetadataStoreNotification);
openTelemetryCacheStats = new OpenTelemetryManagedLedgerCacheStats(openTelemetry, this);
}
static class DefaultBkFactory implements BookkeeperFactoryForCustomEnsemblePlacementPolicy {
private final BookKeeper bkClient;
public DefaultBkFactory(ClientConfiguration bkClientConfiguration)
throws InterruptedException, BKException, IOException {
bkClient = new BookKeeper(bkClientConfiguration);
}
@Override
public CompletableFuture<BookKeeper> get(EnsemblePlacementPolicyConfig policy) {
return CompletableFuture.completedFuture(bkClient);
}
}
private synchronized void handleMetadataStoreNotification(SessionEvent e) {
log.info("Received MetadataStore session event: {}", e);
metadataServiceAvailable = e.isConnected();
}
private synchronized void flushCursors() {
ledgers.values().forEach(mlfuture -> {
if (mlfuture.isDone() && !mlfuture.isCompletedExceptionally()) {
ManagedLedgerImpl ml = mlfuture.getNow(null);
if (ml != null) {
ml.getCursors().forEach(c -> ((ManagedCursorImpl) c).flush());
}
}
});
}
private synchronized void refreshStats() {
long now = System.nanoTime();
long period = now - lastStatTimestamp;
mbean.refreshStats(period, TimeUnit.NANOSECONDS);
ledgers.values().forEach(mlfuture -> {
if (mlfuture.isDone() && !mlfuture.isCompletedExceptionally()) {
ManagedLedgerImpl ml = mlfuture.getNow(null);
if (ml != null) {
ml.mbean.refreshStats(period, TimeUnit.NANOSECONDS);
}
}
});
lastStatTimestamp = now;
}
private synchronized void doCacheEviction() {
long maxTimestamp = System.nanoTime() - cacheEvictionTimeThresholdNanos;
ledgers.values().forEach(mlfuture -> {
if (mlfuture.isDone() && !mlfuture.isCompletedExceptionally()) {
ManagedLedgerImpl ml = mlfuture.getNow(null);
if (ml != null) {
ml.doCacheEviction(maxTimestamp);
}
}
});
}
/**
* Helper for getting stats.
*
* @return
*/
public Map<String, ManagedLedgerImpl> getManagedLedgers() {
// Return a view of already created ledger by filtering futures not yet completed
return Maps.filterValues(Maps.transformValues(ledgers, future -> future.getNow(null)), Predicates.notNull());
}
@Override
public ManagedLedger open(String name) throws InterruptedException, ManagedLedgerException {
return open(name, new ManagedLedgerConfig());
}
@Override
public ManagedLedger open(String name, ManagedLedgerConfig config)
throws InterruptedException, ManagedLedgerException {
class Result {
ManagedLedger l = null;
ManagedLedgerException e = null;
}
final Result r = new Result();
final CountDownLatch latch = new CountDownLatch(1);
asyncOpen(name, config, new OpenLedgerCallback() {
@Override
public void openLedgerComplete(ManagedLedger ledger, Object ctx) {
r.l = ledger;
latch.countDown();
}
@Override
public void openLedgerFailed(ManagedLedgerException exception, Object ctx) {
r.e = exception;
latch.countDown();
}
}, null, null);
latch.await();
if (r.e != null) {
throw r.e;
}
return r.l;
}
@Override
public void asyncOpen(String name, OpenLedgerCallback callback, Object ctx) {
asyncOpen(name, new ManagedLedgerConfig(), callback, null, ctx);
}
@Override
public void asyncOpen(final String name, final ManagedLedgerConfig config, final OpenLedgerCallback callback,
Supplier<CompletableFuture<Boolean>> mlOwnershipChecker, final Object ctx) {
if (closed) {
callback.openLedgerFailed(new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx);
return;
}
// If the ledger state is bad, remove it from the map.
CompletableFuture<ManagedLedgerImpl> existingFuture = ledgers.get(name);
if (existingFuture != null) {
if (existingFuture.isDone()) {
try {
ManagedLedgerImpl l = existingFuture.get();
if (l.getState().isFenced() || l.getState() == State.Closed) {
// Managed ledger is in unusable state. Recreate it.
log.warn("[{}] Attempted to open ledger in {} state. Removing from the map to recreate it",
name, l.getState());
ledgers.remove(name, existingFuture);
}
} catch (Exception e) {
// Unable to get the future
log.warn("[{}] Got exception while trying to retrieve ledger", name, e);
}
} else {
PendingInitializeManagedLedger pendingLedger = pendingInitializeLedgers.get(name);
if (null != pendingLedger) {
long pendingMs = System.currentTimeMillis() - pendingLedger.createTimeMs;
if (pendingMs > TimeUnit.SECONDS.toMillis(config.getMetadataOperationsTimeoutSeconds())) {
log.warn("[{}] Managed ledger has been pending in initialize state more than {} milliseconds,"
+ " remove it from cache to retry ...", name, pendingMs);
ledgers.remove(name, existingFuture);
pendingInitializeLedgers.remove(name, pendingLedger);
}
}
}
}
// Ensure only one managed ledger is created and initialized
ledgers.computeIfAbsent(name, (mlName) -> {
// Create the managed ledger
CompletableFuture<ManagedLedgerImpl> future = new CompletableFuture<>();
bookkeeperFactory.get(
new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(),
config.getBookKeeperEnsemblePlacementPolicyProperties()))
.thenAccept(bk -> {
final ManagedLedgerImpl newledger = config.getShadowSource() == null
? new ManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name,
mlOwnershipChecker)
: new ShadowManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name,
mlOwnershipChecker);
PendingInitializeManagedLedger pendingLedger = new PendingInitializeManagedLedger(newledger);
pendingInitializeLedgers.put(name, pendingLedger);
newledger.initialize(new ManagedLedgerInitializeLedgerCallback() {
@Override
public void initializeComplete() {
log.info("[{}] Successfully initialize managed ledger", name);
pendingInitializeLedgers.remove(name, pendingLedger);
future.complete(newledger);
// May need to update the cursor position
newledger.maybeUpdateCursorBeforeTrimmingConsumedLedger();
// May need to trigger offloading
if (config.isTriggerOffloadOnTopicLoad()) {
newledger.maybeOffloadInBackground(NULL_OFFLOAD_PROMISE);
}
}
@Override
public void initializeFailed(ManagedLedgerException e) {
if (config.isCreateIfMissing()) {
log.error("[{}] Failed to initialize managed ledger: {}", name, e.getMessage());
}
// Clean the map if initialization fails
ledgers.remove(name, future);
if (pendingInitializeLedgers.remove(name, pendingLedger)) {
pendingLedger.ledger.asyncClose(new CloseCallback() {
@Override
public void closeComplete(Object ctx) {
// no-op
}
@Override
public void closeFailed(ManagedLedgerException exception, Object ctx) {
log.warn("[{}] Failed to a pending initialization managed ledger", name,
exception);
}
}, null);
}
future.completeExceptionally(e);
}
}, null);
}).exceptionally(ex -> {
future.completeExceptionally(ex);
return null;
});
return future;
}).thenAccept(ml -> callback.openLedgerComplete(ml, ctx)).exceptionally(exception -> {
callback.openLedgerFailed((ManagedLedgerException) exception.getCause(), ctx);
return null;
});
}
@Override
public void asyncOpenReadOnlyManagedLedger(String managedLedgerName,
AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback,
ManagedLedgerConfig config, Object ctx) {
if (closed) {
callback.openReadOnlyManagedLedgerFailed(
new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx);
}
bookkeeperFactory
.get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(),
config.getBookKeeperEnsemblePlacementPolicyProperties()))
.thenCompose(bk -> {
ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this, bk,
store, config, scheduledExecutor, managedLedgerName);
return roManagedLedger.initialize().thenApply(v -> roManagedLedger);
}).thenAccept(roManagedLedger -> {
log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName);
callback.openReadOnlyManagedLedgerComplete(roManagedLedger, ctx);
}).exceptionally(e -> {
log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e);
callback.openReadOnlyManagedLedgerFailed((ManagedLedgerException) e.getCause(), ctx);
return null;
});
}
@Override
public ReadOnlyCursor openReadOnlyCursor(String managedLedgerName, Position startPosition,
ManagedLedgerConfig config)
throws InterruptedException, ManagedLedgerException {
class Result {
ReadOnlyCursor c = null;
ManagedLedgerException e = null;
}
final Result r = new Result();
final CountDownLatch latch = new CountDownLatch(1);
asyncOpenReadOnlyCursor(managedLedgerName, startPosition, config, new OpenReadOnlyCursorCallback() {
@Override
public void openReadOnlyCursorComplete(ReadOnlyCursor cursor, Object ctx) {
r.c = cursor;
latch.countDown();
}
@Override
public void openReadOnlyCursorFailed(ManagedLedgerException exception, Object ctx) {
r.e = exception;
latch.countDown();
}
}, null);
latch.await();
if (r.e != null) {
throw r.e;
}
return r.c;
}
@Override
public void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosition, ManagedLedgerConfig config,
OpenReadOnlyCursorCallback callback, Object ctx) {
if (closed) {
callback.openReadOnlyCursorFailed(new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx);
return;
}
AsyncCallbacks.OpenReadOnlyManagedLedgerCallback openReadOnlyManagedLedgerCallback =
new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() {
@Override
public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) {
callback.openReadOnlyCursorComplete(readOnlyManagedLedger.
createReadOnlyCursor(startPosition), ctx);
}
@Override
public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) {
callback.openReadOnlyCursorFailed(exception, ctx);
}
};
asyncOpenReadOnlyManagedLedger(managedLedgerName, openReadOnlyManagedLedgerCallback, config, null);
}
void close(ManagedLedger ledger) {
// If the future in map is not done or has exceptionally complete, it means that @param-ledger is not in the
// map.
CompletableFuture<ManagedLedgerImpl> ledgerFuture = ledgers.get(ledger.getName());
if (ledgerFuture == null || !ledgerFuture.isDone() || ledgerFuture.isCompletedExceptionally()){
return;
}
if (ledgerFuture.join() != ledger){
return;
}
// Remove the ledger from the internal factory cache.
if (ledgers.remove(ledger.getName(), ledgerFuture)) {
entryCacheManager.removeEntryCache(ledger.getName());
}
}
public CompletableFuture<Void> shutdownAsync() throws ManagedLedgerException {
if (closed) {
throw new ManagedLedgerException.ManagedLedgerFactoryClosedException();
}
closed = true;
statsTask.cancel(true);
flushCursorsTask.cancel(true);
cacheEvictionExecutor.shutdownNow();
List<String> ledgerNames = new ArrayList<>(this.ledgers.keySet());
List<CompletableFuture<Void>> futures = new ArrayList<>(ledgerNames.size());
int numLedgers = ledgerNames.size();
log.info("Closing {} ledgers", numLedgers);
for (String ledgerName : ledgerNames) {
CompletableFuture<ManagedLedgerImpl> ledgerFuture = ledgers.remove(ledgerName);
if (ledgerFuture == null) {
continue;
}
CompletableFuture<Void> future = new CompletableFuture<>();
futures.add(future);
ledgerFuture.whenCompleteAsync((managedLedger, throwable) -> {
if (throwable != null || managedLedger == null) {
future.complete(null);
return;
}
managedLedger.asyncClose(new AsyncCallbacks.CloseCallback() {
@Override
public void closeComplete(Object ctx) {
future.complete(null);
}
@Override
public void closeFailed(ManagedLedgerException exception, Object ctx) {
log.warn("[{}] Got exception when closing managed ledger: {}", managedLedger.getName(),
exception);
future.complete(null);
}
}, null);
}, scheduledExecutor.chooseThread());
//close pendingInitializeManagedLedger directly to make sure all callbacks is called.
PendingInitializeManagedLedger pendingLedger = pendingInitializeLedgers.get(ledgerName);
if (pendingLedger != null && !ledgerFuture.isDone()) {
ledgerFuture.completeExceptionally(new ManagedLedgerException.ManagedLedgerFactoryClosedException());
}
}
CompletableFuture<BookKeeper> bookkeeperFuture = isBookkeeperManaged
? bookkeeperFactory.get()
: CompletableFuture.completedFuture(null);
return bookkeeperFuture
.thenRun(() -> {
log.info("Closing {} ledgers.", ledgers.size());
//make sure all callbacks is called.
ledgers.forEach(((ledgerName, ledgerFuture) -> {
if (!ledgerFuture.isDone()) {
ledgerFuture.completeExceptionally(
new ManagedLedgerException.ManagedLedgerFactoryClosedException());
} else {
ManagedLedgerImpl managedLedger = ledgerFuture.getNow(null);
if (managedLedger == null) {
return;
}
try {
managedLedger.close();
} catch (Throwable throwable) {
log.warn("[{}] Got exception when closing managed ledger: {}", managedLedger.getName(),
throwable);
}
}
}));
}).thenAcceptAsync(__ -> {
//wait for tasks in scheduledExecutor executed.
openTelemetryCacheStats.close();
scheduledExecutor.shutdownNow();
entryCacheManager.clear();
});
}
@Override
public void shutdown() throws InterruptedException, ManagedLedgerException {
try {
shutdownAsync().get();
} catch (ExecutionException e) {
throw getManagedLedgerException(e.getCause());
}
}
@Override
public CompletableFuture<Boolean> asyncExists(String ledgerName) {
return store.asyncExists(ledgerName);
}
@Override
public ManagedLedgerInfo getManagedLedgerInfo(String name) throws InterruptedException, ManagedLedgerException {
class Result {
ManagedLedgerInfo info = null;
ManagedLedgerException e = null;
}
final Result r = new Result();
final CountDownLatch latch = new CountDownLatch(1);
asyncGetManagedLedgerInfo(name, new ManagedLedgerInfoCallback() {
@Override
public void getInfoComplete(ManagedLedgerInfo info, Object ctx) {
r.info = info;
latch.countDown();
}
@Override
public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
r.e = exception;
latch.countDown();
}
}, null);
latch.await();
if (r.e != null) {
throw r.e;
}
return r.info;
}
@Override
public void asyncGetManagedLedgerInfo(String name, ManagedLedgerInfoCallback callback, Object ctx) {
store.getManagedLedgerInfo(name, false /* createIfMissing */,
new MetaStoreCallback<MLDataFormats.ManagedLedgerInfo>() {
@Override
public void operationComplete(MLDataFormats.ManagedLedgerInfo pbInfo, Stat stat) {
ManagedLedgerInfo info = new ManagedLedgerInfo();
info.version = stat.getVersion();
info.creationDate = DateFormatter.format(stat.getCreationTimestamp());
info.modificationDate = DateFormatter.format(stat.getModificationTimestamp());
info.ledgers = new ArrayList<>(pbInfo.getLedgerInfoCount());
if (pbInfo.hasTerminatedPosition()) {
info.terminatedPosition = new PositionInfo();
info.terminatedPosition.ledgerId = pbInfo.getTerminatedPosition().getLedgerId();
info.terminatedPosition.entryId = pbInfo.getTerminatedPosition().getEntryId();
}
if (pbInfo.getPropertiesCount() > 0) {
info.properties = new TreeMap();
for (int i = 0; i < pbInfo.getPropertiesCount(); i++) {
MLDataFormats.KeyValue property = pbInfo.getProperties(i);
info.properties.put(property.getKey(), property.getValue());
}
}
for (int i = 0; i < pbInfo.getLedgerInfoCount(); i++) {
MLDataFormats.ManagedLedgerInfo.LedgerInfo pbLedgerInfo = pbInfo.getLedgerInfo(i);
LedgerInfo ledgerInfo = new LedgerInfo();
ledgerInfo.ledgerId = pbLedgerInfo.getLedgerId();
ledgerInfo.entries = pbLedgerInfo.hasEntries() ? pbLedgerInfo.getEntries() : null;
ledgerInfo.size = pbLedgerInfo.hasSize() ? pbLedgerInfo.getSize() : null;
ledgerInfo.timestamp = pbLedgerInfo.hasTimestamp() ? pbLedgerInfo.getTimestamp() : null;
ledgerInfo.isOffloaded = pbLedgerInfo.hasOffloadContext();
if (pbLedgerInfo.hasOffloadContext()) {
MLDataFormats.OffloadContext offloadContext = pbLedgerInfo.getOffloadContext();
UUID uuid = new UUID(offloadContext.getUidMsb(), offloadContext.getUidLsb());
ledgerInfo.offloadedContextUuid = uuid.toString();
}
info.ledgers.add(ledgerInfo);
}
store.getCursors(name, new MetaStoreCallback<List<String>>() {
@Override
public void operationComplete(List<String> cursorsList, Stat stat) {
// Get the info for each cursor
info.cursors = new ConcurrentSkipListMap<>();
List<CompletableFuture<Void>> cursorsFutures = new ArrayList<>();
for (String cursorName : cursorsList) {
CompletableFuture<Void> cursorFuture = new CompletableFuture<>();
cursorsFutures.add(cursorFuture);
store.asyncGetCursorInfo(name, cursorName,
new MetaStoreCallback<MLDataFormats.ManagedCursorInfo>() {
@Override
public void operationComplete(ManagedCursorInfo pbCursorInfo, Stat stat) {
CursorInfo cursorInfo = new CursorInfo();
cursorInfo.version = stat.getVersion();
cursorInfo.creationDate = DateFormatter.format(stat.getCreationTimestamp());
cursorInfo.modificationDate = DateFormatter
.format(stat.getModificationTimestamp());
cursorInfo.cursorsLedgerId = pbCursorInfo.getCursorsLedgerId();
if (pbCursorInfo.hasMarkDeleteLedgerId()) {
cursorInfo.markDelete = new PositionInfo();
cursorInfo.markDelete.ledgerId = pbCursorInfo.getMarkDeleteLedgerId();
cursorInfo.markDelete.entryId = pbCursorInfo.getMarkDeleteEntryId();
}
if (pbCursorInfo.getPropertiesCount() > 0) {
cursorInfo.properties = new TreeMap();
for (int i = 0; i < pbCursorInfo.getPropertiesCount(); i++) {
LongProperty property = pbCursorInfo.getProperties(i);
cursorInfo.properties.put(property.getName(), property.getValue());
}
}
if (pbCursorInfo.getIndividualDeletedMessagesCount() > 0) {
cursorInfo.individualDeletedMessages = new ArrayList<>();
for (int i = 0; i < pbCursorInfo
.getIndividualDeletedMessagesCount(); i++) {
MessageRange range = pbCursorInfo.getIndividualDeletedMessages(i);
MessageRangeInfo rangeInfo = new MessageRangeInfo();
rangeInfo.from.ledgerId = range.getLowerEndpoint().getLedgerId();
rangeInfo.from.entryId = range.getLowerEndpoint().getEntryId();
rangeInfo.to.ledgerId = range.getUpperEndpoint().getLedgerId();
rangeInfo.to.entryId = range.getUpperEndpoint().getEntryId();
cursorInfo.individualDeletedMessages.add(rangeInfo);
}
}
info.cursors.put(cursorName, cursorInfo);
cursorFuture.complete(null);
}
@Override
public void operationFailed(MetaStoreException e) {
cursorFuture.completeExceptionally(e);
}
});
}
Futures.waitForAll(cursorsFutures).thenRun(() -> {
// Completed all the cursors info
callback.getInfoComplete(info, ctx);
}).exceptionally((ex) -> {
callback.getInfoFailed(getManagedLedgerException(ex.getCause()), ctx);
return null;
});
}
@Override
public void operationFailed(MetaStoreException e) {
callback.getInfoFailed(e, ctx);
}
});
}
@Override
public void operationFailed(MetaStoreException e) {
callback.getInfoFailed(e, ctx);
}
});
}
@Override
public void delete(String name) throws InterruptedException, ManagedLedgerException {
delete(name, CompletableFuture.completedFuture(null));
}
@Override
public void delete(String name, CompletableFuture<ManagedLedgerConfig> mlConfigFuture)
throws InterruptedException, ManagedLedgerException {
class Result {
ManagedLedgerException e = null;
}
final Result r = new Result();
final CountDownLatch latch = new CountDownLatch(1);
asyncDelete(name, mlConfigFuture, new DeleteLedgerCallback() {
@Override
public void deleteLedgerComplete(Object ctx) {
latch.countDown();
}
@Override
public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) {
r.e = exception;
latch.countDown();
}
}, null);
latch.await();
if (r.e != null) {
throw r.e;
}
}
@Override
public void asyncDelete(String name, DeleteLedgerCallback callback, Object ctx) {
asyncDelete(name, CompletableFuture.completedFuture(null), callback, ctx);
}
@Override
public void asyncDelete(String name, CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
DeleteLedgerCallback callback, Object ctx) {
CompletableFuture<ManagedLedgerImpl> future = ledgers.get(name);
if (future == null) {
// Managed ledger does not exist and we're not currently trying to open it
deleteManagedLedger(name, mlConfigFuture, callback, ctx);
} else {
future.thenAccept(ml -> {
// If it's open, delete in the normal way
ml.asyncDelete(callback, ctx);
}).exceptionally(ex -> {
// If it fails to get open, it will be cleaned by managed ledger opening error handling.
// then retry will go to `future=null` branch.
final Throwable rc = FutureUtil.unwrapCompletionException(ex);
callback.deleteLedgerFailed(getManagedLedgerException(rc), ctx);
return null;
});
}
}
/**
* Delete all managed ledger resources and metadata.
*/
void deleteManagedLedger(String managedLedgerName, CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
DeleteLedgerCallback callback, Object ctx) {
// Read the managed ledger metadata from store
asyncGetManagedLedgerInfo(managedLedgerName, new ManagedLedgerInfoCallback() {
@Override
public void getInfoComplete(ManagedLedgerInfo info, Object ctx) {
getBookKeeper().thenCompose(bk -> {
// First delete all cursors resources
List<CompletableFuture<Void>> futures = info.cursors.entrySet().stream()
.map(e -> deleteCursor(bk, managedLedgerName, e.getKey(), e.getValue()))
.collect(Collectors.toList());
return Futures.waitForAll(futures).thenApply(v -> bk);
}).thenAccept(bk -> {
deleteManagedLedgerData(bk, managedLedgerName, info, mlConfigFuture, callback, ctx);
}).exceptionally(ex -> {
callback.deleteLedgerFailed(new ManagedLedgerException(ex), ctx);
return null;
});
}
@Override
public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
callback.deleteLedgerFailed(exception, ctx);
}
}, ctx);
}
private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info,
CompletableFuture<ManagedLedgerConfig> mlConfigFuture,
DeleteLedgerCallback callback, Object ctx) {
final CompletableFuture<Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo>>
ledgerInfosFuture = new CompletableFuture<>();
store.getManagedLedgerInfo(managedLedgerName, false, null,
new MetaStoreCallback<>() {
@Override
public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) {
Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> infos = new HashMap<>();
for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : mlInfo.getLedgerInfoList()) {
infos.put(ls.getLedgerId(), ls);
}
ledgerInfosFuture.complete(infos);
}
@Override
public void operationFailed(MetaStoreException e) {
log.error("Failed to get managed ledger info for {}", managedLedgerName, e);
ledgerInfosFuture.completeExceptionally(e);
}
});
Futures.waitForAll(info.ledgers.stream()
.map(li -> {
final CompletableFuture<Void> res;
if (li.isOffloaded) {
res = mlConfigFuture
.thenCombine(ledgerInfosFuture, Pair::of)
.thenCompose(pair -> {
ManagedLedgerConfig mlConfig = pair.getLeft();
Map<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgerInfos = pair.getRight();
if (mlConfig == null || ledgerInfos == null) {
return CompletableFuture.completedFuture(null);
}
MLDataFormats.ManagedLedgerInfo.LedgerInfo ls = ledgerInfos.get(li.ledgerId);
if (ls.getOffloadContext().hasUidMsb()) {
MLDataFormats.ManagedLedgerInfo.LedgerInfo.Builder newInfoBuilder = ls.toBuilder();
newInfoBuilder.getOffloadContextBuilder().setBookkeeperDeleted(true);
String driverName = OffloadUtils.getOffloadDriverName(ls,
mlConfig.getLedgerOffloader().getOffloadDriverName());
Map<String, String> driverMetadata = OffloadUtils.getOffloadDriverMetadata(ls,
mlConfig.getLedgerOffloader().getOffloadDriverMetadata());
OffloadUtils.setOffloadDriverMetadata(newInfoBuilder, driverName, driverMetadata);
UUID uuid = new UUID(ls.getOffloadContext().getUidMsb(),
ls.getOffloadContext().getUidLsb());
return OffloadUtils.cleanupOffloaded(li.ledgerId, uuid, mlConfig,
OffloadUtils.getOffloadDriverMetadata(ls,
mlConfig.getLedgerOffloader().getOffloadDriverMetadata()),
"Deletion", managedLedgerName, scheduledExecutor);
}
return CompletableFuture.completedFuture(null);
});
} else {
res = CompletableFuture.completedFuture(null);
}
return res.thenCompose(__ -> bkc.newDeleteLedgerOp().withLedgerId(li.ledgerId).execute()
.handle((result, ex) -> {
if (ex != null) {
int rc = BKException.getExceptionCode(ex);
if (rc == BKException.Code.NoSuchLedgerExistsOnMetadataServerException
|| rc == BKException.Code.NoSuchLedgerExistsException) {
log.info("Ledger {} does not exist, ignoring", li.ledgerId);
return null;
}
throw new CompletionException(ex);
}
return result;
}));
})
.collect(Collectors.toList()))
.thenRun(() -> {
// Delete the metadata
store.removeManagedLedger(managedLedgerName, new MetaStoreCallback<Void>() {
@Override
public void operationComplete(Void result, Stat stat) {
callback.deleteLedgerComplete(ctx);
}
@Override
public void operationFailed(MetaStoreException e) {
callback.deleteLedgerFailed(new ManagedLedgerException(e), ctx);
}
});
}).exceptionally(ex -> {
callback.deleteLedgerFailed(new ManagedLedgerException(ex), ctx);
return null;
});
}
private CompletableFuture<Void> deleteCursor(BookKeeper bkc, String managedLedgerName, String cursorName,
CursorInfo cursor) {
CompletableFuture<Void> future = new CompletableFuture<>();
CompletableFuture<Void> cursorLedgerDeleteFuture;
// Delete the cursor ledger if present
if (cursor.cursorsLedgerId != -1) {
cursorLedgerDeleteFuture = bkc.newDeleteLedgerOp().withLedgerId(cursor.cursorsLedgerId)
.execute()
.handle((result, ex) -> {
if (ex != null) {
int rc = BKException.getExceptionCode(ex);
if (rc == BKException.Code.NoSuchLedgerExistsOnMetadataServerException
|| rc == BKException.Code.NoSuchLedgerExistsException) {
log.info("Ledger {} does not exist, ignoring", cursor.cursorsLedgerId);
return null;
}
throw new CompletionException(ex);