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

Stephan Ewen commented on FLINK-2659:
-------------------------------------

Yes, this clearly looks like a bug.

Thank you for so thoroughly checking out the object-reuse mode...

> Object reuse in UnionWithTempOperator
> -------------------------------------
>
>                 Key: FLINK-2659
>                 URL: https://issues.apache.org/jira/browse/FLINK-2659
>             Project: Flink
>          Issue Type: Bug
>          Components: Distributed Runtime
>    Affects Versions: master
>            Reporter: Greg Hogan
>
> The first loop in UnionWithTempOperator.run() executes until null, then the 
> second loop attempts to reuse this null value. [~StephanEwen], would you like 
> me to submit a pull request?
> Stack trace:
> {noformat}
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Job execution failed.
>       at org.apache.flink.client.program.Client.run(Client.java:381)
>       at org.apache.flink.client.program.Client.run(Client.java:319)
>       at org.apache.flink.client.program.Client.run(Client.java:312)
>       at 
> org.apache.flink.client.program.ContextEnvironment.execute(ContextEnvironment.java:63)
>       at 
> org.apache.flink.api.java.ExecutionEnvironment.execute(ExecutionEnvironment.java:790)
>       at Driver.main(Driver.java:376)
>       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 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:437)
>       at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:353)
>       at org.apache.flink.client.program.Client.run(Client.java:278)
>       at 
> org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:630)
>       at org.apache.flink.client.CliFrontend.run(CliFrontend.java:318)
>       at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:953)
>       at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1003)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job 
> execution failed.
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1.applyOrElse(JobManager.scala:418)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
>       at 
> org.apache.flink.runtime.LeaderSessionMessageFilter$$anonfun$receive$1.applyOrElse(LeaderSessionMessageFilter.scala:36)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
>       at 
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:40)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:28)
>       at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:118)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.applyOrElse(LogMessages.scala:28)
>       at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager.aroundReceive(JobManager.scala:104)
>       at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
>       at akka.actor.ActorCell.invoke(ActorCell.scala:487)
>       at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:254)
>       at akka.dispatch.Mailbox.run(Mailbox.scala:221)
>       at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
>       at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>       at 
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>       at 
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>       at 
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> Caused by: java.lang.NullPointerException
>       at 
> org.apache.flink.api.java.typeutils.runtime.TupleSerializer.deserialize(TupleSerializer.java:136)
>       at 
> org.apache.flink.api.java.typeutils.runtime.TupleSerializer.deserialize(TupleSerializer.java:30)
>       at 
> org.apache.flink.runtime.plugable.ReusingDeserializationDelegate.read(ReusingDeserializationDelegate.java:57)
>       at 
> org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer.getNextRecord(SpillingAdaptiveSpanningRecordDeserializer.java:125)
>       at 
> org.apache.flink.runtime.io.network.api.reader.AbstractRecordReader.getNextRecord(AbstractRecordReader.java:65)
>       at 
> org.apache.flink.runtime.io.network.api.reader.MutableRecordReader.next(MutableRecordReader.java:34)
>       at 
> org.apache.flink.runtime.operators.util.ReaderIterator.next(ReaderIterator.java:59)
>       at 
> org.apache.flink.runtime.operators.resettable.SpillingResettableMutableObjectIterator.next(SpillingResettableMutableObjectIterator.java:149)
>       at 
> org.apache.flink.runtime.operators.UnionWithTempOperator.run(UnionWithTempOperator.java:71)
>       at 
> org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:489)
>       at 
> org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:144)
>       at 
> org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask.run(IterationIntermediatePactTask.java:92)
>       at 
> org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:354)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:581)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}



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

Reply via email to