You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pulsar.apache.org by Yan Zhao <ho...@apache.org> on 2023/03/02 15:34:59 UTC

Re: [DISCUSS] PIP-186: Introduce two phase deletion protocol based on system topic

Hi, Asaf. Tune the pip-186(https://github.com/apache/pulsar/issues/16569), please help to view it again. thanks!

On 2023/02/16 13:50:54 Yan Zhao wrote:
> > If understood correctly, every broker will have a consumer right? You will
> > use a fail-over subscription? The retry-topic is consumed by the same
> > subscription, same consumer?
> Yes, you are right, there is the case you mention. The deletion is idempotent, I'm not sure if it's worth making it sync for it.
> 
> > In this very long mailing list thread, we have mentioned many fixes to be
> > done. Can you ping in the mailing list once you have managed to fix it all?
> 
> That's fine, I will push the new pip in two days. After the new pip pushing, I will ping you.
> 

Re: [DISCUSS] PIP-186: Introduce two phase deletion protocol based on system topic

Posted by Asaf Mesika <as...@gmail.com>.
On Sun, Mar 12, 2023 at 7:21 PM Yan Zhao <ho...@apache.org> wrote:

> > > In the LedgerDeletionService start, it will create a producer for
> sending
> > > pending delete ledger. When deleting a ledger, the producer sends
> > > PendingDeleteLedgerInfo to the system-topic. If the sent succeeds,
> delete
> > > the ledger id from the whole ledgers and update the ledgers into the
> meta
> > > store. If the sent fails, do nothing.
> >
> > You don't delete the ledger ID from the whole ledgers, then update the
> > ledgers in the metadata store. We described it above already; we remove
> the
> > pending delete ledger ID from the list of ledger IDs, publish the
> message,
> > and only at the end update the metadata store.
> Yes, correct it.
>
> > > In the LedgerDeletionService start, it will create a producer for
> sending
> > > pending delete ledger. When deleting a ledger, the producer sends
> > > PendingDeleteLedgerInfo to the system-topic. If the sent succeeds,
> delete
> > > the ledger id from the whole ledgers and update the ledgers into the
> meta
> > > store. If the sent fails, do nothing.
>
> > After that section, you write the methods of DeletionService and the
> > pseudo-code, but it's hard to understand what goes where: What exactly
> > deletion service does, and where the pseudo code will actually go into.
> It's just a summary, more detailed information on the later sections.
>
> > Also, let's concentrate on the implementation details in the
> Implementation
> > section High-Level Design. Let the readers understand the concept end to
> > end in the High-Level Design, and then in the Implementation, start
> diving
> > into the nitty gritty details.
>
>
> > > The consumer uses pulsar admin to send PendingDeleteLedgerInfo by the
> > > broker restful API.
>
> > Please explain here that you will introduce a new Admin API for that.
> Also,
> > explain that you will issue that API against the broker who owns the
> topic
> > (The schema registry has a ledger per topic?)
> correct it.
>
> > I still don't understand which methods or behavior you're changing
> exactly
> > for the managed ledger, cursor, and schema.
> > Why am I asking?
> > - Ledger deletion can happen because:
> > * Background truncation since ledgers are allowed to be deleted based on
> > expiration policy.
> > * Truncate request by an admin API.
> > * More?
> All the ways you mention do the same work: trigger
> ManagedLedgerImpl#internalTrimLedgers.
> We change the implementation approach in the internalTrimLedgers.
>
> > If you find the ledger ID is still in the meta-store, you return a
> failure,
> > and the consumer acks the message, right?
> > If it's background deletion, all good since it will kick in again in a
> few
> > seconds and retry.
> > But what happens if a user runs a REST admin API command to truncate the
> > topic?
> Using the REST admin API command to truncate the topic is not different
> from the background task. It's happening in the first metadata deletion
> phase, we didn't check whether the ledger is in the metadata or not, the
> first phase only picks all consumed ledgers, then try to delete them.
>
> > > check if the ledger is still in use,
>
> > You mean, check if the ledger ID still exists in the ledger ID list of
> the
> > topic in the metadata store, right?
> >
> >  then check the PendingDeleteLedgerInfo ledgerType.
> >
> > Check for what?
> see 3.1, 3.2.
>
>
> > > 2.2 If the topic not exists, the consumer checks the
> > > PendingDeleteLedgerInfo ledgerType.
> > > 3.1 If the ledgerType is Ledger, delete the data at the bookkeeper
> side.
> > > 3.2 If the ledgerType is Offload-Ledger, delete the data at the
> > > tiered-storage side.
>
> > Something doesn't look good in the numbering and how you structure it.
> 3.1
> > is probably an indent inside 2.2, no?
> Sorry, it may confuse the user. I add the flow hint in the pip.
>
>
> > 1. What happens if the topic doesn't exist when you are in the broker in
> > the delete pending ledger API implementation?
> The broker will respond 404 Topic not found an error to the consumer, the
> consumer knows the topic already does not exist, it will delete the ledger
> locally.
>
> You need to write that in your PIP. In your pseudo code you make it appear
as if the failure is boolean, but in fact the response should be an error
code.



> > 2. Say the ledger ID is still in the metadata store. If the background
> > truncation of the managed ledge was the trigger, there is no real point
> in
> > retrying, right? Once you start over, another message will be sent to the
> > system topic.
> In the storage deletion phase, If the ledger id is still in the metadata
> store, we think it will be used, we didn't delete it and ack the message.
> Due to the ledger is still in the metadata store, it will be deleted at
> the next background truncation task. So the ledger will be deleted finally.
>
>
> > 3. Can you explain exactly the cases in which it is actually worth doing
> > the retry?
> > I mean the obvious: you failed to delete it from BK due to a transient
> > error. But the previous one of checking the list? worth the retry?
> If the ledger is still in the meta store at the second storage deletion,
> we won't retry. we will ack the message to avoid useless retry.
> We retry deletion only on the storage delete failed case.
>
You need to make it clear.
Your response to new REST ADMIN API should be an error code:
TOPIC_NOT_EXISTS
LEDGER_STILL_IN_METADATA_STORE
STORAGE_DELETION_FAILED

then retry only the last type of error.


>
>
> > then respond to the error msg "ledger still in use" to the consumer.
>
> > You need to have those error codes in your pseudo code.
> > and your consumer pseudo code should check and act accordingly
> Fine. correct it.
>
I haven't seen it.
Also you need to make it clear also in the interface method definition of
the REST API of delete pending ledger.



>
> > > For security, the hacker may fake a PendingDeleteLedgerInfo and send
> it to
> > > the system-topic. So we need to do some checks works before deleting
> the
> > > data.
> > >
> > I think we’re over-engineering this or overprotecting.
> > Today, anybody with access to Pulsar Admin (who has a token with a role)
> > can do a lot of harm - delete an entire topic, and truncate the topic.
> Why
> > do you want to protect yourself from something like that? It’s too much,
> > IMO.
> Ok, we need more discussion about this case. I will ack for more eyes.
>
> > > Build an offloader according to the
> PendingDeleteLedgerInfo.offloadContext
> > >
> > >  Isn’t that expensive?
> We must use offloaded to delete the offload ledger. We can use the cache
> to avoid building the same offloader.
>
>
> > > There are some errors but we think it was successful, and avoid
> retrying
> > > it again.
> > >
> > >    - The Ledger is Still in use
> > >    - The PendingDeleteLedgerInfo is not matched with the bookkeeper
> > >    ledger metadata
> > >    - The data is not exist in the bookkeeper or tiered-storage
> > >    - The PendingDeleteLedgerInfo param is not incorrect, miss some
> > >    essential property
> > >
> > >
> > Maybe specify for each your resolution:
> > * For ledger ID that still exists in the metadata store - don't retry,
> ack
> > the message because ...
> > * ...
> Correct it.
>
> >
> > > Add new description API in pulsarAdmin.BrokerStats for getting it.
> >
> > What?
> Could you describe it detailed?
>

What is a description API in BrokerStats?


>
> >
> > So LedgerDeletionService.asyncDelete*() calls
> > ManagedLedger.asyncDeleteLedger()?
> No, it invokes bookkeeper.asyncDeleteLedger or offloader.deleteOffloaded
>
> >
> > > Consumer use pulsar.admin().topics().asyncDeleteLedger() to send
> request
> > > to the broker
>
> >  So, I presume we don't really want users to have that ability, right?
> > It should be "private" for the users.
>
> > But once we expose it via AdminAPI, it's out there, ready to be used by
> > anyone.
> Yes. So now, we check if the bookkeeper metadata is whether matches the
> param, to avoid mistake operations.
>
>
> > 1. Perhaps we should introduce a new concept of private AdminAPI which
> can
> > only be consumed internally by other brokers.
> > 2. What's the needed authorization to make such a delete command of a
> > ledger?
> Sounds good, we can draft a new pip to discuss it.
>

Agree, but I wouldn't continue with this PIP before you have that private
admin API.

Penghui suggested instead of communicating via Admin API we use the same
methodology used in the new load balancer - using a compacted topic as a
queue:
The system topic you create will have compaction enabled.
You write the pending ledger delete, and the key is ledger-ID and
ledger-type (BK, offloaded).
All brokers will consume this compacted topic as table-view. Basically upon
restart, you read all messages into memory.
When you reading you only keep the messages related to topic the broker
owns, others you discard (table view have filter feature).
For each message in table-view, you delete the ledgers, then write a
message to the topic with same key, null message, meaning it will be
deleted.

This way you don't need to invent a new private Admin API.

WDYT?

Re: [DISCUSS] PIP-186: Introduce two phase deletion protocol based on system topic

Posted by Yan Zhao <ho...@apache.org>.
> > In the LedgerDeletionService start, it will create a producer for sending
> > pending delete ledger. When deleting a ledger, the producer sends
> > PendingDeleteLedgerInfo to the system-topic. If the sent succeeds, delete
> > the ledger id from the whole ledgers and update the ledgers into the meta
> > store. If the sent fails, do nothing.
>
> You don't delete the ledger ID from the whole ledgers, then update the
> ledgers in the metadata store. We described it above already; we remove the
> pending delete ledger ID from the list of ledger IDs, publish the message,
> and only at the end update the metadata store.
Yes, correct it.

> > In the LedgerDeletionService start, it will create a producer for sending
> > pending delete ledger. When deleting a ledger, the producer sends
> > PendingDeleteLedgerInfo to the system-topic. If the sent succeeds, delete
> > the ledger id from the whole ledgers and update the ledgers into the meta
> > store. If the sent fails, do nothing.

> After that section, you write the methods of DeletionService and the
> pseudo-code, but it's hard to understand what goes where: What exactly
> deletion service does, and where the pseudo code will actually go into.
It's just a summary, more detailed information on the later sections. 

> Also, let's concentrate on the implementation details in the Implementation
> section High-Level Design. Let the readers understand the concept end to
> end in the High-Level Design, and then in the Implementation, start diving
> into the nitty gritty details.


> > The consumer uses pulsar admin to send PendingDeleteLedgerInfo by the
> > broker restful API.

> Please explain here that you will introduce a new Admin API for that. Also,
> explain that you will issue that API against the broker who owns the topic
> (The schema registry has a ledger per topic?)
correct it.

> I still don't understand which methods or behavior you're changing exactly
> for the managed ledger, cursor, and schema.
> Why am I asking?
> - Ledger deletion can happen because:
> * Background truncation since ledgers are allowed to be deleted based on
> expiration policy.
> * Truncate request by an admin API.
> * More?
All the ways you mention do the same work: trigger ManagedLedgerImpl#internalTrimLedgers. 
We change the implementation approach in the internalTrimLedgers.

> If you find the ledger ID is still in the meta-store, you return a failure,
> and the consumer acks the message, right?
> If it's background deletion, all good since it will kick in again in a few
> seconds and retry.
> But what happens if a user runs a REST admin API command to truncate the
> topic?
Using the REST admin API command to truncate the topic is not different from the background task. It's happening in the first metadata deletion phase, we didn't check whether the ledger is in the metadata or not, the first phase only picks all consumed ledgers, then try to delete them.

> > check if the ledger is still in use,

> You mean, check if the ledger ID still exists in the ledger ID list of the
> topic in the metadata store, right?
> 
>  then check the PendingDeleteLedgerInfo ledgerType.
> 
> Check for what?
see 3.1, 3.2.


> > 2.2 If the topic not exists, the consumer checks the
> > PendingDeleteLedgerInfo ledgerType.
> > 3.1 If the ledgerType is Ledger, delete the data at the bookkeeper side.
> > 3.2 If the ledgerType is Offload-Ledger, delete the data at the
> > tiered-storage side.

> Something doesn't look good in the numbering and how you structure it. 3.1
> is probably an indent inside 2.2, no?
Sorry, it may confuse the user. I add the flow hint in the pip.


> 1. What happens if the topic doesn't exist when you are in the broker in
> the delete pending ledger API implementation?
The broker will respond 404 Topic not found an error to the consumer, the consumer knows the topic already does not exist, it will delete the ledger locally.

> 2. Say the ledger ID is still in the metadata store. If the background
> truncation of the managed ledge was the trigger, there is no real point in
> retrying, right? Once you start over, another message will be sent to the
> system topic.
In the storage deletion phase, If the ledger id is still in the metadata store, we think it will be used, we didn't delete it and ack the message. 
Due to the ledger is still in the metadata store, it will be deleted at the next background truncation task. So the ledger will be deleted finally.


> 3. Can you explain exactly the cases in which it is actually worth doing
> the retry?
> I mean the obvious: you failed to delete it from BK due to a transient
> error. But the previous one of checking the list? worth the retry?
If the ledger is still in the meta store at the second storage deletion, we won't retry. we will ack the message to avoid useless retry. 
We retry deletion only on the storage delete failed case.


> then respond to the error msg "ledger still in use" to the consumer.

> You need to have those error codes in your pseudo code.
> and your consumer pseudo code should check and act accordingly
Fine. correct it.

> > For security, the hacker may fake a PendingDeleteLedgerInfo and send it to
> > the system-topic. So we need to do some checks works before deleting the
> > data.
> >
> I think we’re over-engineering this or overprotecting.
> Today, anybody with access to Pulsar Admin (who has a token with a role)
> can do a lot of harm - delete an entire topic, and truncate the topic. Why
> do you want to protect yourself from something like that? It’s too much,
> IMO.
Ok, we need more discussion about this case. I will ack for more eyes.

> > Build an offloader according to the PendingDeleteLedgerInfo.offloadContext
> >
> >  Isn’t that expensive?
We must use offloaded to delete the offload ledger. We can use the cache to avoid building the same offloader.


> > There are some errors but we think it was successful, and avoid retrying
> > it again.
> >
> >    - The Ledger is Still in use
> >    - The PendingDeleteLedgerInfo is not matched with the bookkeeper
> >    ledger metadata
> >    - The data is not exist in the bookkeeper or tiered-storage
> >    - The PendingDeleteLedgerInfo param is not incorrect, miss some
> >    essential property
> >
> >
> Maybe specify for each your resolution:
> * For ledger ID that still exists in the metadata store - don't retry, ack
> the message because ...
> * ...
Correct it.

> 
> > Add new description API in pulsarAdmin.BrokerStats for getting it.
> 
> What?
Could you describe it detailed?

> 
> So LedgerDeletionService.asyncDelete*() calls
> ManagedLedger.asyncDeleteLedger()?
No, it invokes bookkeeper.asyncDeleteLedger or offloader.deleteOffloaded

> 
> > Consumer use pulsar.admin().topics().asyncDeleteLedger() to send request
> > to the broker

>  So, I presume we don't really want users to have that ability, right?
> It should be "private" for the users.

> But once we expose it via AdminAPI, it's out there, ready to be used by
> anyone.
Yes. So now, we check if the bookkeeper metadata is whether matches the param, to avoid mistake operations.


> 1. Perhaps we should introduce a new concept of private AdminAPI which can
> only be consumed internally by other brokers.
> 2. What's the needed authorization to make such a delete command of a
> ledger?
Sounds good, we can draft a new pip to discuss it.
 

Re: [DISCUSS] PIP-186: Introduce two phase deletion protocol based on system topic

Posted by Asaf Mesika <as...@gmail.com>.
Thanks for the modifications. Looking much better!

In the LedgerDeletionService start, it will create a producer for sending
> pending delete ledger. When deleting a ledger, the producer sends
> PendingDeleteLedgerInfo to the system-topic. If the sent succeeds, delete
> the ledger id from the whole ledgers and update the ledgers into the meta
> store. If the sent fails, do nothing.

You don't delete the ledger ID from the whole ledgers, then update the
ledgers in the metadata store. We described it above already; we remove the
pending delete ledger ID from the list of ledger IDs, publish the message,
and only at the end update the metadata store.


> In the LedgerDeletionService start, it will create a producer for sending
> pending delete ledger. When deleting a ledger, the producer sends
> PendingDeleteLedgerInfo to the system-topic. If the sent succeeds, delete
> the ledger id from the whole ledgers and update the ledgers into the meta
> store. If the sent fails, do nothing.
>
> After that section, you write the methods of DeletionService and the
pseudo-code, but it's hard to understand what goes where: What exactly
deletion service does, and where the pseudo code will actually go into.

Also, let's concentrate on the implementation details in the Implementation
section High-Level Design. Let the readers understand the concept end to
end in the High-Level Design, and then in the Implementation, start diving
into the nitty gritty details.

The consumer uses pulsar admin to send PendingDeleteLedgerInfo by the
> broker restful API.

Please explain here that you will introduce a new Admin API for that. Also,
explain that you will issue that API against the broker who owns the topic
(The schema registry has a ledger per topic?)

I still don't understand which methods or behavior you're changing exactly
for the managed ledger, cursor, and schema.
Why am I asking?
- Ledger deletion can happen because:
* Background truncation since ledgers are allowed to be deleted based on
expiration policy.
* Truncate request by an admin API.
* More?

If you find the ledger ID is still in the meta-store, you return a failure,
and the consumer acks the message, right?
If it's background deletion, all good since it will kick in again in a few
seconds and retry.
But what happens if a user runs a REST admin API command to truncate the
topic?

check if the ledger is still in use,

You mean, check if the ledger ID still exists in the ledger ID list of the
topic in the metadata store, right?

 then check the PendingDeleteLedgerInfo ledgerType.

Check for what?


2.2 If the topic not exists, the consumer checks the
> PendingDeleteLedgerInfo ledgerType.

3.1 If the ledgerType is Ledger, delete the data at the bookkeeper side.

3.2 If the ledgerType is Offload-Ledger, delete the data at the
> tiered-storage side.


Something doesn't look good in the numbering and how you structure it. 3.1
is probably an indent inside 2.2, no?

---
1. What happens if the topic doesn't exist when you are in the broker in
the delete pending ledger API implementation?
2. Say the ledger ID is still in the metadata store. If the background
truncation of the managed ledge was the trigger, there is no real point in
retrying, right? Once you start over, another message will be sent to the
system topic.
3. Can you explain exactly the cases in which it is actually worth doing
the retry?
I mean the obvious: you failed to delete it from BK due to a transient
error. But the previous one of checking the list? worth the retry?

then respond to the error msg "ledger still in use" to the consumer.

You need to have those error codes in your pseudo code.
and your consumer pseudo code should check and act accordingly

For security, the hacker may fake a PendingDeleteLedgerInfo and send it to
> the system-topic. So we need to do some checks works before deleting the
> data.
>
> I think we’re over-engineering this or overprotecting.
Today, anybody with access to Pulsar Admin (who has a token with a role)
can do a lot of harm - delete an entire topic, and truncate the topic. Why
do you want to protect yourself from something like that? It’s too much,
IMO.

Build an offloader according to the PendingDeleteLedgerInfo.offloadContext
>
>  Isn’t that expensive?


> There are some errors but we think it was successful, and avoid retrying
> it again.
>
>    - The Ledger is Still in use
>    - The PendingDeleteLedgerInfo is not matched with the bookkeeper
>    ledger metadata
>    - The data is not exist in the bookkeeper or tiered-storage
>    - The PendingDeleteLedgerInfo param is not incorrect, miss some
>    essential property
>
>
> Maybe specify for each your resolution:
* For ledger ID that still exists in the metadata store - don't retry, ack
the message because ...
* ...

Add new description API in pulsarAdmin.BrokerStats for getting it.

What?

---

So LedgerDeletionService.asyncDelete*() calls
ManagedLedger.asyncDeleteLedger()?

----

> Consumer use pulsar.admin().topics().asyncDeleteLedger() to send request
> to the broker


 So, I presume we don't really want users to have that ability, right?
It should be "private" for the users.

But once we expose it via AdminAPI, it's out there, ready to be used by
anyone.

1. Perhaps we should introduce a new concept of private AdminAPI which can
only be consumed internally by other brokers.
2. What's the needed authorization to make such a delete command of a
ledger?


Thanks,

Asaf









On Sat, Mar 4, 2023 at 6:42 PM Yan Zhao <ho...@apache.org> wrote:

> Hi, Asaf, Tune the pip https://github.com/apache/pulsar/issues/16569,
> please help to review it again, thanks!
>

Re: [DISCUSS] PIP-186: Introduce two phase deletion protocol based on system topic

Posted by Yan Zhao <ho...@apache.org>.
Hi, Asaf, Tune the pip https://github.com/apache/pulsar/issues/16569, please help to review it again, thanks!