forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
InflightReadsLimiter - limit the memory used by reads end-to-end (fro…
…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
Showing
17 changed files
with
524 additions
and
5 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
137 changes: 137 additions & 0 deletions
137
...d-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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; | ||
} | ||
|
||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.