21
21
import static com .google .common .base .Preconditions .checkArgument ;
22
22
import static com .google .common .base .Preconditions .checkNotNull ;
23
23
import static org .apache .bookkeeper .mledger .impl .ManagedLedgerImpl .createManagedLedgerException ;
24
+ import com .google .common .annotations .VisibleForTesting ;
24
25
import com .google .common .collect .Lists ;
25
26
import com .google .common .primitives .Longs ;
26
27
import io .netty .buffer .ByteBuf ;
27
28
import io .netty .buffer .PooledByteBufAllocator ;
28
29
import java .util .Collection ;
29
30
import java .util .Iterator ;
30
31
import java .util .List ;
32
+ import java .util .concurrent .CompletableFuture ;
33
+ import java .util .concurrent .atomic .AtomicInteger ;
31
34
import org .apache .bookkeeper .client .api .BKException ;
32
35
import org .apache .bookkeeper .client .api .LedgerEntry ;
33
36
import org .apache .bookkeeper .client .api .ReadHandle ;
37
+ import org .apache .bookkeeper .mledger .AsyncCallbacks ;
34
38
import org .apache .bookkeeper .mledger .AsyncCallbacks .ReadEntriesCallback ;
35
39
import org .apache .bookkeeper .mledger .AsyncCallbacks .ReadEntryCallback ;
40
+ import org .apache .bookkeeper .mledger .Entry ;
41
+ import org .apache .bookkeeper .mledger .ManagedLedgerConfig ;
36
42
import org .apache .bookkeeper .mledger .ManagedLedgerException ;
37
43
import org .apache .bookkeeper .mledger .impl .EntryImpl ;
38
44
import org .apache .bookkeeper .mledger .impl .ManagedLedgerImpl ;
48
54
*/
49
55
public class RangeEntryCacheImpl implements EntryCache {
50
56
57
+ /**
58
+ * Overhead per-entry to take into account the envelope.
59
+ */
60
+ private static final long BOOKKEEPER_READ_OVERHEAD_PER_ENTRY = 64 ;
61
+
51
62
private final RangeEntryCacheManagerImpl manager ;
52
63
private final ManagedLedgerImpl ml ;
53
64
private ManagedLedgerInterceptor interceptor ;
54
65
private final RangeCache <PositionImpl , EntryImpl > entries ;
55
66
private final boolean copyEntries ;
67
+ private volatile long estimatedEntrySize = 10 * 1024 ;
56
68
69
+ private final long readEntryTimeoutMillis ;
57
70
private static final double MB = 1024 * 1024 ;
58
71
59
72
public RangeEntryCacheImpl (RangeEntryCacheManagerImpl manager , ManagedLedgerImpl ml , boolean copyEntries ) {
60
73
this .manager = manager ;
61
74
this .ml = ml ;
75
+ this .pendingReadsManager = new PendingReadsManager (this );
62
76
this .interceptor = ml .getManagedLedgerInterceptor ();
77
+ this .readEntryTimeoutMillis = getManagedLedgerConfig ().getReadEntryTimeoutSeconds ();
63
78
this .entries = new RangeCache <>(EntryImpl ::getLength , EntryImpl ::getTimestamp );
64
79
this .copyEntries = copyEntries ;
65
80
@@ -68,11 +83,21 @@ public RangeEntryCacheImpl(RangeEntryCacheManagerImpl manager, ManagedLedgerImpl
68
83
}
69
84
}
70
85
86
+ @ VisibleForTesting
87
+ ManagedLedgerConfig getManagedLedgerConfig () {
88
+ return ml .getConfig ();
89
+ }
90
+
71
91
@ Override
72
92
public String getName () {
73
93
return ml .getName ();
74
94
}
75
95
96
+ @ VisibleForTesting
97
+ InflightReadsLimiter getPendingReadsLimiter () {
98
+ return manager .getInflightReadsLimiter ();
99
+ }
100
+
76
101
public static final PooledByteBufAllocator ALLOCATOR = new PooledByteBufAllocator (true , // preferDirect
77
102
0 , // nHeapArenas,
78
103
PooledByteBufAllocator .defaultNumDirectArena (), // nDirectArena
@@ -256,6 +281,19 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boole
256
281
@ SuppressWarnings ({ "unchecked" , "rawtypes" })
257
282
private void asyncReadEntry0 (ReadHandle lh , long firstEntry , long lastEntry , boolean isSlowestReader ,
258
283
final ReadEntriesCallback callback , Object ctx ) {
284
+ asyncReadEntry0WithLimits (lh , firstEntry , lastEntry , shouldCacheEntry , callback , ctx , null );
285
+ }
286
+
287
+ void asyncReadEntry0WithLimits (ReadHandle lh , long firstEntry , long lastEntry , boolean shouldCacheEntry ,
288
+ final ReadEntriesCallback originalCallback , Object ctx , InflightReadsLimiter .Handle handle ) {
289
+
290
+ final AsyncCallbacks .ReadEntriesCallback callback =
291
+ handlePendingReadsLimits (lh , firstEntry , lastEntry , shouldCacheEntry ,
292
+ originalCallback , ctx , handle );
293
+ if (callback == null ) {
294
+ return ;
295
+ }
296
+
259
297
final long ledgerId = lh .getId ();
260
298
final int entriesToRead = (int ) (lastEntry - firstEntry ) + 1 ;
261
299
final PositionImpl firstPosition = PositionImpl .get (lh .getId (), firstEntry );
@@ -329,6 +367,75 @@ private void asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, boo
329
367
}
330
368
}
331
369
370
+
371
+ private AsyncCallbacks .ReadEntriesCallback handlePendingReadsLimits (ReadHandle lh ,
372
+ long firstEntry , long lastEntry ,
373
+ boolean shouldCacheEntry ,
374
+ AsyncCallbacks .ReadEntriesCallback originalCallback ,
375
+ Object ctx , InflightReadsLimiter .Handle handle ) {
376
+ InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter ();
377
+ if (pendingReadsLimiter .isDisabled ()) {
378
+ return originalCallback ;
379
+ }
380
+ long estimatedReadSize = (1 + lastEntry - firstEntry )
381
+ * (estimatedEntrySize + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY );
382
+ final AsyncCallbacks .ReadEntriesCallback callback ;
383
+ InflightReadsLimiter .Handle newHandle = pendingReadsLimiter .acquire (estimatedReadSize , handle );
384
+ if (!newHandle .success ) {
385
+ long now = System .currentTimeMillis ();
386
+ if (now - newHandle .creationTime > readEntryTimeoutMillis ) {
387
+ String message = "Time-out elapsed while acquiring enough permits "
388
+ + "on the memory limiter to read from ledger "
389
+ + lh .getId ()
390
+ + ", " + getName ()
391
+ + ", estimated read size " + estimatedReadSize + " bytes"
392
+ + " for " + (1 + lastEntry - firstEntry )
393
+ + " entries (check managedLedgerMaxReadsInFlightSizeInMB)" ;
394
+ log .error (message );
395
+ pendingReadsLimiter .release (newHandle );
396
+ originalCallback .readEntriesFailed (
397
+ new ManagedLedgerException .TooManyRequestsException (message ), ctx );
398
+ return null ;
399
+ }
400
+ ml .getExecutor ().submitOrdered (lh .getId (), () -> {
401
+ asyncReadEntry0WithLimits (lh , firstEntry , lastEntry , shouldCacheEntry ,
402
+ originalCallback , ctx , newHandle );
403
+ return null ;
404
+ });
405
+ return null ;
406
+ } else {
407
+ callback = new AsyncCallbacks .ReadEntriesCallback () {
408
+
409
+ @ Override
410
+ public void readEntriesComplete (List <Entry > entries , Object ctx ) {
411
+ if (!entries .isEmpty ()) {
412
+ long size = entries .get (0 ).getLength ();
413
+ estimatedEntrySize = size ;
414
+
415
+ AtomicInteger remainingCount = new AtomicInteger (entries .size ());
416
+ for (Entry entry : entries ) {
417
+ ((EntryImpl ) entry ).onDeallocate (() -> {
418
+ if (remainingCount .decrementAndGet () <= 0 ) {
419
+ pendingReadsLimiter .release (newHandle );
420
+ }
421
+ });
422
+ }
423
+ } else {
424
+ pendingReadsLimiter .release (newHandle );
425
+ }
426
+ originalCallback .readEntriesComplete (entries , ctx );
427
+ }
428
+
429
+ @ Override
430
+ public void readEntriesFailed (ManagedLedgerException exception , Object ctx ) {
431
+ pendingReadsLimiter .release (newHandle );
432
+ originalCallback .readEntriesFailed (exception , ctx );
433
+ }
434
+ };
435
+ }
436
+ return callback ;
437
+ }
438
+
332
439
@ Override
333
440
public void clear () {
334
441
Pair <Integer , Long > removedPair = entries .clear ();
0 commit comments