[
https://issues.apache.org/jira/browse/FLINK-1959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14540146#comment-14540146
]
Alexander Alexandrov commented on FLINK-1959:
---------------------------------------------
After some more debugging with [~elbehery] we managed to identify the issue.
Adding {{partitionByHash}} into the pipeline breaks the local chain two and
creates two tasks. The second task (the receiver of the partition) starts with
a RegularPactTask which runs a NoOpDriver and does not have a UDF stub. Because
of that, the test at [line
511|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java#L511]
and the accumulators are never called.
@[~StephanEwen]: Unless you see a problem with that solution, I suggest to
remove the {{stub != null}} check at [line
511|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java#L511]
and report accumulators always.
Regards,
Alexander
> Accumulators BROKEN after Partitioning
> --------------------------------------
>
> Key: FLINK-1959
> URL: https://issues.apache.org/jira/browse/FLINK-1959
> Project: Flink
> Issue Type: Bug
> Components: Examples
> Affects Versions: master
> Reporter: mustafa elbehery
> Priority: Critical
> Fix For: master
>
>
> while running the Accumulator example in
> https://github.com/Elbehery/flink/blob/master/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java,
>
> I tried to alter the data flow with "PartitionByHash" function before
> applying "Filter", and the resulted accumulator was NULL.
> By Debugging, I could see the accumulator in the RunTime Map. However, by
> retrieving the accumulator from the JobExecutionResult object, it was NULL.
> The line caused the problem is "file.partitionByHash(1).filter(new
> EmptyFieldFilter())" instead of "file.filter(new EmptyFieldFilter())"
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)