eolivelli commented on code in PR #17953: URL: https://github.com/apache/pulsar/pull/17953#discussion_r995443883
########## managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsLimiter.java: ########## @@ -0,0 +1,126 @@ +/** + * 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 PendingReadsLimiter { + + private static final Gauge PULSAR_ML_PENDING_READS_BUFFER_SIZE = Gauge + .build() + .name("pulsar_ml_pending_reads_buffer_size") + .help("Estimated number of bytes for pending reads from storage") + .register(); + + private final long maxPendingReadsBufferSize; + private long remainingPendingRequestsBytes; + + public PendingReadsLimiter(long maxPendingReadsBufferSize) { + this.maxPendingReadsBufferSize = maxPendingReadsBufferSize; + this.remainingPendingRequestsBytes = maxPendingReadsBufferSize; + } + + @VisibleForTesting + public synchronized long getRemainingPendingRequestsBytes() { + return remainingPendingRequestsBytes; + } + + @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 (maxPendingReadsBufferSize <= 0) { + // feature is disabled + return DISABLED; + } + synchronized (this) { Review Comment: here we don't have control about the thread that is going to run this code. also here the thread that runs this code will not be the thread that will allocate the memory, because that thread will be the BK internal thread for the Ledger (not the ManagedLedger). ########## managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java: ########## @@ -437,6 +458,60 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { } } + private AsyncCallbacks.ReadEntriesCallback handlePendingReadsLimits(ReadHandle lh, + long firstEntry, long lastEntry, + boolean shouldCacheEntry, + AsyncCallbacks.ReadEntriesCallback originalCallback, + Object ctx, PendingReadsLimiter.Handle handle) { + if (pendingReadsLimiter.isDisabled()) { + return originalCallback; + } + long estimatedReadSize = (1 + lastEntry - firstEntry) + * (estimatedEntrySize + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY); + final AsyncCallbacks.ReadEntriesCallback callback; + PendingReadsLimiter.Handle newHandle = pendingReadsLimiter.acquire(estimatedReadSize, handle); + if (!newHandle.success) { + long now = System.currentTimeMillis(); + if (now - newHandle.creationTime > readEntryTimeoutMillis) { + String message = "Time-out elapsed reading from ledger " + + lh.getId() + + ", " + rangeEntryCache.getName() + + ", estimated read size " + estimatedReadSize + " bytes" + + " for " + (1 + lastEntry - firstEntry) + " entries"; + log.error(message); + pendingReadsLimiter.release(newHandle); + originalCallback.readEntriesFailed( + new ManagedLedgerException.TooManyRequestsException(message), ctx); + return null; + } + this.rangeEntryCache.ml.getExecutor().submitOrdered(lh.getId(), () -> { + readEntriesInternal(lh, firstEntry, lastEntry, shouldCacheEntry, + originalCallback, ctx, newHandle); + return null; + }); + return null; + } else { + callback = new AsyncCallbacks.ReadEntriesCallback() { + @Override + public void readEntriesComplete(List<Entry> entries, Object ctx) { + if (!entries.isEmpty()) { + long size = entries.get(0).getLength(); + estimatedEntrySize = size; + } + pendingReadsLimiter.release(newHandle); Review Comment: When we reach this point the "read" has been performed and now it is the higher level component (whatever it is, not only Cursors may use ManagedLedgers) that is in charge of the memory allocated by BK. Here we are counting the size of this virtual "buffer" that contains the bytes read from BK but not yet passed to the rest of the application ########## managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsLimiter.java: ########## @@ -0,0 +1,126 @@ +/** + * 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 PendingReadsLimiter { + + private static final Gauge PULSAR_ML_PENDING_READS_BUFFER_SIZE = Gauge + .build() + .name("pulsar_ml_pending_reads_buffer_size") + .help("Estimated number of bytes for pending reads from storage") + .register(); + + private final long maxPendingReadsBufferSize; + private long remainingPendingRequestsBytes; + + public PendingReadsLimiter(long maxPendingReadsBufferSize) { + this.maxPendingReadsBufferSize = maxPendingReadsBufferSize; + this.remainingPendingRequestsBytes = maxPendingReadsBufferSize; + } + + @VisibleForTesting + public synchronized long getRemainingPendingRequestsBytes() { + return remainingPendingRequestsBytes; + } + + @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 (maxPendingReadsBufferSize <= 0) { + // feature is disabled + return DISABLED; + } + synchronized (this) { + try { + if (current == null) { + if (remainingPendingRequestsBytes == 0) { + return new Handle(0, false, 1, System.currentTimeMillis()); + } + long needed = permits; Review Comment: sure ########## managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsLimiter.java: ########## @@ -0,0 +1,126 @@ +/** + * 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 PendingReadsLimiter { + + private static final Gauge PULSAR_ML_PENDING_READS_BUFFER_SIZE = Gauge + .build() + .name("pulsar_ml_pending_reads_buffer_size") + .help("Estimated number of bytes for pending reads from storage") + .register(); Review Comment: added -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
