I suspect that this is one of the cases where an exception in an actor causes the actor to die (here the job manager)
On Thu, Feb 5, 2015 at 10:40 AM, Till Rohrmann <trohrm...@apache.org> wrote: > It looks to me that the TaskManager does not receive a > ConsumerNotificationResult after having send the ScheduleOrUpdateConsumers > message. This can either mean that something went wrong in > ExecutionGraph.scheduleOrUpdateConsumers method or the connection was > disassociated for some reasons. The logs would indeed be very helpful to > understand what happened. > > On Thu, Feb 5, 2015 at 10:36 AM, Ufuk Celebi <u...@apache.org> wrote: > > > Hey Chesnay, > > > > I will look into it. Can you share the complete LOGs? > > > > – Ufuk > > > > On 04 Feb 2015, at 14:49, Chesnay Schepler < > chesnay.schep...@fu-berlin.de> > > wrote: > > > > > Hello, > > > > > > I'm trying to run python jobs with the latest master on a cluster and > > get the following exception: > > > > > > Error: The program execution failed: JobManager not reachable anymore. > > Terminate waiting for job answer. > > > org.apache.flink.client.program.ProgramInvocationException: The program > > execution failed: JobManager not reachable anymore. Terminate waiting for > > job answer. > > > at org.apache.flink.client.program.Client.run(Client.java:345) > > > at org.apache.flink.client.program.Client.run(Client.java:304) > > > at org.apache.flink.client.program.Client.run(Client.java:298) > > > at > > > org.apache.flink.client.program.ContextEnvironment.execute(ContextEnvironment.java:55) > > > at > > > org.apache.flink.api.java.ExecutionEnvironment.execute(ExecutionEnvironment.java:677) > > > at > > > org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.runPlan(PythonPlanBinder.java:106) > > > at > > > org.apache.flink.languagebinding.api.java.python.PythonPlanBinder.main(PythonPlanBinder.java:79) > > > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > > > at > > > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > > > at > > > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > > > at java.lang.reflect.Method.invoke(Method.java:606) > > > 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:250) > > > at > > org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:387) > > > at org.apache.flink.client.CliFrontend.run(CliFrontend.java:356) > > > at > > > org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1066) > > > at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1090) > > > > > > In the jobmanager log file i find this exception: > > > > > > java.lang.IllegalStateException: Buffer has already been recycled. > > > at > > > org.apache.flink.shaded.com.google.common.base.Preconditions.checkState(Preconditions.java:176) > > > at > > > org.apache.flink.runtime.io.network.buffer.Buffer.ensureNotRecycled(Buffer.java:131) > > > at > > org.apache.flink.runtime.io.network.buffer.Buffer.setSize(Buffer.java:95) > > > at > > > org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer.getCurrentBuffer(SpanningRecordSerializer.java:151) > > > at > > > org.apache.flink.runtime.io.network.api.writer.RecordWriter.clearBuffers(RecordWriter.java:158) > > > at > > > org.apache.flink.runtime.operators.RegularPactTask.clearWriters(RegularPactTask.java:1533) > > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:367) > > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:204) > > > at java.lang.Thread.run(Thread.java:745) > > > > > > the same exception is in the task manager logs, along with the > following > > one: > > > > > > java.util.concurrent.TimeoutException: Futures timed out after [100 > > seconds] > > > at > > scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:219) > > > at > > scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223) > > > at > > scala.concurrent.Await$$anonfun$result$1.apply(package.scala:107) > > > at > > > scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:53) > > > at scala.concurrent.Await$.result(package.scala:107) > > > at > > org.apache.flink.runtime.akka.AkkaUtils$.ask(AkkaUtils.scala:265) > > > at org.apache.flink.runtime.akka.AkkaUtils.ask(AkkaUtils.scala) > > > at > > > org.apache.flink.runtime.io.network.partition.IntermediateResultPartition.scheduleOrUpdateConsumers(IntermediateResultPartition.java:247) > > > at > > > org.apache.flink.runtime.io.network.partition.IntermediateResultPartition.maybeNotifyConsumers(IntermediateResultPartition.java:240) > > > at > > > org.apache.flink.runtime.io.network.partition.IntermediateResultPartition.add(IntermediateResultPartition.java:144) > > > at > > > org.apache.flink.runtime.io.network.api.writer.BufferWriter.writeBuffer(BufferWriter.java:74) > > > at > > > org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:91) > > > at > > > org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:88) > > > at > > > org.apache.flink.languagebinding.api.java.common.streaming.Receiver.collectBuffer(Receiver.java:253) > > > at > > > org.apache.flink.languagebinding.api.java.common.streaming.Streamer.streamBufferWithoutGroups(Streamer.java:193) > > > at > > > org.apache.flink.languagebinding.api.java.python.functions.PythonMapPartition.mapPartition(PythonMapPartition.java:54) > > > at > > > org.apache.flink.runtime.operators.MapPartitionDriver.run(MapPartitionDriver.java:98) > > > at > > > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:496) > > > at > > > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:360) > > > at > > > org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:204) > > > at java.lang.Thread.run(Thread.java:745) > > > > > > > >