[ https://issues.apache.org/jira/browse/FLINK-29927?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17630326#comment-17630326 ]
Chesnay Schepler commented on FLINK-29927: ------------------------------------------ Unless I'm several underestimating Scala magic this leak should've been there for a long time. Should be easy to fix though; I've opened a PR. > AkkaUtils#getAddress may cause memory leak > ------------------------------------------ > > Key: FLINK-29927 > URL: https://issues.apache.org/jira/browse/FLINK-29927 > Project: Flink > Issue Type: Bug > Components: Runtime / RPC > Affects Versions: 1.16.0, 1.15.2 > Reporter: Gen Luo > Assignee: Chesnay Schepler > Priority: Major > Labels: pull-request-available > Fix For: 1.17.0, 1.15.3, 1.16.1 > > Attachments: RemoteAddressExtensionLeaking.png > > > We found a slow memory leak in JM. When MetricFetcherImpl tries to retrieve > metrics, it always call MetricQueryServiceRetriever#retrieveService first. > And the method will acquire the address of a task manager, which will use > AkkaUtil#getAddress internally. While the getAddress method is implemented > like this: > {code:java} > public static Address getAddress(ActorSystem system) { > return new RemoteAddressExtension().apply(system).getAddress(); > } > {code} > and the RemoteAddressExtension#apply is like this: > {code:scala} > def apply(system: ActorSystem): T = { > java.util.Objects.requireNonNull(system, "system must not be > null!").registerExtension(this) > } > {code} > This means every call of AkkaUtils#getAddress will register a new extension > to the ActorSystem, and can never be released until the ActorSystem exits. > Most of the usage of the method are called only once while initializing, but > as described above, MetricFetcherImpl will also use the method. It can > happens periodically while users open the WebUI, or happens when the users > call the RESTful API directly to get metrics. This means the memory may keep > leaking. > The leak may be introduced in FLINK-23662 when porting the scala version of > AkkaUtils to the java one, while I'm not sure if the scala version has the > same issue. > The leak seems very slow. We observed it on a job running for more than one > month with only 1G memory for job manager. So I suppose it's not an emergency > one but still needs to fix. -- This message was sent by Atlassian Jira (v8.20.10#820010)