Hi Lari, Thanks for the suggestion. I've created PIP-476 to address exactly this — it covers method visibility changes, a broker config for loading custom dispatchers, and lifecycle hooks for cleanup.
PIP: https://github.com/apache/pulsar/pull/25726 Would appreciate your review. Thanks, Xiangying Meng On Fri, May 8, 2026 at 10:06 PM Lari Hotari <[email protected]> wrote: > > Hi all, > > Suggestion: a separate, focused PIP first to add a "pluggable > dispatcher" extension point — method hooks in the existing Key_Shared > / Shared dispatchers for subclass extension, plus broker config to > select a third-party implementation. That would let "overflow ML" (and > any future variant) ship as its own dispatcher rather than as > conditionals in the core classes — less maintenance overhead, > reversible. We'd also likely need lifecycle hooks so plugins can clean > up state on topic/subscription deletion. > > -Lari > > On Thu, 7 May 2026 at 17:31, xiangying meng <[email protected]> wrote: > > > > Thank you very much for your feedback. > > I've shared my thoughts on these three alternatives in > > https://github.com/apache/pulsar/pull/25706#issuecomment-4398007309, > > and I look forward to more input from the community. > > > > Thanks, > > xiangying meng > > > > On Thu, May 7, 2026 at 7:21 PM Lari Hotari <[email protected]> wrote: > > > > > > There's a follow-up discussion on the PIP PR: > > > https://github.com/apache/pulsar/pull/25706. I've shared a detailed > > > write-up in > > > https://github.com/apache/pulsar/pull/25706#issuecomment-4396605560. > > > > > > It would be valuable to gather more thought and additional perspectives > > > on the various options for solving the problem stated in PIP-474 before > > > we decide on the final solution. > > > > > > -Lari > > > > > > On 2026/05/07 07:42:29 Lari Hotari wrote: > > > > Thanks for bringing up a real problem and driving the work to solve > > > > this issue. > > > > > > > > I'd suggest analyzing 3 alternative designs before deciding on the > > > > solution. > > > > > > > > Alternative 1: > > > > I'd suggest looking into an alternative design that achieves the same > > > > outcome of allowing the subscription cursor to advance. Instead of > > > > making copies of the messages, an alternative design would be to create > > > > another subscription to track the slow or hot keys. Essentially, the > > > > design could be very similar to diverting to the overflow managed > > > > ledger, but there wouldn't be a need to duplicate the data and get into > > > > a situation where different failure modes cause unnecessary > > > > complications. > > > > > > > > Alternative 2: > > > > Simply optimize the replay queue solution together with improving the > > > > scalability of individualDeletedMessages so that it scales to 1,000,000 > > > > ack holes and beyond. This would result in the simplest solution, which > > > > would cover most use cases. There are multiple benefits to keeping the > > > > solution simple. For example, backlog management doesn't change. > > > > > > > > Together with the PIP-430 broker cache (since 4.1.0), the replay queue > > > > solution already avoids most unnecessary BK reads when the broker cache > > > > is sufficiently tuned for high-scale use cases. The PIP-430 broker > > > > cache could be improved further to achieve high cache hit rates if it > > > > turns out to be a problem. > > > > > > > > Alternative 3: > > > > The client-side code could simply route to a separate topic on its own > > > > when it detects a hot key and acknowledge the original message. > > > > > > > > Regarding Alternative 2, I believe that individualDeletedMessages can > > > > already scale to 1,000,000 ack holes and beyond when the broker is > > > > properly configured. It could be tested with this type of configuration: > > > > > > > > managedLedgerMaxUnackedRangesToPersist=1000000 > > > > managedLedgerMaxBatchDeletedIndexToPersist=1000000 > > > > managedLedgerPersistIndividualAckAsLongArray=true > > > > managedCursorInfoCompressionType=LZ4 > > > > managedLedgerInfoCompressionType=LZ4 > > > > > > > > (The last config is unrelated, but it makes sense to also switch to > > > > using compression.) > > > > > > > > I hope you could also analyze these alternatives before we proceed with > > > > making the decision on solving the hot (or slow) key problem. Thank you > > > > for focusing on solving this problem! > > > > > > > > -Lari > > > > > > > > On 2026/05/07 05:18:35 xiangying meng wrote: > > > > > Hi all, > > > > > > > > > > I'd like to propose PIP-474: Key_Shared Hot Key Overflow Mechanism. > > > > > > > > > > Key_Shared is Pulsar's only built-in solution for parallel consumption > > > > > with per-key ordering. But it has a critical production issue: a > > > > > single stuck consumer can starve ALL other keys across ALL partitions > > > > > within minutes, due to the containsStickyKeyHash ordering check > > > > > flooding the Replay queue. > > > > > > > > > > This becomes especially urgent as AI inference workloads adopt MQ as > > > > > their transport layer — slow consumption (seconds per request) plus > > > > > strict per-key ordering is exactly what Key_Shared is designed for, > > > > > yet the hot-key starvation bug makes it unusable in production. > > > > > > > > > > PIP-474 proposes diverting hot-key messages to an independent Overflow > > > > > ManagedLedger, unblocking Normal Read and mark-delete advancement > > > > > while preserving at-least-once delivery and per-key ordering. Zero > > > > > overhead when no hot keys are present. > > > > > > > > > > PIP: https://github.com/apache/pulsar/pull/25706 > > > > > > > > > > Feedback welcome. > > > > > > > > > > Thanks, Xiangying Meng > > > > > > > > >
