[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-18 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


master: d5e151f72336abcc13082fe4bb3e05fd5a785e86

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Assignee: Yangze Guo
>Priority: Blocker
>  Labels: pull-request-available
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-13 Thread Zili Chen (Jira)


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

Zili Chen commented on FLINK-33053:
---

But it's possible to add an option to explicitly identify the ownership. You 
can open an issue on the Curator JIRA project and let me with the other 
maintainers to figure it out.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Assignee: Yangze Guo
>Priority: Blocker
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-13 Thread Zili Chen (Jira)


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

Zili Chen commented on FLINK-33053:
---

No. Both {{CuratorCache}} and {{TreeCache}} doesn't "own" the path so it's 
unclear if other recipes share the same client (connection) set up watches 
also. This is different from {{LeaderLatch}} which owns the path so it can 
ensure that no one else (should) access the related nodes.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Assignee: Yangze Guo
>Priority: Blocker
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-13 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


Thanks for the pointer [~tison] . I'd like to add a safetynet atm as 
ZOOKEEPER-4625 has not been fixed and the patch might not be cherry-pick to old 
version. BTW, could we fix it in curator?

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Blocker
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-13 Thread Zili Chen (Jira)


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

Zili Chen commented on FLINK-33053:
---

But we don't have other shared watchers so we can force remove watches as above.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Blocker
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-13 Thread Zili Chen (Jira)


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

Zili Chen commented on FLINK-33053:
---

See https://lists.apache.org/thread/3b9hn9j4c05yfztlr2zcctbg7sqwdh58.

This seems to be a ZK issue that I met one year ago..

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Blocker
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-13 Thread Zili Chen (Jira)


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

Zili Chen commented on FLINK-33053:
---

I noticed that the {{TreeCache}}'s close call {{removeWatches}} instead of 
{{removeAllWatches}} called by your scripts above.

{{removeWatches}} only remove the watcher in client side so remain the server 
side watcher as is.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Blocker
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-13 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


JFYI, I'm still investigating the root cause of this, but I found the issue 
will be fixed if we add a safetynet in 
ZooKeeperLeaderRetrievalDriver#close like this:
 
{code:java}
client.watchers()
   .removeAll()
   .ofType(Watcher.WatcherType.Any)
   .forPath(connectionInformationPath);{code}

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Blocker
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-11 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


I just find that this issue is very easy to reproduce. To observe the watch 
leak, it only requires deploying a local standalone HA cluster and submitting a 
single wordcount job. Thus, I'd like to promote it to blocker of 1.18. WDYT 
[~renqs] ?

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.18.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-09 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


[~tison] Yes, it seems teh curator framework issued the watcher removal request 
at least. But will there be any logs printed when this request is successful?

 

[~mapohl] Thead dump of a TM with watch leak 
[^taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json]

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
> Attachments: 26.dump.zip, 26.log, 
> taskmanager_flink-native-test-117-taskmanager-1-9_thread_dump (1).json
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Zili Chen (Jira)


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

Zili Chen commented on FLINK-33053:
---

The log seems trimed. I saw:

2023-09-08 11:09:03,738 DEBUG 
org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.WatcherRemovalManager
 [] - Removing watcher for path: 
/flink/flink-native-test-117/leader/7db5c7316828f598234677e2169e7b0f/connection_info

So the TM has issued watcher removal request.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
> Attachments: 26.dump.zip, 26.log
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Matthias Pohl (Jira)


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

Matthias Pohl commented on FLINK-33053:
---

Thanks for sharing this. I guess, the thread dump (which you can generate by 
sending {{kill -3 }} to the TaskManager process) would be more helpful 
than the heap dump. Or am I missing something. The thread dump might tell us 
more of what threads are still available even after the JobMaster is closed.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
> Attachments: 26.dump.zip, 26.log
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


I reproduce the issue and filter out some logs. [^26.dump.zip] [^26.log] This 
tm still watches the leader of job 7db5c7316828f598234677e2169e7b0f. 

{{/flink/flink-native-test-117/leader/7db5c7316828f598234677e2169e7b0f/connection_info}}
{{        0x6400013b748a4420}}

 

[~mapohl] [~tison] 

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
> Attachments: 26.dump.zip, 26.log
>
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


Thanks for the pointer [~mapohl]. I'll try to get a debug log and heap dump.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Matthias Pohl (Jira)


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

Matthias Pohl commented on FLINK-33053:
---

FLINK-29813 is already covering the migration to {{CuratorCache}}. I haven't 
had a chance to look into it, yet. We need to do an analysis whether switching 
would cause some side effects.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


[~tison] Could we use CuratorCache instead? Would it be more stable?

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Zili Chen (Jira)


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

Zili Chen commented on FLINK-33053:
---

Perhaps you can enable debug logs and check "Removing watcher for path: " from 
Curator to see if the related watchers are issued removing.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Zili Chen (Jira)


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

Zili Chen commented on FLINK-33053:
---

The recipe in use is {{TreeCache}}, which doesn't change from 5.0.0. And it 
also closes watches on {{close}}.

Do you have a bisect which version introduced this regression?

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Matthias Pohl (Jira)


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

Matthias Pohl commented on FLINK-33053:
---

Can you share the logs and the thread dump of this run?

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


In our test, the log shows the ZooKeeperLeaderRetrievalDriver has been close 
correctly. So, it's likely to be a curator issue.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-07 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


[~mapohl] Thanks for your help. I check it with curator 5.5.0 but observe the 
same issue. cc [~Tison]

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-06 Thread Matthias Pohl (Jira)


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

Matthias Pohl commented on FLINK-33053:
---

Thanks for bringing this up, [~guoyangze]. I'm going to have a look at it.

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-06 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


Also cc [~wangyang0918]

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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


[jira] [Commented] (FLINK-33053) Watcher leak in Zookeeper HA mode

2023-09-06 Thread Yangze Guo (Jira)


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

Yangze Guo commented on FLINK-33053:


[~mapohl] Would you like to take a look?

> Watcher leak in Zookeeper HA mode
> -
>
> Key: FLINK-33053
> URL: https://issues.apache.org/jira/browse/FLINK-33053
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / Coordination
>Affects Versions: 1.17.0, 1.17.1
>Reporter: Yangze Guo
>Priority: Critical
>
> We observe a watcher leak in our OLAP stress test when enabling Zookeeper HA 
> mode. TM's watches on the leader of JobMaster has not been stopped after job 
> finished.
> Here is how we re-produce this issue:
>  - Start a session cluster and enable Zookeeper HA mode.
>  - Continuously and concurrently submit short queries, e.g. WordCount to the 
> cluster.
>  - echo -n wchp | nc \{zk host} \{zk port} to get current watches.
> We can see a lot of watches on 
> /flink/\{cluster_name}/leader/\{job_id}/connection_info.



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