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

Retry when zookeeper session expired at broker side. #6259

Closed
codelipenghui opened this issue Feb 7, 2020 · 4 comments
Closed

Retry when zookeeper session expired at broker side. #6259

codelipenghui opened this issue Feb 7, 2020 · 4 comments
Assignees
Labels
type/feature The PR added a new feature or issue requested a new feature
Milestone

Comments

@codelipenghui
Copy link
Contributor

Is your feature request related to a problem? Please describe.
Currently, when a broker met Zookeeper session expired event, the broker shutdown itself. When the broker or zookeeper servicer under high load and the session timeout is short, this can easily cause the broker to go down. There are some related issues #6251.

At Apache BookKeeper, when met Zookeeper session expired event, the bookie will re-register metadata. So broker also can refer to such processing.

@codelipenghui codelipenghui added the type/feature The PR added a new feature or issue requested a new feature label Feb 7, 2020
@codelipenghui codelipenghui self-assigned this Feb 7, 2020
@codelipenghui codelipenghui added this to the 2.6.0 milestone Feb 7, 2020
@merlimat
Copy link
Contributor

merlimat commented Feb 7, 2020

For pulsar broker the issue is slightly more complicated since there’s the concept of ownership of topics. Also we need to make sure to sync back all the metadata versions to account for failure of request during the session expired period.

@codelipenghui
Copy link
Contributor Author

Yes, thanks for reminding me. I will try to achieve it. If I encounter problems, I will ask here.

@codelipenghui
Copy link
Contributor Author

codelipenghui commented Feb 13, 2020

When the session timeout happens on the local Zookeeper, all of the EPHEMERAL znode maintained by this broker will delete automatically. We need some mechanism to avoid the unnecessary ownership transfer of the bundles. Since the broker cached the owned bundles in memory, the broker can use the cache to re-own the bundles.

Firstly, the broker should check if the znode exists for the bundle and the bundle owner is this broker. If the znode exists and the owner is this broker, it may be that the znode has not been deleted. The broker should check if the ephemeral owner is the current session ID. If not, the broker should wait for the znode deletion.

Then the broker tries to own the bundle. If the broker owns the bundle success means the bundle is not owned by other brokers, the broker should check whether to preload the topics under the bundle. If the broker failed to own the bundle means the bundle owned by another broker. The broker should unload the bundle.

@merlimat

Please help check if the processing idea about the ownership is correct, I checked other places using the local zookeeper, It should be easy to handle.

If the idea is right, I will draft a PIP

@lhotari
Copy link
Member

lhotari commented Oct 30, 2020

I was running into some issues where there were timeouts in the Pulsar client application logs. The Pulsar brokers were restarting. This was happening in a load test and it seemed that the broker restarts made the problem worse.

To mitigate the issue, the brokers were recently configured to use zookeeperSessionExpiredPolicy=reconnect setting. The broker zookeeper timeout related settings are at the default values (zooKeeperSessionTimeoutMillis=30000, zooKeeperOperationTimeoutSeconds=30), since it seems odd that the Zookeeper interactions would take longer since the CPU consumption looks very low on zk in the load test.

Now, we reran the load test and the brokers became unavailable. The logs were filled with this type of errors:

07:37:09.740 [pulsar-zk-session-watcher-5-1] ERROR org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - timeout expired for reconnecting, invoking shutdown service
07:37:11.740 [pulsar-zk-session-watcher-5-1] ERROR org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - timeout expired for reconnecting, invoking shutdown service
07:37:13.741 [pulsar-zk-session-watcher-5-1] ERROR org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - timeout expired for reconnecting, invoking shutdown service
07:37:15.741 [pulsar-zk-session-watcher-5-1] ERROR org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - timeout expired for reconnecting, invoking shutdown service

This is with zookeeperSessionExpiredPolicy=reconnect. Previously with the default setting (zookeeperSessionExpiredPolicy=shutdown), the broker would resume operations after a restart. Now all brokers in a 3 node cluster get unavailable.

The benefit of using zookeeperSessionExpiredPolicy=reconnect is that now it was possible to get a thread dump from the system when it got unavailable, although the broker didn't ever resume operations.

In the thread dump, there are quite a few hanging threads. Some stack traces are similar to what #8406 fixes.

"ForkJoinPool.commonPool-worker-7" #1258 daemon prio=5 os_prio=0 tid=0x00007f67a09f9000 nid=0x50a waiting on condition [0x00007f676c258000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x000000044ca60338> (a java.util.concurrent.CompletableFuture$Signaller)
	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
	at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1707)
	at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3313)
	at java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1742)
	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
	at com.github.benmanes.caffeine.cache.LocalAsyncLoadingCache$LoadingCacheView.get(LocalAsyncLoadingCache.java:400)
	at org.apache.pulsar.common.naming.NamespaceBundleFactory.getBundles(NamespaceBundleFactory.java:160)
	at org.apache.pulsar.broker.namespace.NamespaceService.getBundle(NamespaceService.java:192)
	at org.apache.pulsar.broker.namespace.NamespaceService.isTopicOwned(NamespaceService.java:913)
	at org.apache.pulsar.broker.namespace.NamespaceService.isServiceUnitOwned(NamespaceService.java:877)
	at org.apache.pulsar.broker.service.BrokerService.checkTopicNsOwnership(BrokerService.java:1359)
	at org.apache.pulsar.broker.service.persistent.PersistentTopic.subscribe(PersistentTopic.java:521)

Full thread dump of broker:
https://gist.github.com/lhotari/ba9c45cf4111a66f5e8960df4955bbf8
Online thread dump report: https://jstack.review/?https://gist.github.com/lhotari/ba9c45cf4111a66f5e8960df4955bbf8

The reason why I'm bringing this up in this issue is that a Zookeeper timeout exception in the broker logs can actually be caused by threads hanging in the broker. Hopefully #8406 gets also fixed in the upcoming Pulsar 2.6.2 version

UPDATE: I now noticed that there's an incoming PR #8304 (fix for #4635 which is also a deadlock).

@codelipenghui codelipenghui modified the milestones: 2.7.0, 2.8.0 Nov 4, 2020
@codelipenghui codelipenghui modified the milestones: 2.8.0, 2.9.0 May 21, 2021
@eolivelli eolivelli modified the milestones: 2.9.0, 2.10.0 Oct 6, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/feature The PR added a new feature or issue requested a new feature
Projects
None yet
Development

No branches or pull requests

5 participants