pulsar
pulsar copied to clipboard
[improve] [broker] Do not try to open ML when the topic meta does not exist and do not expect to create a new one. #21995
Motivation
When we call pulsar-admin topics stats <topic name>, Pulsar will try to open a managed ledger even if the topic metadata does not exist.
It costs a lot of resources when there are many requests trying to call pulsar-admin topics stats <topic name> on a deleted partition.
2024-01-30T15:52:17,143+0000 [bookkeeper-ml-scheduler-OrderedScheduler-2-0] ERROR org.apache.pulsar.broker.admin.v2.PersistentTopics - [[email protected]] Failed to get stats for persistent://xx/xx/xx-partition-0
java.util.concurrent.CompletionException: org.apache.pulsar.broker.web.RestException: Topic partitions were not yet created
at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:315) ~[?:?]
at java.util.concurrent.CompletableFuture.uniApplyNow(CompletableFuture.java:687) ~[?:?]
at java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:662) ~[?:?]
at java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:2168) ~[?:?]
at org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.topicNotFoundReasonAsync(PersistentTopicsBase.java:4519) ~[io.streamnative-pulsar-broker-3.0.2.1.jar:3.0.2.1]
at org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.lambda$getTopicReferenceAsync$438(PersistentTopicsBase.java:4499) ~[io.streamnative-pulsar-broker-3.0.2.1.jar:3.0.2.1]
at java.util.Optional.orElseGet(Optional.java:364) ~[?:?]
at org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.lambda$getTopicReferenceAsync$439(PersistentTopicsBase.java:4499) ~[io.streamnative-pulsar-broker-3.0.2.1.jar:3.0.2.1]
at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:1150) ~[?:?]
at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510) ~[?:?]
at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2147) ~[?:?]
at org.apache.pulsar.broker.service.BrokerService$2.openLedgerFailed(BrokerService.java:1800) ~[io.streamnative-pulsar-broker-3.0.2.1.jar:3.0.2.1]
at org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl.lambda$asyncOpen$8(ManagedLedgerFactoryImpl.java:421) ~[io.streamnative-managed-ledger-3.0.2.1.jar:3.0.2.1]
at java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:990) ~[?:?]
at java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:974) ~[?:?]
at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510) ~[?:?]
at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162) ~[?:?]
at org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl$2.initializeFailed(ManagedLedgerFactoryImpl.java:416) ~[io.streamnative-managed-ledger-3.0.2.1.jar:3.0.2.1]
at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$1.operationFailed(ManagedLedgerImpl.java:458) ~[io.streamnative-managed-ledger-3.0.2.1.jar:3.0.2.1]
at org.apache.bookkeeper.mledger.impl.MetaStoreImpl.lambda$getManagedLedgerInfo$3(MetaStoreImpl.java:150) ~[io.streamnative-managed-ledger-3.0.2.1.jar:3.0.2.1]
at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:718) ~[?:?]
at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:482) ~[?:?]
at org.apache.bookkeeper.common.util.OrderedExecutor$TimedRunnable.run(OrderedExecutor.java:201) ~[org.apache.bookkeeper-bookkeeper-common-4.16.3.jar:4.16.3]
at org.apache.bookkeeper.common.util.SingleThreadSafeScheduledExecutorService$SafeRunnable.run(SingleThreadSafeScheduledExecutorService.java:46) ~[org.apache.bookkeeper-bookkeeper-common-4.16.3.jar:4.16.3]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) ~[?:?]
at java.util.concurrent.FutureTask.run(FutureTask.java:264) ~[?:?]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304) ~[?:?]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?]
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.100.Final.jar:4.1.100.Final]
at java.lang.Thread.run(Thread.java:840) ~[?:?]
Caused by: org.apache.pulsar.broker.web.RestException: Topic partitions were not yet created
at org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.lambda$topicNotFoundReasonAsync$442(PersistentTopicsBase.java:4521) ~[io.streamnative-pulsar-broker-3.0.2.1.jar:3.0.2.1]
at java.util.concurrent.CompletableFuture.uniApplyNow(CompletableFuture.java:684) ~[?:?]
... 29 more
2024-01-30T15:52:17,143+0000 [bookkeeper-ml-scheduler-OrderedScheduler-2-0] INFO org.eclipse.jetty.server.RequestLog - 127.0.0.6 - - [30/Jan/2024:15:52:17 +0000] "GET /admin/v2/persistent/xx/xx/xx-partition-0/stats?getPreciseBacklog=true&authoritative=false&subscriptionBacklogSize=true&getEarliestTimeInBacklog=false HTTP/1.1" 404 50 "-" "Pulsar-Java-v2.11.1" 7
Modifications
- Do not try to open ML when the topic meta does not exist and do not expect to create a new one.
Behavior changes of TopicEventsListener
- Before changes: the listener will receive a
Topic Loadevent when callingpulsar-admin topics stats <topic name>even if the topic was not created. - After changes: the listener will not receive a
Topic Loadevent when callingpulsar-admin topics stats <topic name>if the topic was not created.
Documentation
- [ ]
doc - [ ]
doc-required - [x]
doc-not-needed - [ ]
doc-complete
Matching PR in forked repository
PR in forked repository: x