[ 
https://issues.apache.org/jira/browse/FLINK-32204?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17726613#comment-17726613
 ] 

Matthias Pohl commented on FLINK-32204:
---------------------------------------

The curator's {{TreeCache}} isn't thread-safe: The event processing within the 
cache happens is triggered from within the client's EventThread and calls 
{{TreeCache#publishEvent}}:
{code}
[...]
May 25 18:45:50         at 
org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache.publishEvent(TreeCache.java:902)
May 25 18:45:50         at 
org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache.publishEvent(TreeCache.java:894)
[...]
May 25 18:45:50         at 
org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:634)
May 25 18:45:50         at 
org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:553)
[...]
{code}
{{TreeCache#publishEvent}} will submit a new task to the cache's 
{{executorService}} (see 
[TreeCache:901|https://github.com/apache/curator/blob/844c0ad36340b695b2784489c078cfd78522143c/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java#L901])
 which is a {{directExecutorService}} in the case of Flink's ZooKeeper 
LeaderElectionDriver implementations (see 
[ZooKeeperUtils:764|https://github.com/apache/flink/blob/4576e4384ff36623712043564039f654c3b44a30/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java#L764]
 for the legacy {{ZooKeeperLeaderElectionDriver}} and 
[ZooKeeperMultipleComponentLeaderElectionDriver:76|https://github.com/apache/flink/blob/8ddfd590ebba7fc727e79db41b82d3d40a02b56a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriver.java#L76]).
 The close call happens in the test's main thread.

The {{TreeCache#close}} call sets the cache's state to {{CLOSED}} in 
[TreeCache:628|https://github.com/apache/curator/blob/844c0ad36340b695b2784489c078cfd78522143c/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java#L628].
 {{TreeCache#publishEvent}} checks this state in 
[TreeCache:898|https://github.com/apache/curator/blob/844c0ad36340b695b2784489c078cfd78522143c/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java#L898].
 The latter one doesn't use a lock. This can cause a race condition where the 
{{publishEvent}} method is called and passes the if condition before the test's 
main thread can trigger the close method but after the task is actually 
submitted causing the {{RejectedExecutionException}} which we're observing 
right now.

[~dmvk] may you verify my finding? I would suggest adding a less restrictive 
version of the {{DirectExecutorService}} that we could use in the production 
code to avoid running into this bug. We could continue to use the more 
restrictive version (which was introduced in FLINK-31995) in the test code. 
WDYT David?

> ZooKeeperLeaderElectionTest.testZooKeeperReelectionWithReplacement fails with 
> The ExecutorService is shut down already. No Callables can be executed on AZP
> -----------------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-32204
>                 URL: https://issues.apache.org/jira/browse/FLINK-32204
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.18.0
>            Reporter: Sergey Nuyanzin
>            Assignee: Matthias Pohl
>            Priority: Critical
>              Labels: test-stability
>
> [This 
> build|https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=49386&view=logs&j=0da23115-68bb-5dcd-192c-bd4c8adebde1&t=24c3384f-1bcb-57b3-224f-51bf973bbee8&l=7095]
>  fails as
> {noformat}
> May 25 18:45:50 Caused by: java.util.concurrent.RejectedExecutionException: 
> The ExecutorService is shut down already. No Callables can be executed.
> May 25 18:45:50       at 
> org.apache.flink.util.concurrent.DirectExecutorService.throwRejectedExecutionExceptionIfShutdown(DirectExecutorService.java:237)
> May 25 18:45:50       at 
> org.apache.flink.util.concurrent.DirectExecutorService.submit(DirectExecutorService.java:100)
> May 25 18:45:50       at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache.publishEvent(TreeCache.java:902)
> May 25 18:45:50       at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache.publishEvent(TreeCache.java:894)
> May 25 18:45:50       at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache.access$1200(TreeCache.java:79)
> May 25 18:45:50       at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache$TreeNode.processResult(TreeCache.java:489)
> May 25 18:45:50       at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.CuratorFrameworkImpl.sendToBackgroundCallback(CuratorFrameworkImpl.java:926)
> May 25 18:45:50       at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.CuratorFrameworkImpl.processBackgroundOperation(CuratorFrameworkImpl.java:683)
> May 25 18:45:50       at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.WatcherRemovalFacade.processBackgroundOperation(WatcherRemovalFacade.java:152)
> May 25 18:45:50       at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.GetDataBuilderImpl$3.processResult(GetDataBuilderImpl.java:272)
> May 25 18:45:50       at 
> org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:634)
> May 25 18:45:50       at 
> org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:553)
> May 25 18:45:50
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to