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

ASF GitHub Bot commented on FLINK-2662:
---------------------------------------

GitHub user fhueske opened a pull request:

    https://github.com/apache/flink/pull/2848

    [FLINK-2662] [optimizer] Fix computation of global properties of union 
operator.

    Fix computation of global properties of union operator.
    This solves the problem of invalid shipping strategy between consecutive 
unions.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/fhueske/flink unionBug

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2848.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2848
    
----
commit a226bcb13f6398988d9f4359ae371cbb1bf71465
Author: Fabian Hueske <fhue...@apache.org>
Date:   2016-11-21T18:06:42Z

    [FLINK-2662] [optimizer] Fix computation of global properties of union 
operator.
    
    - Fixes invalid shipping strategy between consecutive unions.

----


> CompilerException: "Bug: Plan generation for Unions picked a ship strategy 
> between binary plan operators."
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-2662
>                 URL: https://issues.apache.org/jira/browse/FLINK-2662
>             Project: Flink
>          Issue Type: Bug
>          Components: Optimizer
>    Affects Versions: 0.9.1, 0.10.0
>            Reporter: Gabor Gevay
>            Assignee: Fabian Hueske
>            Priority: Critical
>             Fix For: 1.0.0, 1.2.0, 1.1.3
>
>         Attachments: Bug.java, FlinkBug.scala
>
>
> I have a Flink program which throws the exception in the jira title. Full 
> text:
> Exception in thread "main" org.apache.flink.optimizer.CompilerException: Bug: 
> Plan generation for Unions picked a ship strategy between binary plan 
> operators.
>       at 
> org.apache.flink.optimizer.traversals.BinaryUnionReplacer.collect(BinaryUnionReplacer.java:113)
>       at 
> org.apache.flink.optimizer.traversals.BinaryUnionReplacer.postVisit(BinaryUnionReplacer.java:72)
>       at 
> org.apache.flink.optimizer.traversals.BinaryUnionReplacer.postVisit(BinaryUnionReplacer.java:41)
>       at 
> org.apache.flink.optimizer.plan.DualInputPlanNode.accept(DualInputPlanNode.java:170)
>       at 
> org.apache.flink.optimizer.plan.SingleInputPlanNode.accept(SingleInputPlanNode.java:199)
>       at 
> org.apache.flink.optimizer.plan.DualInputPlanNode.accept(DualInputPlanNode.java:163)
>       at 
> org.apache.flink.optimizer.plan.DualInputPlanNode.accept(DualInputPlanNode.java:163)
>       at 
> org.apache.flink.optimizer.plan.WorksetIterationPlanNode.acceptForStepFunction(WorksetIterationPlanNode.java:194)
>       at 
> org.apache.flink.optimizer.traversals.BinaryUnionReplacer.preVisit(BinaryUnionReplacer.java:49)
>       at 
> org.apache.flink.optimizer.traversals.BinaryUnionReplacer.preVisit(BinaryUnionReplacer.java:41)
>       at 
> org.apache.flink.optimizer.plan.DualInputPlanNode.accept(DualInputPlanNode.java:162)
>       at 
> org.apache.flink.optimizer.plan.SingleInputPlanNode.accept(SingleInputPlanNode.java:199)
>       at 
> org.apache.flink.optimizer.plan.SingleInputPlanNode.accept(SingleInputPlanNode.java:199)
>       at 
> org.apache.flink.optimizer.plan.OptimizedPlan.accept(OptimizedPlan.java:127)
>       at org.apache.flink.optimizer.Optimizer.compile(Optimizer.java:520)
>       at org.apache.flink.optimizer.Optimizer.compile(Optimizer.java:402)
>       at 
> org.apache.flink.client.LocalExecutor.getOptimizerPlanAsJSON(LocalExecutor.java:202)
>       at 
> org.apache.flink.api.java.LocalEnvironment.getExecutionPlan(LocalEnvironment.java:63)
>       at malom.Solver.main(Solver.java:66)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:497)
>       at com.intellij.rt.execution.application.AppMain.main(AppMain.java:140)
> The execution plan:
> http://compalg.inf.elte.hu/~ggevay/flink/plan_3_4_0_0_without_verif.txt
> (I obtained this by commenting out the line that throws the exception)
> The code is here:
> https://github.com/ggevay/flink/tree/plan-generation-bug
> The class to run is "Solver". It needs a command line argument, which is a 
> directory where it would write output. (On first run, it generates some 
> lookuptables for a few minutes, which are then placed to /tmp/movegen)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to