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

zhijiang commented on FLINK-13245:
----------------------------------

Thanks for finding this potential issue and the investigation! [~Zentol] 
[~azagrebin]

After reviewing the relevant codes, it actually has two issues here:
 * Considering handling the `CancelPartitionRequest` issue, I guess we might 
have two assumptions before. One assumption is that `availableReaders` is 
always equivalent to `allReaders`, but now this assumption is not right because 
of credit. The other assumption is that we make the logic of 
`CancelPartitionRequest` as best-effort way, because the last 
`RemoteInputChannel` would send `CloseRequest` message, then the 
`PartitionRequestQueue` would also release all the view readers in 
`allReaders`. The details are in `PartitionRequestQueue#close`. If so, it seems 
no problem even though the previous `CancelPartitionRequest` does not work. But 
I think it would be more proper/strict if we handle the ` 
CancelPartitionRequest` via `allReaders` instead.

 * Another critical problem is that the notification from consumer side of 
releasing subpartition is not actually making sense for the implementation of 
`BoundedBlockingSubpartition`, because it would check `isReleased` tag during 
`releaseReaderReference`, and this tag could only be changed from producer 
call. In other words, for blocking case when to release partition is determined 
by JobMaster based on PartitionReleaseStrategy. So even though we solve the 
above first issue, the file for blocking partition still could not be deleted 
after release notification from network.

In addition, this issue is actually hidden in previous `SpillableSubpartition` 
because in this test the partition is only using the memory type not spilled 
file.

> Network stack is leaking files
> ------------------------------
>
>                 Key: FLINK-13245
>                 URL: https://issues.apache.org/jira/browse/FLINK-13245
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Network
>    Affects Versions: 1.9.0
>            Reporter: Chesnay Schepler
>            Assignee: zhijiang
>            Priority: Blocker
>             Fix For: 1.9.0
>
>
> There's file leak in the network stack / shuffle service.
> When running the {{SlotCountExceedingParallelismTest}} on Windows a large 
> number of {{.channel}} files continue to reside in a 
> {{flink-netty-shuffle-XXX}} directory.
> From what I've gathered so far these files are still being used by a 
> {{BoundedBlockingSubpartition}}. The cleanup logic in this class uses 
> ref-counting to ensure we don't release data while a reader is still present. 
> However, at the end of the job this count has not reached 0, and thus nothing 
> is being released.
> The same issue is also present on the {{ResultPartition}} level; the 
> {{ReleaseOnConsumptionResultPartition}} also are being released while the 
> ref-count is greater than 0.
> Overall it appears like there's some issue with the notifications for 
> partitions being consumed.
> It is feasible that this issue has recently caused issues on Travis where the 
> build were failing due to a lack of disk space.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to