Let me answer each question though most of them were already covered in the
previous reply.

>> Firstly, could you submit an issue that you encountered a ledger lost
issue, and we should solve the issue

There are multiple issues already created about the missing schema ledgers
and that could happen in the system due to multiple reasons but the
question is how to handle it. Just for your reference below are the few of
the open issues because of missing schema ledgers and duplicates are keep
getting created because of it
https://github.com/apache/pulsar/issues/20414
https://github.com/apache/pulsar/issues/14533
https://github.com/apache/pulsar/issues/15267
https://github.com/apache/pulsar/issues/5792


>> Secondly, the ledger containing user messages will also be lost if the
schema ledger is lost.

yes, of course and that's why we have made brokers resilient enough to
handle this situation. and same handling should be available to broken
schema ledger as well.

>> Thirdly, after the PR, users will encounter a issue that is more worse
than
before. They can not consume the original messages before the schema ledger
is broken. After the PR they can not consume the messages continuously
published.

I don't think that is true. Previously connected consumers with correct
schema will fail to connect to the broker due to topic's unavailability but
once the broker handles it with its resilience nature, topic will be
available again and consumer should be able to reconnect and consume the
data.


Let's take a step back and understand the issue fundamentally without
jumping with predefined notion::

- While handling schema retrieval, the broker handles failure with
recoverable/non-recoverable exceptions.
- If error is non-recoverable, due to broker's resiliency, broker considers
that schema is already lost and the broker should move forward to create a
new schema ledger with a new producer/consumer connected.
- Now, for a second if we consider that brokers do not recover and let
system admin manually delete the schema metadata node so, broker can
recreate the schema then it will be the same thing that the broker did
during auto-recovery, We will face the same issues if we clean up schema
metadata node manually which we are talking in this email thread. Manual
clean up doesn't make any difference but yes, it makes a difference for
operational efforts. We are running with millions of topics on the Pulsar
cluster and we saw this issue happening for thousands of topics  due to
some reasons and system admin can not manually fix all thousands of topics.
In that case, we depend on the broker's resiliency to recover and make the
topic available.
- so, let's be honest here, depending on manual cleanup will not help and
we will end up with the same issues which we are talking about if the
broker handles recovery.


>> Makes sense, just wondering if the behavior should only be enabled when
config.isSchemaLedgerForceRecovery() == true (PIP-327)?

Yes, we can also do this, at least  large scale systems with millions of
topics can use this feature to build resilience in the system. I am sure,
this will bite many users and as usual we will see this PR again by someone
in future. Until then I have created a PR to resolve this disagreement with
your suggestion:
PR: https://github.com/apache/pulsar/pull/23428

Thanks,
Rajan






On Wed, Oct 9, 2024 at 4:02 AM Lari Hotari <lhot...@apache.org> wrote:

> Makes sense, just wondering if the behavior should only be enabled when
> config.isSchemaLedgerForceRecovery() == true (PIP-327)?
>
> -Lari
>
> On 2024/10/09 04:51:23 Rajan Dhabalia wrote:
> > >> When the schemas of a topic are lost, all of the messages in the topic
> > can not be consumed successfully, and producers can not publish messages
> > anymore.
> >
> > Well, there are already many issues created related to the lost schema
> > ledgers so, it is a very well known issue that the topic becomes
> > unavailable and producers are not able to publish messages anymore just
> > because of server issues which is extremely critical for mission critical
> > usecases and not acceptable as well. Because of that brokers must be
> fault
> > tolerant to recover in case of missing ledgers. Broker already handles
> such
> > issues during manged-ledger or managed-cursor legers but it should also
> > handle in case of schema ledger as well and make sure that broker won't
> > impact topic's unavailability. It is also very important to just not rely
> > on operational effort to fix such unavailability issues manually but
> broker
> > should have a mechanism to recover by itself,
> > Therefore, this PR is important to make sure tenants don't face topic
> > unavailability due to well known issues of missing schema ledgers.
> >
> > Thanks,
> > Rajan
> >
> > On Tue, Oct 8, 2024 at 8:29 PM Yubiao Feng
> > <yubiao.f...@streamnative.io.invalid> wrote:
> >
> > > Hi all
> > >
> > > Background: When the schemas of a topic are lost, all of the messages
> in
> > > the topic can not be consumed successfully, and producers can not
> publish
> > > messages anymore. This mechanism alerts users to try to recover their
> > > schemas or recreate their topics.
> > >
> > > https://github.com/apache/pulsar/pull/23395 added a patch: producers
> will
> > > rebuild schemas if the original schemas were lost, which will mix the
> old
> > > schema and new schema as the same schema ID. For example:
> > > - send M1 with schema `Int32`, get schema id: `1`
> > > - send M2 with schema `String`, get schema id: `2`
> > > - schemas are lost
> > > - send M3 with schema `String`, get schema id `1`
> > >
> > > The messages `M1` and `M3` use different schemas, but have the same
> schema
> > > id, but users assume all things are fine, which is dangerous. So I
> want to
> > > revert this PR.
> > >
> > > Thanks
> > > Yubiao Feng
> > >
> >
>

Reply via email to