[
https://issues.apache.org/jira/browse/FLINK-13245?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
zhijiang updated FLINK-13245:
-----------------------------
Comment: was deleted
(was: 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 always release all the view readers in
`allReaders` in final. The details are in {{PartitionRequestQueue#close}}. If
so, it seems no problem even though the previous {{CancelPartitionRequest}}
does not work sometimes. But I think it would be more proper/strict if we
handle the {{CancelPartitionRequest}} via `allReaders` instead.
* Another root problem is that the
{{ReleaseOnConsumptionResultPartition#onConsumedSubpartition}} is not working
for the implementation of {{BoundedBlockingSubpartition}}.
{{onConsumedSubpartition}} would be triggered from consumer notification via
network, but it is only feasible for {{RemoteInputChannel}}. For the case of
{{LocalInputChannel}}, it would call
{{ResultSubpartitionView#releaseAllResources}} directly. So in the
{{SlotCountExceedingParallelismTest}} there are some local channels which cause
the reference counter in {{ReleaseOnConsumptionResultPartition}} would never
decrease to 0, then it would never release the {{BoundedBlockingSubpartition}}.
But in the process of {{BoundedBlockingSubpartition#releaseReaderReference}} it
would check the `isReleased` tag before deleting the file. So it has the
conflict here. In fact this issue already exists in previous
{{SpillableSubpartition}} , but in the test the spillable subpartition would
always use the memory way then it hides the potential problem.)
> 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)