[
https://issues.apache.org/jira/browse/FLINK-13245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16885361#comment-16885361
]
Andrey Zagrebin commented on FLINK-13245:
-----------------------------------------
Here is some more clarification about our debugging with [~Zentol]. We
basically found two problems:
* In case of RemoteInputChannel, producer's
PartitionRequestQueue#userEventTriggered always gets a CancelPartitionRequest
from each consumer's RemoteInputChannel except one before CloseRequest comes
and triggers PartitionRequestQueue#close. The problem is that
PartitionRequestQueue#userEventTriggered releases the toCancel reader only if
it is in availableReaders but always removes it from allReaders. This means
that when PartitionRequestQueue#close is called only not canceled readers are
released, others are leaked in PartitionRequestQueue#userEventTriggered.
[~Zentol] has suggested a fix for it but the files are still not deleted when
the test job shuts down. They are deleted only when the TE and network shut
down at the end of the test. To fix this, the next point has to be resolved.
* After our further investigation another problem was found. The final release
of ReleaseOnConsumptionResultPartition happens only if pendingReferences is
zero. This happens only if notifySubpartitionConsumed is called for all readers
in PartitionRequestQueue. notifySubpartitionConsumed is currently called only
in PartitionRequestQueue#close but not in case of CancelPartitionRequest,
handleException and channelInactive. It means that the partition will linger in
those cases and will be released only when the TE and network shut down where
pendingReferences is not checked. It leads to the question of why we have 2
separate reader methods: notifySubpartitionConsumed and releaseAllResources if
they both are basically parts of partition release. Why do we need
notifySubpartitionConsumed, what does it mean and how is the fact of 'being
Consumed' used except releasing? It seems both methods need to be called in all
cases: CancelPartitionRequest, handleException, channelInactive and
CloseRequest when the partition is not needed any more in PartitionRequestQueue.
> 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)