aloyszhang opened a new issue, #18292:
URL: https://github.com/apache/pulsar/issues/18292

   ### Search before asking
   
   - [X] I searched in the [issues](https://github.com/apache/pulsar/issues) 
and found nothing similar.
   
   
   ### Version
   
   2.8.1
   
   ### Minimal reproduce step
   
   1. create a new topic
   2. start up multi producers with the same Schema
   3. after producers are created, check the schemas by `bin/pulsar-admin 
schemas get -a tenant/namespace/topic`
   4. you'll see many versions of schemas
   
   ### What did you expect to see?
   
   All producers with the same schema will only create one ledger to persistent 
schema entry.
   
   ### What did you see instead?
   
   Many duplicated ledgers for schema are created. like:
   
   ```SHELL
   $ bin/pulsar-admin schemas get t_schema/ns_schema/t13 -a
   {
     "name" : "t13",
     "schema" : "",
     "type" : "STRING",
     "timestamp" : 1666852962358,
     "properties" : { },
     "schemaDefinition" : ""
   }
   {
     "name" : "t13",
     "schema" : "",
     "type" : "STRING",
     "timestamp" : 1666852962365,
     "properties" : { },
     "schemaDefinition" : ""
   }
   {
     "name" : "t13",
     "schema" : "",
     "type" : "STRING",
     "timestamp" : 1666852962369,
     "properties" : { },
     "schemaDefinition" : ""
   }
   {
     "name" : "t13",
     "schema" : "",
     "type" : "STRING",
     "timestamp" : 1666852962365,
     "properties" : { },
     "schemaDefinition" : ""
   }
   {
     "name" : "t13",
     "schema" : "",
     "type" : "STRING",
     "timestamp" : 1666852962368,
     "properties" : { },
     "schemaDefinition" : ""
   }
   {
     "name" : "t13",
     "schema" : "",
     "type" : "STRING",
     "timestamp" : 1666852962368,
     "properties" : { },
     "schemaDefinition" : ""
   }
   
   
   ```
   
   ### Anything else?
   
   This problem increases many read and writes requests to zookeeper, which may 
make the pulsar cluster in an unstable state.
   As we saw, there are more than 80 thousand ledgers are created when starting 
1600 producers at the same time(in one minutes).
   
   ```shell
   $grep 'Creating producer' pulsar-broker.log | grep '19:43:' | wc -l
   1632
   
   >>
   19:43:26.994 [pulsar-io-4-102] INFO 
org.apache.pulsar.broker.service.ServerCnx - 
[][persistent://checker/checker/checker-partition-53] Creating producer. 
producerId=53
   19:43:26.994 [pulsar-io-4-102] INFO 
org.apache.pulsar.broker.service.ServerCnx - 
[][persistent://checker/checker/checker-partition-41] Creating producer. 
producerId=41
   19:43:26.994 [pulsar-io-4-102] INFO 
org.apache.pulsar.broker.service.ServerCnx - 
[][persistent://checker/checker/checker-partition-39] Creating producer. 
producerId=39
   19:43:26.994 [pulsar-io-4-102] INFO 
org.apache.pulsar.broker.service.ServerCnx - 
[][persistent://checker/checker/checker-partition-34] Creating producer. 
producerId=34
   19:43:26.994 [pulsar-io-4-102] INFO 
org.apache.pulsar.broker.service.ServerCnx - 
[][persistent://checker/checker/checker-partition-33] Creating producer. 
producerId=33
   19:43:26.995 [pulsar-io-4-102] INFO 
org.apache.pulsar.broker.service.ServerCnx - 
[][persistent://checker/checker/checker-partition-29] Creating producer. 
producerId=29
   
   
   $ grep 'LedgerCreateOp' pulsar-broker.log| grep '19:43:' | wc -l
   87112
   
   >>
   19:43:59.986 [main-EventThread] INFO  
org.apache.bookkeeper.client.LedgerCreateOp - Ensemble: [xxx] for ledger: 524473
   19:43:59.986 [main-EventThread] INFO  
org.apache.bookkeeper.client.LedgerCreateOp - Ensemble: [xxx] for ledger: 524474
   19:43:59.986 [main-EventThread] INFO  
org.apache.bookkeeper.client.LedgerCreateOp - Ensemble: [xxx] for ledger: 524475
   19:43:59.986 [main-EventThread] INFO  
org.apache.bookkeeper.client.LedgerCreateOp - Ensemble: [xxx] for ledger: 524476
   19:43:59.986 [main-EventThread] INFO  
org.apache.bookkeeper.client.LedgerCreateOp - Ensemble: [xxx] for ledger: 524479
   19:43:59.986 [main-EventThread] INFO  
org.apache.bookkeeper.client.LedgerCreateOp - Ensemble: [xxx] for ledger: 524480
   19:43:59.993 [main-EventThread] INFO  
org.apache.bookkeeper.client.LedgerCreateOp - Ensemble: [xxx] for ledger: 524556
   19:43:59.993 [main-EventThread] INFO  
org.apache.bookkeeper.client.LedgerCreateOp - Ensemble: [xxx] for ledger: 524558
   ```
   
   This means the broker create 87111 ledgers unexpectedly, these ledgers (also 
including the znode for ledger and schema) will keep existing until we delete 
schema forcefully. 
   
   
   
   ### Are you willing to submit a PR?
   
   - [X] I'm willing to submit a PR!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to