Hey Ufuk, thank you for this. I have not yet taken a look at this hash table, will look into it tommorrow at the office.
Sebastian 2014-07-27 21:02 GMT+02:00 Ufuk Celebi <[email protected]>: > Thanks! I didn't take a look at the plan or code yet, but the call of > joinFunction.join(probeSideRecord, null, collector) in > JoinWithSolutionSetSecondDriver.java:143 is the root of the problem. It is > taking the branch, because the solution set join is *not* finding a match > in the hash table for some record. > > Did you specifically look into the hash table related parts while > debugging? This is something I would do, since actually the initial > solution set should ensure that there is always a matching record in the > hash table. > > If this is not of much help, we can also do a debugging session via screen > sharing tomorrow. > > Ufuk > > On 27 Jul 2014, at 20:17, Sebastian Kunert <[email protected]> wrote: > > > Hi, > > > > attached is the whole stacktrace. I am working in this branch > > https://github.com/skunert/incubator-flink/tree/constantFields_renamed > .The > > question is whether the plan is incorrectly build due to my changes or if > > there is maybe a optimizer bug which only comes to effect because of my > > changes. Thatswhy I am investigating if the plan itself is correct. > > > > 19:56:38,774 ERROR org.apache.flink.runtime.operators.RegularPactTask > > - Error in task code: > > > Join(org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter) > > (1/4) > > java.lang.NullPointerException > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:176) > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:171) > > at > > > org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver.run(JoinWithSolutionSetSecondDriver.java:143) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:509) > > at > > > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:137) > > at > > > org.apache.flink.runtime.iterative.task.IterationTailPactTask.run(IterationTailPactTask.java:106) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:374) > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:265) > > at java.lang.Thread.run(Thread.java:745) > > 19:56:38,802 ERROR org.apache.flink.runtime.operators.RegularPactTask > > - Error in task code: > > > Join(org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter) > > (2/4) > > java.lang.NullPointerException > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:176) > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:171) > > at org.apache.flink.runtime.operators.JoinWithNullpoint exception when > > termination requested was false > > Nullpoint exception when termination requested was false > > SolutionSetSecondDriver.run(JoinWithSolutionSetSecondDriver.java:143) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:509) > > at > > > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:137) > > at > > > org.apache.flink.runtime.iterative.task.IterationTailPactTask.run(IterationTailPactTask.java:106) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:374) > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:265) > > at java.lang.Thread.run(Thread.java:745) > > java.lang.NullPointerException > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:176) > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:171) > > at > > > org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver.run(JoinWithSolutionSetSecondDriver.java:143) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:509) > > at > > > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:137) > > at > > > org.apache.flink.runtime.iterative.task.IterationTailPactTask.run(IterationTailPactTask.java:106) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:374) > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:265) > > at java.lang.Thread.run(Thread.java:745) > > java.lang.NullPointerException > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:176) > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:171) > > at > > > org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver.run(JoinWithSolutionSetSecondDriver.java:143) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:509) > > at > > > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:137) > > at > > > org.apache.flink.runtime.iterative.task.IterationTailPactTask.run(IterationTailPactTask.java:106) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:374) > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:265) > > at java.lang.Thread.run(Thread.java:745) > > 19:56:38,831 ERROR org.apache.flink.runtime.operators.RegularPactTask > > - Error in task code: > > > Join(org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter) > > (4/4) > > java.lang.NullPointerException > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:176) > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:171) > > at > > > org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver.run(JoinWithSolutionSetSecondDriver.java:143) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:509) > > at > > > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:137) > > at > > > org.apache.flink.runtime.iterative.task.IterationTailPactTask.run(IterationTailPactTask.java:106) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:374) > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:265) > > at java.lang.Thread.run(Thread.java:745) > > java.lang.NullPointerException > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:176) > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:171) > > at > > > org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver.run(JoinWithSolutionSetSecondDriver.java:143) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:509) > > at > > > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:137) > > at > > > org.apache.flink.runtime.iterative.task.IterationTailPactTask.run(IterationTailPactTask.java:106) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:374) > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:265) > > at java.lang.Thread.run(Thread.java:745) > > 19:56:38,831 ERROR org.apache.flink.runtime.operators.RegularPactTask > > - Error in task code: > > > Join(org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter) > > (3/4) > > java.lang.NullPointerException > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:176) > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:171) > > at > > > org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver.run(JoinWithSolutionSetSecondDriver.java:143) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:509) > > at > > > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:137) > > at > > > org.apache.flink.runtime.iterative.task.IterationTailPactTask.run(IterationTailPactTask.java:106) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:374) > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:265) > > at java.lang.Thread.run(Thread.java:745) > > java.lang.NullPointerException > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:176) > > at > > > org.apache.flink.example.java.graph.ConnectedComponents$ComponentIdFilter.join(ConnectedComponents.java:171) > > at > > > org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver.run(JoinWithSolutionSetSecondDriver.java:143) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:509) > > at > > > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:137) > > at > > > org.apache.flink.runtime.iterative.task.IterationTailPactTask.run(IterationTailPactTask.java:106) > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:374) > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:265) > > at java.lang.Thread.run(Thread.java:745) > > Nullpoint exception when termination requested was false > > Nullpoint exception when termination requested was false > > > > java.lang.AssertionError: Vertex is in wrong component. > > Expected :2 > > Actual :514 > > <Click to see difference> > > at org.junit.Assert.fail(Assert.java:91) > > at org.junit.Assert.failNotEquals(Assert.java:645) > > at org.junit.Assert.assertEquals(Assert.java:126) > > at org.junit.Assert.assertEquals(Assert.java:470) > > at > > > org.apache.flink.test.testdata.ConnectedComponentsData.checkOddEvenResult(ConnectedComponentsData.java:104) > > at > > > org.apache.flink.test.exampleJavaPrograms.ConnectedComponentsITCase.postSubmit(ConnectedComponentsITCase.java:57) > > at > > > org.apache.flink.test.util.JavaProgramTestBase.testJob(JavaProgramTestBase.java:113) > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > > at > > > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > > at > > > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > > at > > > org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:44) > > at > > > org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:15) > > at > > > org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:41) > > at > > > org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:20) > > at > > > org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:28) > > at > > > org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:31) > > at > > > org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:76) > > at > > > org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50) > > at org.junit.runners.ParentRunner$3.run(ParentRunner.java:193) > > at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:52) > > at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:191) > > at org.junit.runners.ParentRunner.access$000(ParentRunner.java:42) > > at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:184) > > at org.junit.runners.ParentRunner.run(ParentRunner.java:236) > > at org.junit.runner.JUnitCore.run(JUnitCore.java:157) > > at > > > com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:74) > > at > > > com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:211) > > at > com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:67) > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > > at > > > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > > at com.intellij.rt.execution.application.AppMain.main(AppMain.java:134) > > > > > > 2014-07-27 20:06 GMT+02:00 Ufuk Celebi <[email protected]>: > > > >> Hey Sebastian, > >> > >> Could you also post the exception? > >> > >> Thanks! > >> > >> Ufuk > >> > >>> On 27 Jul 2014, at 18:23, Sebastian Kunert <[email protected]> > wrote: > >>> > >>> Hey guys, > >>> > >>> I am currently working on optimizer integration of forwarded fields. I > >> get > >>> NullPointerExceptions during the execution of our > >> ConnectedComponentITCase > >>> in the NeighborWithComponentIDJoin. A second pair of eyes on this plan > >>> would help me, maybe there are some obvious problems with it that I may > >>> have overlooked: > >>> > >>> http://pastebin.com/beBjpSzZ > >>> > >>> Thank you :) > >>> > >>> Sebastian > >> > >
