Skip to content

Commit

Permalink
InflightReadsLimiter - limit the memory used by reads end-to-end (fro…
Browse files Browse the repository at this point in the history
…m storage/cache to the write to the consumer channel) (apache#18245)

* InflightReadsLimiter - limit the memory used by reads end-to-end (from storage/cache to the write to the consumer channel)

Motivation:

Broker can go out of memory due to many reads enqueued on the PersistentDispatcherMultipleConsumers dispatchMessagesThread (that is used in case of dispatcherDispatchMessagesInSubscriptionThread set to true, that is the default value)
The limit of the amount of memory retained due to reads MUST take into account also the entries coming from the Cache.

When dispatcherDispatchMessagesInSubscriptionThread is false (the behaviour of Pulsar 2.10) there is some kind of natural (but still unpredictable!!) back pressure mechanism because the thread that receives the entries from BK of the cache dispatches immediately and synchronously the entries to the consumer and releases them

Modifications:

- Add a new component (InflightReadsLimiter) that keeps track of the overall amount of memory retained due to inflight reads.
- Add a new configuration entry managedLedgerMaxReadsInFlightSizeInMB
- The feature is disabled by default
- Add new metrics to track the values

* Change error message

* checkstyle

* Fix license

* remove duplicate method after cherry-pick

* Rename onDeallocate
  • Loading branch information
eolivelli authored Nov 11, 2022
1 parent 1db89d7 commit 6fec66b
Show file tree
Hide file tree
Showing 17 changed files with 524 additions and 5 deletions.
8 changes: 8 additions & 0 deletions conf/broker.conf
Original file line number Diff line number Diff line change
Expand Up @@ -1172,6 +1172,14 @@ managedLedgerCursorRolloverTimeInSeconds=14400
# crashes.
managedLedgerMaxUnackedRangesToPersist=10000

# Maximum amount of memory used hold data read from storage (or from the cache).
# This mechanism prevents the broker to have too many concurrent
# reads from storage and fall into Out of Memory errors in case
# of multiple concurrent reads to multiple concurrent consumers.
# Set 0 in order to disable the feature.
#
managedLedgerMaxReadsInFlightSizeInMB=0

# Max number of "acknowledgment holes" that can be stored in MetadataStore. If number of unack message range is higher
# than this limit then broker will persist unacked ranges into bookkeeper to avoid additional data overhead into
# MetadataStore.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,11 @@ public class ManagedLedgerFactoryConfig {
*/
private boolean copyEntriesInCache = false;

/**
* Maximum number of (estimated) data in-flight reading from storage and the cache.
*/
private long managedLedgerMaxReadsInFlightSize = 0;

/**
* Whether trace managed ledger task execution time.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@ protected EntryImpl newObject(Handle<EntryImpl> handle) {
private long entryId;
ByteBuf data;

private Runnable onDeallocate;

public static EntryImpl create(LedgerEntry ledgerEntry) {
EntryImpl entry = RECYCLER.get();
entry.timestamp = System.nanoTime();
Expand Down Expand Up @@ -102,6 +104,22 @@ private EntryImpl(Recycler.Handle<EntryImpl> recyclerHandle) {
this.recyclerHandle = recyclerHandle;
}

public void onDeallocate(Runnable r) {
if (this.onDeallocate == null) {
this.onDeallocate = r;
} else {
// this is not expected to happen
Runnable previous = this.onDeallocate;
this.onDeallocate = () -> {
try {
previous.run();
} finally {
r.run();
}
};
}
}

public long getTimestamp() {
return timestamp;
}
Expand Down Expand Up @@ -167,6 +185,13 @@ public ReferenceCounted touch(Object hint) {
@Override
protected void deallocate() {
// This method is called whenever the ref-count of the EntryImpl reaches 0, so that now we can recycle it
if (onDeallocate != null) {
try {
onDeallocate.run();
} finally {
onDeallocate = null;
}
}
data.release();
data = null;
timestamp = -1;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
/*
* 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.cache;

import com.google.common.annotations.VisibleForTesting;
import io.prometheus.client.Gauge;
import lombok.AllArgsConstructor;
import lombok.ToString;
import lombok.extern.slf4j.Slf4j;

@Slf4j
public class InflightReadsLimiter {

private static final Gauge PULSAR_ML_READS_BUFFER_SIZE = Gauge
.build()
.name("pulsar_ml_reads_inflight_bytes")
.help("Estimated number of bytes retained by data read from storage or cache")
.register();

private static final Gauge PULSAR_ML_READS_AVAILABLE_BUFFER_SIZE = Gauge
.build()
.name("pulsar_ml_reads_available_inflight_bytes")
.help("Available space for inflight data read from storage or cache")
.register();

private final long maxReadsInFlightSize;
private long remainingBytes;

public InflightReadsLimiter(long maxReadsInFlightSize) {
if (maxReadsInFlightSize <= 0) {
// set it to -1 in order to show in the metrics that the metric is not available
PULSAR_ML_READS_BUFFER_SIZE.set(-1);
PULSAR_ML_READS_AVAILABLE_BUFFER_SIZE.set(-1);
}
this.maxReadsInFlightSize = maxReadsInFlightSize;
this.remainingBytes = maxReadsInFlightSize;
}

@VisibleForTesting
public synchronized long getRemainingBytes() {
return remainingBytes;
}

@AllArgsConstructor
@ToString
static class Handle {
final long acquiredPermits;
final boolean success;
final int trials;

final long creationTime;
}

private static final Handle DISABLED = new Handle(0, true, 0, -1);

Handle acquire(long permits, Handle current) {
if (maxReadsInFlightSize <= 0) {
// feature is disabled
return DISABLED;
}
synchronized (this) {
try {
if (current == null) {
if (remainingBytes == 0) {
return new Handle(0, false, 1, System.currentTimeMillis());
}
if (remainingBytes >= permits) {
remainingBytes -= permits;
return new Handle(permits, true, 1, System.currentTimeMillis());
} else {
long possible = remainingBytes;
remainingBytes = 0;
return new Handle(possible, false, 1, System.currentTimeMillis());
}
} else {
if (current.trials >= 4 && current.acquiredPermits > 0) {
remainingBytes += current.acquiredPermits;
return new Handle(0, false, 1, current.creationTime);
}
if (remainingBytes == 0) {
return new Handle(current.acquiredPermits, false, current.trials + 1,
current.creationTime);
}
long needed = permits - current.acquiredPermits;
if (remainingBytes >= needed) {
remainingBytes -= needed;
return new Handle(permits, true, current.trials + 1, current.creationTime);
} else {
long possible = remainingBytes;
remainingBytes = 0;
return new Handle(current.acquiredPermits + possible, false,
current.trials + 1, current.creationTime);
}
}
} finally {
updateMetrics();
}
}
}

void release(Handle handle) {
if (handle == DISABLED) {
return;
}
synchronized (this) {
remainingBytes += handle.acquiredPermits;
updateMetrics();
}
}

private synchronized void updateMetrics() {
PULSAR_ML_READS_BUFFER_SIZE.set(maxReadsInFlightSize - remainingBytes);
PULSAR_ML_READS_AVAILABLE_BUFFER_SIZE.set(remainingBytes);
}

public boolean isDisabled() {
return maxReadsInFlightSize <= 0;
}


}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.concurrent.atomic.AtomicBoolean;
import lombok.AllArgsConstructor;
import lombok.Value;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.Entry;
Expand All @@ -37,6 +38,7 @@
/**
* PendingReadsManager tries to prevent sending duplicate reads to BK.
*/
@Slf4j
public class PendingReadsManager {

private static final Counter COUNT_ENTRIES_READ_FROM_BK = Counter
Expand Down Expand Up @@ -315,8 +317,6 @@ synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback,

void readEntries(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry,
final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) {


final PendingReadKey key = new PendingReadKey(firstEntry, lastEntry);

Map<PendingReadKey, PendingRead> pendingReadsForLedger =
Expand Down Expand Up @@ -442,6 +442,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
}
}


void clear() {
cachedPendingReads.clear();
}
Expand Down
Loading

0 comments on commit 6fec66b

Please sign in to comment.