michaeljmarshall commented on code in PR #17953: URL: https://github.com/apache/pulsar/pull/17953#discussion_r993795673
########## 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: Another interesting metric would be the amount of free space left for pending reads. It could technically be computed based on this metric as long as an operator knows the configured limit, so it might not be important to add. Although, it would be a good signal that the broker is under heavy load, though if the load is from a single topic with many subscriptions, load balancing will not help. ########## managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java: ########## @@ -57,6 +57,11 @@ public class ManagedLedgerFactoryConfig { */ private boolean copyEntriesInCache = false; + /** + * Maximum number of (estimated) data in-flight reading from storage. Review Comment: Nit ```suggestion * Maximum number of (estimated) bytes in-flight reading from storage. ``` ########## 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: Nit. Can we remove the `needed` variable? ########## 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: I agree with the design choice to release the acquired byte permit here, but I think it leaves us open to potential OOM issues in the case that the consumers are slow to consume from the TCP connection. In that case, you could have many consumers request entries then not consume them. Also in that case, we know the exact size of the entries, and we are also potentially putting them in the read cache in catch up scenarios. It might be worth exploring another feature to limit these in memory messages to also add back pressure to consumers that are requesting more entries. Technically, there is an attempted back pressure implementation here https://github.com/apache/pulsar/blob/db26073728bf86fc80deecaece2dc02b50bbb9b5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java#L386-L392 However, I think we want something more robust. Even in the case where the outbound consumer connection is writable, we might be filling up memory with many messages. Additionally, there is a limit on the size of entries cached, but if we have many reads that are in flight in the broker's memory, we could have issues if there isn't an upper bound on the already read but not yet released entries in the broker's memory. ########## 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; Review Comment: Can we make this reactive by queuing cursors that have requested more entries and then feed those cursors as memory gets freed? We'd need a scheduled task to see if the next request in the queue is timed out, but it'd prevent us from putting these executables into the queue with high frequency. ########## 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: This could create additional contention in the read path. I don't see an immediate solution. I am only mentioning in case someone else has concerns. I assume this is why `maxMessagePublishBufferSizeInMB` is divided up to create limits for each thread. -- 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]
