I'm glad you're concerned about the lost schema ledger. Here are some of the fixes I tried for lost schema ledger: https://github.com/apache/pulsar/issues/20414 https://github.com/apache/pulsar/pull/20415
And the previous related discussions. https://lists.apache.org/thread/zbkpypb1yw89op510f7zg7cdkns1om0q At the beginning of this discussion, I made some summary, which is directly quoted here. SiNan Liu <liusinan1...@gmail.com> > 2024年4月16日 21:51 > 发送至 dev > #17221 describes an environment when multiple bookie copies are corrupted, > or a Ledger has been deleted. The loss of schema ledger results in new > producers and consumers not even being created and working properly. > At present, if the integrity of the schema is damaged, it cannot be > repaired because this function does not exist at present. > But the current behavior is that even if the scheme is lost, the connected > producers and consumers can work normally. > So we need to discuss solutions for the schema that has been lost: > 1. The first is to skip the non-recoverable ledger error. > - Description in https://github.com/apache/pulsar/pull/18010: If enabled > autoSkipNonRecoverableData, when the schema ledger is lost, the consumer > and producer can add new schemas without compatibility check(because the > original schema definition cannot be found). > - Description in https://github.com/apache/pulsar/pull/22469: Schema > should be recovered if schema ledger is failing to open due to > non-recoverable ledger error. > The second PR has been Merged, which causes producers and consumers who > are already connected may not work properly. > https://github.com/apache/pulsar/pull/22469#issuecomment-2057198666 > Compared with #18010, there is no configuration to control this behavior. > The default behavior is to automatically skip when the integrity of the > schema is destroyed. > 2. If we don't just skip error, we can fix the schema in some way to > maintain the integrity of the schema version. Even if this requires the > user to manually handle the missing schema, and this topic cannot be used > during this period. This is also better than just skipping the error. > Skipping errors will bring more problems. > - https://github.com/apache/pulsar/pull/20415 ( > https://github.com/apache/pulsar/issues/20414) > Currently this PR tries to fix the missing schema. > I hope you can discuss these two schemes and what to do with the #22469 > that has been Merged. > If for the second solution +1, we can talk about > https://github.com/apache/pulsar/issues/20414. The way to manually fix > the missing schema is described in the `Alternatives`. > I think we can add this functions to the `upload schema` admin api ( > https://pulsar.apache.org/docs/3.2.x/admin-api-schemas/#upload-a-schema) Thanks, sinan Lari Hotari <lhot...@apache.org> 于2024年10月9日周三 19:02写道: > 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 > > > > > >