Skip to content
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

Closed
cerebrotecnologico opened this issue Mar 21, 2022 · 10 comments · Fixed by #20363
Closed

BadVersionException , unable to consume topic, failing to reconnect #14779

cerebrotecnologico opened this issue Mar 21, 2022 · 10 comments · Fixed by #20363
Assignees
Labels
lifecycle/stale Stale type/bug The PR fixed a bug or issue reported a bug

Comments

@cerebrotecnologico
Copy link

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:
image

@codelipenghui
Copy link
Contributor

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

@hangc0276 hangc0276 self-assigned this Mar 22, 2022
@cerebrotecnologico
Copy link
Author

cerebrotecnologico commented Apr 20, 2022

We observed the same issue again. Here are the logs from the broker:
untitled_discover_search.csv

Again, we unloaded the affected topic and our app was able to consume the messages.

@github-actions
Copy link

The issue had no activity for 30 days, mark with Stale label.

@github-actions
Copy link

The issue had no activity for 30 days, mark with Stale label.

@Shawyeok
Copy link
Contributor

Shawyeok commented May 8, 2023

Similar issue: #13489

@Shawyeok
Copy link
Contributor

Shawyeok commented May 8, 2023

We are encountering the same issue with the Pulsar 2.8.1 codebase. Here are my observations:

  1. The occurrence of this issue for a specific broker/topic/subscription is unpredictable.

  2. This issue only arises in our test environment (with limited resources) after the overload shedder is triggered:

2023-05-07T06:18:15.865Z [pulsar-load-manager-1-1] INFO org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl - [Overload shedder] Unloading bundle: platform/dp/0x40000000_0x80000000 from broker 172.17.5.176:8080

The log for the new topic owner broker indicates that the managed-ledger initializes successfully, but it doesn't load any cursor.
image

When a user attempts to consume this topic, they'll encounter the following error: org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /managed-ledgers/platform/dp/persistent/resume.complete.result-ai.bert.cv.change.nlp.sub-DLQ/ai.bert.cv.change.nlp.sub

  1. Unload the topic resolves the problem.

@Shawyeok
Copy link
Contributor

Shawyeok commented May 8, 2023

After analyze the broker log, I'm suspect that get cursors of the managed-ledger in new owner broker returns empty result (consumers.isEmpty() is true), hence no cursor was load.

private void initializeCursors(final ManagedLedgerInitializeLedgerCallback callback) {
if (log.isDebugEnabled()) {
log.debug("[{}] initializing cursors", name);
}
store.getCursors(name, new MetaStoreCallback<List<String>>() {
@Override
public void operationComplete(List<String> consumers, Stat s) {
// Load existing cursors
final AtomicInteger cursorCount = new AtomicInteger(consumers.size());
if (log.isDebugEnabled()) {
log.debug("[{}] Found {} cursors", name, consumers.size());
}
if (consumers.isEmpty()) {
callback.initializeComplete();
return;
}
if (!ManagedLedgerImpl.this.config.isLazyCursorRecovery()) {
log.debug("[{}] Loading cursors", name);
for (final String cursorName : consumers) {
log.info("[{}] Loading cursor {}", name, cursorName);

But cursor znode ctime indicates cursor is already created, but getChildren returns empty? This is the part that confuses me.

[zk: localhost:2181(CONNECTED) 0] stat /managed-ledgers/platform/dp/persistent/resume.complete.result-ai.bert.cv.change.nlp.sub-DLQ/ai.bert.cv.change.nlp.sub
cZxid = 0x1510c7a7d2
ctime = Sat May 06 13:54:54 CST 2023
mZxid = 0x1510de2965
mtime = Sun May 07 14:18:15 CST 2023
pZxid = 0x1510c7a7d2
cversion = 0
dataVersion = 1
aclVersion = 0
ephemeralOwner = 0x0
dataLength = 34
numChildren = 0
[zk: localhost:2181(CONNECTED) 1] stat /managed-ledgers/platform/dp/persistent/resume.complete.result-ai.bert.cv.change.nlp.sub-DLQ
cZxid = 0x1510c7a7ca
ctime = Sat May 06 13:54:54 CST 2023
mZxid = 0x1510f5986f
mtime = Mon May 08 11:31:12 CST 2023
pZxid = 0x1510c7a7d2
cversion = 1
dataVersion = 3
aclVersion = 0
ephemeralOwner = 0x0
dataLength = 9
numChildren = 1
[zk: localhost:2181(CONNECTED) 2] 2023-05-08 16:37:23,195 [myid:] - ERROR [main:ServiceUtils@42] - Exiting JVM with code 0

btw, the version of zookeeper server is 3.6.2.

I added a log here to try verify my hypothesis above.

if (consumers.isEmpty()) {
  log.info("[{}] No cursors to recover", name);
  callback.initializeComplete();
  return;
}

More updates is coming...

@Shawyeok
Copy link
Contributor

Shawyeok commented May 19, 2023

My hypothesis is true, here is the steps to reproduce this problem in a multi nodes cluster:

  1. Create a topic and identify its owning broker, referred to as 'Node1'.
  2. Delete the topic and unload the associated bundle, resulting in the bundle being relocated to another node, denoted as 'Node2'.
  3. Recreate the topic, and initiate a consumer associated with it. At this point, the consumer will establish a connection with 'Node2'.
  4. Unload the topic's bundle once more, triggering a return to 'Node1'. Concurrently, the consumer will disconnect from 'Node2' and establish a new connection with 'Node1', generating a 'BadVersion' error at this stage.

The root cause is in step 2. Broker didn't invalidate childrenCache in AbstractMetadataStore, childrenCache is a Caffeine cache with refreshAfterWrite policy (refreshAfterWrite will return old value in the first stale request). Hence in step 4, store.getCursors will return an empty list even there is a cursor in zk.

The fix is quiet straightforward, just invalidate childrenCache in org.apache.pulsar.metadata.impl.AbstractMetadataStore#delete

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 2.8.1 codebase, but I believe this problem can be reproduced in the master branch, after #14154 , just ensure that steps 2 to 4 are completed within 5 minutes (AbstractMetadataStore #CACHE_REFRESH_TIME_MILLIS).

@Shawyeok
Copy link
Contributor

I believe this problem can be reproduced in the master branch, after #14154, just ensure that steps 2 to 4 are completed within 5 minutes (AbstractMetadataStore #CACHE_REFRESH_TIME_MILLIS).

I was wrong about above, #11198 use zk persistent watchs, it'd also fix this issue

@tisonkun
Copy link
Member

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.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
lifecycle/stale Stale type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants