Hi Rajan,

10M unack-message ranges can respond to most use-cases
but there are still some scenarios that cannot be dealt with.
We can make an abstraction for users who want to extend the implementation.
I think this will not conflict with your improvement.

This PIP not just to improve the persistent of the individual acks,
there are others that the PR does not been covered such as the individual
acks memory limitation.
Every time mark delete changes, we need to write a few MB of data to
bookies, if you have 1000 topics with
this scenario, it's also a point to worry about.
We do not expect one implementation to solve all the problems of users.

BTW, I don't think introducing an abstraction will bring complexity in
message acknowledgment.
This is help to make the various components of managed-ledger are clearly
blamed.

Thanks
Penghui



Rajan Dhabalia <rdhaba...@apache.org> 于2021年1月27日周三 下午12:00写道:

> I have created a PR which should allow brokers to store up to 10M
> unack-message ranges. I think it should be large enough for any usecases
> and probably now, we might not need to introduce abstraction for ack
> management to avoid any further complexity in message acknowledgement path
> as well.
> https://github.com/apache/pulsar/pull/9292
>
> Thanks,
> Rajan
>
>
>
> On Sun, Jan 24, 2021 at 7:14 PM Sijie Guo <guosi...@gmail.com> wrote:
>
> > Rajan,
> >
> > Thank you for sharing the prototype! That looks great to me.
> >
> > In order for the community to evolve and experiment with different
> > approaches, how about we abstract this acknowledgment management as an
> > interface?
> >
> > If this approach works for you, maybe Lin Lin can focus on abstracting
> this
> > interface. Then we can implement your approach and his approach using
> this
> > interface. Users can decide which implementation to use.
> >
> > Thanks,
> > Sijie
> >
> > On Sat, Jan 23, 2021 at 7:46 PM Rajan Dhabalia <rdhaba...@apache.org>
> > wrote:
> >
> > > *Hi, >> Delayed messages or certain user logic can introduce a lot of
> > > message-holes. We have seen this issue in quite a lot of customers'
> > > production environment.I agree that we need a larger buffer to store
> and
> > > recover individually deleted messages, and the existing 150K limit
> might
> > > not be enough for many usescases. However, I would also like to
> highlight
> > > issues that happened in the past due to keeping a large number of
> deleted
> > > messages in the broker’s memory and one of them was high GC pauses.
> > > Therefore, we introduced ConcurrentOpenLongPairRangeSet
> > > <https://github.com/apache/pulsar/pull/3818> to manage deleted
> messages
> > > without actually storing range objects in memory. OpenRangeSet uses
> > bitset
> > > to store ranges in memory and we can also utilize it to persist in disk
> > for
> > > the recovery. This approach has various advantages: simple
> > implementation,
> > > large enough range for recovery, and it skips intermediate conversion
> > from
> > > unack-messages to bitset in OpenRangeSet which saves extra CPU while
> > > recovery.I implemented a simple prototype
> > > <
> > >
> >
> https://github.com/rdhabalia/pulsar/commit/1f8e5e745e9f1d1429697b5dee1da70545385653
> > > >
> > > to store deleted messages using bitset in OpenRangeSet and we can
> persist
> > > 10M ranges with 5MB data size which I guess is large enough for any
> > > usecases. So, we can use this approach to solve the problem without
> > > introducing unnecessary complexity in managed-cursor.Thanks,Rajan*
> > >
> > > On Fri, Jan 22, 2021 at 7:52 PM Sijie Guo <guosi...@gmail.com> wrote:
> > >
> > > > Joe - Delayed messages or certain user logic can introduce a lot of
> > > message
> > > > holes. We have seen this issue in quite a lot of customers'
> production
> > > > environment. Hence we need to find a solution for solving these
> > problems.
> > > > If you are skeptical of an implementation like that, how about us
> > making
> > > > cursor implementation pluggable. We can make this proposal
> implemented
> > as
> > > > one plugin. So it will not impact any existing logic but allowing
> > people
> > > > use a plugin to solve this problem.
> > > >
> > > > Thanks,
> > > > Sijie
> > > >
> > > > On Fri, Jan 22, 2021 at 5:00 PM Joe Francis
> > > <j...@verizonmedia.com.invalid
> > > > >
> > > > wrote:
> > > >
> > > > > Let me take a step back and explain  how I am looking at this from
> a
> > > > > high-level
> > > > > design viewpoint
> > > > >
> > > > >
> > > > > Bookkeeper (BK) is like an LSM implementation of a KV store. Writes
> > to
> > > > all
> > > > > keys are appended to a single file; deletes are logical.
> Compaction
> > > > > reclaims space.  An Index is used locate entries, tracking logical
> > > > deletes
> > > > > and reclaim space.
> > > > >
> > > > >
> > > > > The index in BK  is another LSM.  Again, writes are appended,
> deletes
> > > are
> > > > > logical, and  an index is used to  locate entries , account for
> > deletes
> > > > and
> > > > > compaction to reclaim space (the implementation within rocksdb is
> far
> > > > more
> > > > > complex with bloom filters and memtables, but you get the idea )
>  BK
> > > > just
> > > > > uses a sophisticated index (rocksdb) which is tiny and cacheable
> and
> > > > > rocksdb has within it a sophisticated index which is small and
> > > cacheable
> > > > >
> > > > >
> > > > > So when I look at this proposal, what I see is the same - another
> > > attempt
> > > > > to build an LSM with a sophisticated index/cache mechanism using
> log
> > > > > structured storage. So I am quite skeptical that this needs to
> solved
> > > > this
> > > > > way,  within Pulsar.
> > > > >
> > > > >
> > > > >
> > > > > Joe
> > > > >
> > > > > On Wed, Jan 20, 2021 at 12:30 AM linlin <lin...@apache.org> wrote:
> > > > >
> > > > > > We can look at
> > ManagedCursorImpl.buildIndividualDeletedMessageRanges
> > > > > >
> > > > > > What is saved in the entry is not a bitSet, but a messageRange
> one
> > by
> > > > > one,
> > > > > > which contains information such as ledgerId and entryId. BitSet
> > only
> > > > > exists
> > > > > > in the memory and is used to quickly determine whether it already
> > > > exists.
> > > > > > In addition, the position of each ack will be stored in the
> > > > > > individualDeletedMessages queue. When persisted to the entry, the
> > > queue
> > > > > > will be traversed, and the position information of each ack will
> > > > > generate a
> > > > > > messageRange.
> > > > > > A messageRange contains lowerEndpoint (ledgerId+entryId),
> > > upperEndpoint
> > > > > > (ledgerId+entryId), 4 longs, about 256 bits.
> > > > > >
> > > > > > We assume a more extreme scenario, 300K messages, every other ack
> > has
> > > > an
> > > > > > unacknowledged, that is, 150K location information will be stored
> > in
> > > > > > individualDeletedMessages. 150K * 256/8/1024 /1024 ≈ 4.6MB
> > > > > > Of course, there are also scenarios where the customer's ack
> spans
> > > > > several
> > > > > > ledgers.
> > > > > >
> > > > > >
> > > > > > On 2021/01/20 00:38:47, Joe F <j...@gmail.com> wrote:
> > > > > > > I have a simpler question. Just storing the message-ids raw
> will
> > > fit
> > > > > > ~300K>
> > > > > > > entries in one ledger entry. With the bitmap  changes, we can
> > store
> > > > a>
> > > > > > > couple of million  within one 5MB ledger entry.  So can you
> tell
> > us
> > > > > what>
> > > > > > > numbers of unacked messages are  creating a problem?  What
> > exactly
> > > > are
> > > > > > the>
> > > > > > > issues you face, and at what numbers of unacked messages/memory
> > use
> > > > > etc?>
> > > > > > >
> > > > > > > I have my own concerns about this proposal, but I would like to
> > > > > > understand>
> > > > > > > the problem first>
> > > > > > >
> > > > > > > Joe>
> > > > > > >
> > > > > > > On Sun, Jan 17, 2021 at 10:16 PM Sijie Guo <gu...@gmail.com>
> > > wrote:>
> > > > > > >
> > > > > > > > Hi Lin,>
> > > > > > > >>
> > > > > > > > Thanks you and Penghui for drafting this! We have seen a lot
> of
> > > > pain
> > > > > > points>
> > > > > > > > of `managedLedgerMaxUnackedRangesToPersist` when enabling
> > delayed
> > > > > > messages.>
> > > > > > > > Glad that you and Penghui are spending time on resolving
> this!>
> > > > > > > >>
> > > > > > > > Overall the proposal looks good. But I have a couple of
> > questions
> > > > > about
> > > > > > the>
> > > > > > > > proposal.>
> > > > > > > >>
> > > > > > > > 1. What happens if the broker fails to write the entry
> marker?
> > > For
> > > > > > example,>
> > > > > > > > at t0, the broker flushes dirty pages and successfully writes
> > an
> > > > > entry>
> > > > > > > > marker. At t1, the broker tries to flushes dirty pages but
> > failed
> > > > to
> > > > > > write>
> > > > > > > > the new entry marker. How can you recover the entry marker?>
> > > > > > > >>
> > > > > > > > 2.  When a broker crashes and recovers the managed ledger,
> the
> > > > > cursor>
> > > > > > > > ledger is not writable anymore. Are you going to create a new
> > > > cursor
> > > > > > ledger>
> > > > > > > > and copy all the entries from the old cursor ledger to the
> new
> > > > one?>
> > > > > > > >>
> > > > > > > > It would be good if you can clarify these two questions.>
> > > > > > > >>
> > > > > > > > - Sijie>
> > > > > > > >>
> > > > > > > > On Sun, Jan 17, 2021 at 9:48 PM linlin <li...@apache.org>
> > > wrote:>
> > > > > > > >>
> > > > > > > > > Hi, community:>
> > > > > > > > >     Recently we encountered some problems when using
> > > individual>
> > > > > > > > > acknowledgments, such as:>
> > > > > > > > > when the amount of acknowledgment is large, entry writing
> > > fails;
> > > > a
> > > > > > large>
> > > > > > > > > amount of cache causes OOM, etc.>
> > > > > > > > > So I drafted a PIP in `>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing`
> <https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing>
> > <
> https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing
> >
> > > <
> >
> https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing
> > >
> > > > <
> > >
> >
> https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing
> > > >
> > > > > <
> > > >
> > >
> >
> https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing
> > > > >
> > > > > > <
> > > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing
> > > > > >
> > > > > > >
> > > > > >
> > > > > > > > <
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing
> > > > > > >>
> > > > > >
> > > > > > > > > <>
> > > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/1uQtyb8t6X04v2vrSrdGWLFkuCkBcGYZbqK8XsVJ4qkU/edit?usp=sharing
> > > > > > >
> > > > > >
> > > > > > > > >>
> > > > > > > > > ,>
> > > > > > > > > any voice is welcomed.>
> > > > > > > > >>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to