You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "fengjiajie (via GitHub)" <gi...@apache.org> on 2024/01/29 02:48:33 UTC
[PR] [FLINK-33981][runtime] Fix not closing DirectoryStream after listing local state files [flink]
fengjiajie opened a new pull request, #24212:
URL: https://github.com/apache/flink/pull/24212
<!--
*Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
*Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
## Contribution Checklist
- Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
- Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
- Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
- Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
- Each pull request should address only one issue, not mix up code from multiple issues.
- Each commit in the pull request has a meaningful commit message (including the JIRA id)
- Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
**(The sections below can be removed for hotfixes of typos)**
-->
## What is the purpose of the change
When using MiniCluster mode, file descriptors like /tmp/minicluster_6e62ab17ab058e0d5df56b97e9ccef71/tm_0/localState are not released after a Job completes. Executing multiple Jobs in the same JVM might result in leftover file descriptors, potentially leading to problems.
After executing the reproducing code provided below (after entering the sleep), running lsof -p 18162 reveals:
```
...
java 18162 sa_cluster 30r DIR 253,1 0 1311962 /tmp/minicluster_79186ba3856bb18fd21c20d9adbaa3da/tm_0/localState (deleted)
java 18162 sa_cluster 31r DIR 253,1 0 1311962 /tmp/minicluster_79186ba3856bb18fd21c20d9adbaa3da/tm_0/localState (deleted)
java 18162 sa_cluster 32r DIR 253,1 0 1310787 /tmp/minicluster_f3ac056ffa1c47010698fc02c5e6d4cf/tm_0/localState (deleted)
java 18162 sa_cluster 33r DIR 253,1 0 1310787 /tmp/minicluster_f3ac056ffa1c47010698fc02c5e6d4cf/tm_0/localState (deleted)
java 18162 sa_cluster 34r DIR 253,1 0 1311960 /tmp/minicluster_ef6177c97a2c9096758d0a8f132f9f02/tm_0/localState (deleted)
java 18162 sa_cluster 35r DIR 253,1 0 1311960 /tmp/minicluster_ef6177c97a2c9096758d0a8f132f9f02/tm_0/localState (deleted)
java 18162 sa_cluster 36r DIR 253,1 0 1311974 /tmp/minicluster_661a2967ea5c377265fa9f41c768db21/tm_0/localState (deleted)
java 18162 sa_cluster 37r DIR 253,1 0 1311974 /tmp/minicluster_661a2967ea5c377265fa9f41c768db21/tm_0/localState (deleted)
java 18162 sa_cluster 38r DIR 253,1 0 1311979 /tmp/minicluster_8413f476b77245203ed1ef759eb0d2de/tm_0/localState (deleted)
...
```
The code used for reproduction is as follows:
```
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.graph.StreamGraph;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
/**
* javac -cp 'lib/*' TestReleaseFd.java
* java -Xmx600m -cp '.:lib/*' TestReleaseFd
*/
public class TestReleaseFd {
public static void main(String[] args) throws Exception {
for (int i = 0; i < 10; ++i) {
int round = i;
Thread thread = new Thread(() -> {
try {
Configuration configuration = new Configuration();
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(configuration);
env.setParallelism(1);
DataStreamSource<Long> longDataStreamSource = env.fromSequence(1, 100000);
longDataStreamSource.addSink(new DiscardingSink<>());
StreamGraph streamGraph = env.getStreamGraph();
streamGraph.setJobName("test-" + System.nanoTime());
JobClient jobClient = env.executeAsync(streamGraph);
CompletableFuture<JobExecutionResult> jobExecutionResultCompletableFuture = jobClient.getJobExecutionResult();
JobExecutionResult jobExecutionResult = null;
while (jobExecutionResult == null) {
try {
jobExecutionResult = jobExecutionResultCompletableFuture.get(20, TimeUnit.SECONDS);
} catch (TimeoutException timeoutException) {
// ignore
}
}
System.out.println("finished round: " + round);
env.close();
} catch (Exception e) {
throw new RuntimeException(e);
}
});
thread.setDaemon(true);
thread.start();
thread.join();
System.out.println("done ... " + i);
}
// ======================= lsof -p 18162
Thread.sleep(500_000_000);
}
}
```
## Brief change log
Close the DirectoryStream after using.
## Verifying this change
This change is a trivial rework / code cleanup without any test coverage.
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): (yes / **no**)
- The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
- The serializers: (yes / **no** / don't know)
- The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
- Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / **no** / don't know)
- The S3 file system connector: (yes / **no** / don't know)
## Documentation
- Does this pull request introduce a new feature? (yes / **no**)
- If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
Re: [PR] [FLINK-33981][runtime] Fix not closing DirectoryStream after listing local state files [flink]
Posted by "fengjiajie (via GitHub)" <gi...@apache.org>.
fengjiajie commented on PR #24212:
URL: https://github.com/apache/flink/pull/24212#issuecomment-1925600132
Hi @1996fanrui , please take a look at this fix when you have a moment to see if it can be merged, or if I need to reach out to someone else? thanks.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
Re: [PR] [FLINK-33981][runtime] Fix not closing DirectoryStream after listing local state files [flink]
Posted by "1996fanrui (via GitHub)" <gi...@apache.org>.
1996fanrui merged PR #24212:
URL: https://github.com/apache/flink/pull/24212
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
Re: [PR] [FLINK-33981][runtime] Fix not closing DirectoryStream after listing local state files [flink]
Posted by "fengjiajie (via GitHub)" <gi...@apache.org>.
fengjiajie commented on PR #24212:
URL: https://github.com/apache/flink/pull/24212#issuecomment-1926335570
> Thanks @fengjiajie for the fix!
>
> I see other callers of `Files.list` has already used the `try-with-resource`, and the comment also mentions it. So LGTM.
@1996fanrui Thanks for reviewing and merging the PRs. I've submitted PRs to the other two branches.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
Re: [PR] [FLINK-33981][runtime] Fix not closing DirectoryStream after listing local state files [flink]
Posted by "1996fanrui (via GitHub)" <gi...@apache.org>.
1996fanrui commented on PR #24212:
URL: https://github.com/apache/flink/pull/24212#issuecomment-1926354179
Thank you @fengjiajie for the great work. I have merged all PRs.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
Re: [PR] [FLINK-33981][runtime] Fix not closing DirectoryStream after listing local state files [flink]
Posted by "flinkbot (via GitHub)" <gi...@apache.org>.
flinkbot commented on PR #24212:
URL: https://github.com/apache/flink/pull/24212#issuecomment-1913877869
<!--
Meta data
{
"version" : 1,
"metaDataEntries" : [ {
"hash" : "3d66ebaffc112e0b3777fc3e3841b64f045357bb",
"status" : "UNKNOWN",
"url" : "TBD",
"triggerID" : "3d66ebaffc112e0b3777fc3e3841b64f045357bb",
"triggerType" : "PUSH"
} ]
}-->
## CI report:
* 3d66ebaffc112e0b3777fc3e3841b64f045357bb UNKNOWN
<details>
<summary>Bot commands</summary>
The @flinkbot bot supports the following commands:
- `@flinkbot run azure` re-run the last Azure build
</details>
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org