-
Notifications
You must be signed in to change notification settings - Fork 3.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
BadVersionException , unable to consume topic, failing to reconnect #14779
Comments
Hi @cerebrotecnologico thanks for creating the issue, is it able to share the broker logs file? I want to check what happens before the error logs, this will help to investigate the problem. /cc @hangc0276 @zymap |
We observed the same issue again. Here are the logs from the broker: Again, we unloaded the affected topic and our app was able to consume the messages. |
The issue had no activity for 30 days, mark with Stale label. |
The issue had no activity for 30 days, mark with Stale label. |
Similar issue: #13489 |
After analyze the broker log, I'm suspect that get cursors of the managed-ledger in new owner broker returns empty result ( pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java Lines 498 to 520 in 09a6b58
But cursor znode ctime indicates cursor is already created, but getChildren returns empty? This is the part that confuses me.
btw, the version of zookeeper server is I added a log here to try verify my hypothesis above.
More updates is coming... |
My hypothesis is true, here is the steps to reproduce this problem in a multi nodes cluster:
The root cause is in step 2. Broker didn't invalidate childrenCache in The fix is quiet straightforward, just invalidate childrenCache in Index: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
--- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java (revision 850389a511e5ec86b772d5b501c0e96708901310)
+++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java (date 1684490211421)
@@ -191,6 +191,7 @@
return storeDelete(path, expectedVersion)
.thenRun(() -> {
existsCache.synchronous().invalidate(path);
+ childrenCache.synchronous().invalidate(path);
String parent = parent(path);
if (parent != null) {
childrenCache.synchronous().invalidate(parent); Notes: I've reproduced in |
Although, your patch won't bring new bugs or decrease the performance since the cache would be invalidated anyway. The impact is that early versions that won't be affected can choose not to pick the patch. |
Describe the bug
I am using Pulsar 2.8.0.
The topic is partitioned, 3 partitions.
The backlog quota is set to 50h
The TTL is set to 48h.
This is the second time that we observe this error. The application reads from 5 different topics, but only one topic has been affected twice.
This is the topic with the highest traffic processed by the same application. (Yet, our app is not really data intensive when compared to other use cases, it process less than 1 million events per hour)
My application suddenly stopped processing messages, tries to reconnect but keeps getting errors about ZK BadVersion.
[persistent://tenant/namespace/mytopic-partition-2] [platform_persistent://tenant/namespace/mytopic] Could not get connection to broker: java.util.concurrent.CompletionException: org.apache.pulsar.broker.service.BrokerServiceException$PersistenceException: org.apache.bookkeeper.mledger.ManagedLedgerException$BadVersionException: org.apache.pulsar.metadata.api.MetadataStoreException$BadVersionException: org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /managed-ledgers/tenant/namespace/persistent/my-topic-partition-2/platform_persistent%3A%2F%2Ftenant%2Fnamespace%2Fmytopic -- Will try again in 0.1 s
We unloaded the topic and the application restarted processing messages. (The application did not need to be restarted).
Any idea what causes this? How to prevent it.
To Reproduce
Unknown.
Expected behavior
I expected that the topic would be unloaded automatically, causing the clients to be able to reconnect to a new broker.
** Logs
Broker:
09:13:45.822 [pulsar-io-4-2] WARN org.apache.pulsar.broker.service.BrokerService - Namespace is being unloaded, cannot add topic persistent://platform/system/workflow-event-even-partition-0
09:13:45.823 [BookKeeperClientWorker-OrderedExecutor-1-0] WARN org.apache.pulsar.broker.service.AbstractTopic - [persistent://platform/system/workflow-event-even-partition-0] Attempting to add producer to a fenced topic
09:13:45.823 [BookKeeperClientWorker-OrderedExecutor-1-0] ERROR org.apache.pulsar.broker.service.ServerCnx - [/10.42.105.16:49756] Failed to add producer to topic persistent://platform/system/workflow-event-even-partition-0: producerId=289578, org.apache.pulsar.broker.service.BrokerServiceException$TopicFencedException: Topic is temporarily unavailable
09:13:45.844 [ForkJoinPool.commonPool-worker-3] WARN org.apache.pulsar.broker.service.BrokerService - Namespace bundle for topic (persistent://platform/system/workflow-event-even-partition-0) not served by this instance. Please redo the lookup. Request is denied: namespace=platform/system
09:13:45.854 [ForkJoinPool.commonPool-worker-3] WARN org.apache.pulsar.broker.service.BrokerService - Namespace bundle for topic (persistent://platform/system/workflow-event-even-partition-0) not served by this instance. Please redo the lookup. Request is denied: namespace=platform/system
09:13:45.877 [pulsar-2-2] WARN org.apache.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate - Broker pulsar-broker-2.pulsar-broker.pulsar.svc.cluster.local:8080 is overloaded: CPU: 96.51538%, MEMORY: 51.04191%, DIRECT MEMORY: 12.5%, BANDWIDTH IN: 2.6368966%, BANDWIDTH OUT: 0.3707455%
09:13:45.877 [pulsar-2-2] WARN org.apache.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate - Broker http://pulsar-broker-2.pulsar-broker.pulsar.svc.cluster.local:8080 is overloaded: max usage=0.9651538133621216
09:28:46.277 [bookkeeper-ml-scheduler-OrderedScheduler-1-0] WARN org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [platform/system/persistent/workflow-event-even-partition-2] Failed to update consumer platform_persistent%3A%2F%2Fplatform%2Fsystem%2Fworkflow-event-even_queue
at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$7.operationFailed(ManagedLedgerImpl.java:940) ~[io.streamnative-managed-ledger-2.8.2.0.jar:2.8.2.0]
at org.apache.bookkeeper.util.SafeRunnable$1.safeRun(SafeRunnable.java:43) [org.apache.bookkeeper-bookkeeper-server-4.14.3.jar:4.14.3]
09:28:46.281 [bookkeeper-ml-scheduler-OrderedScheduler-1-0] ERROR org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://platform/system/workflow-event-even-partition-2] Failed to create subscription: platform_persistent://platform/system/workflow-event-even_queue
java.util.concurrent.CompletionException: org.apache.pulsar.broker.service.BrokerServiceException$PersistenceException: org.apache.bookkeeper.mledger.ManagedLedgerException$BadVersionException: org.apache.pulsar.metadata.api.MetadataStoreException$BadVersionException: org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /managed-ledgers/platform/system/persistent/workflow-event-even-partition-2/platform_persistent%3A%2F%2Fplatform%2Fsystem%2Fworkflow-event-even_queue
at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:704) ~[?:?]
at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) ~[?:?]
at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088) ~[?:?]
at org.apache.pulsar.broker.service.persistent.PersistentTopic$3.openCursorFailed(PersistentTopic.java:887) ~[io.streamnative-pulsar-broker-2.8.2.0.jar:2.8.2.0]
at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$2.operationFailed(ManagedCursorImpl.java:567) ~[io.streamnative-managed-ledger-2.8.2.0.jar:2.8.2.0]
at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$23.operationFailed(ManagedCursorImpl.java:2348) ~[io.streamnative-managed-ledger-2.8.2.0.jar:2.8.2.0]
At the time this occurred, I saw long GC in a ZK node:
The text was updated successfully, but these errors were encountered: