Hi Max, thank you for the fast reply and sorry: I use flink-1.0.3. Yes I tested on dummy dataset with numOfBuffers = 16384 and decreasing the parallelism degree and this solution solved the first exception. Anyway on the 80GiB dataset I struggle with the second exception.
Regards, Andrea 2016-06-28 12:08 GMT+02:00 Maximilian Michels <m...@apache.org>: > Hi Andrea, > > The number of network buffers should be sufficient. Actually, assuming > you have 16 task slots on each of the 25 nodes, it should be enough to > have 16^2 * 25 * 4 = 14400 network buffers. > > See > https://ci.apache.org/projects/flink/flink-docs-master/setup/config.html#background > > So we have to investigate a little more. Which version of Flink are you > using? > > Cheers, > Max > > On Tue, Jun 28, 2016 at 11:57 AM, ANDREA SPINA > <74...@studenti.unimore.it> wrote: > > Hi everyone, > > > > I am running some Flink experiments with Peel benchmark > > http://peel-framework.org/ and I am struggling with exceptions: the > > environment is a 25-nodes cluster, 16 cores per nodes. The dataset is > ~80GiB > > and is located on Hdfs 2.7.1. > > > > At the beginning I tried with 400 as degree of parallelism and with the > > following configuration: > > > > jobmanager.rpc.address = ${runtime.hostname} > > akka.log.lifecycle.events = ON > > akka.ask.timeout = 300s > > jobmanager.rpc.port = 6002 > > > > jobmanager.heap.mb = 1024 > > jobmanager.web.port = 6004 > > > > taskmanager.heap.mb = 28672 > > taskmanager.memory.fraction = 0.7 > > taskmanager.network.numberOfBuffers = 32768 > > taskmanager.network.bufferSizeInBytes = 16384 > > taskmanager.tmp.dirs = > > > "/data/1/peel/flink/tmp:/data/2/peel/flink/tmp:/data/3/peel/flink/tmp:/data/4/peel/flink/tmp" > > taskmanager.debug.memory.startLogThread = true > > > > the following exception will raise > > > > Caused by: java.io.IOException: Insufficient number of network buffers: > > required 350, but only 317 available. The total number of network > buffers is > > currently set to 32768. You can increase this number by setting the > > configuration key 'taskmanager.network.numberOfBuffers'. > > at > > > org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.createBufferPool(NetworkBufferPool.java:196) > > at > > > org.apache.flink.runtime.io.network.NetworkEnvironment.registerTask(NetworkEnvironment.java:327) > > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:469) > > at java.lang.Thread.run(Thread.java:745) > > > > So I tried different solutions, both with increasing numberOfBuffers (Max > > value tried 98304) or decreasing the degreeOfParallelism (Min value tried > > 300) and testing those configs with a dummy dataset seems to solve the > > number of buffers issue. > > But In each case with the 80GiB dataset now I struggle with a new > exception; > > the following with a degree of parallelism = 300 and numberOfBuffers = > 32768 > > > > org.apache.flink.client.program.ProgramInvocationException: The program > > execution failed: Job execution failed. > > at org.apache.flink.client.program.Client.runBlocking(Client.java:381) > > at org.apache.flink.client.program.Client.runBlocking(Client.java:355) > > at org.apache.flink.client.program.Client.runBlocking(Client.java:315) > > at > > > org.apache.flink.client.program.ContextEnvironment.execute(ContextEnvironment.java:60) > > at > > > org.apache.flink.api.scala.ExecutionEnvironment.execute(ExecutionEnvironment.scala:652) > > at > > > dima.tu.berlin.benchmark.flink.mlr.FlinkSLRTrainCommon$.main(FlinkSLRTrainCommon.scala:110) > > at > > > dima.tu.berlin.benchmark.flink.mlr.FlinkSLRTrainCommon.main(FlinkSLRTrainCommon.scala) > > 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:498) > > at > > > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:505) > > at > > > org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:403) > > at org.apache.flink.client.program.Client.runBlocking(Client.java:248) > > at > > > org.apache.flink.client.CliFrontend.executeProgramBlocking(CliFrontend.java:866) > > at org.apache.flink.client.CliFrontend.run(CliFrontend.java:333) > > at > > > org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1192) > > at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1243) > > Caused by: org.apache.flink.runtime.client.JobExecutionException: Job > > execution failed. > > at > > > org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$7.apply$mcV$sp(JobManager.scala:717) > > at > > > org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$7.apply(JobManager.scala:663) > > at > > > org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$7.apply(JobManager.scala:663) > > at > > > scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24) > > at > > > scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24) > > at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:41) > > at > > > akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:401) > > at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > > at > > > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.pollAndExecAll(ForkJoinPool.java:1253) > > at > > > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1346) > > at > scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > > at > > > scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > > Caused by: java.lang.RuntimeException: Emitting the record caused an I/O > > exception: Channel to path > > > '/data/3/peel/flink/tmp/flink-io-3a97d62c-ada4-44f1-ab72-dd018386f9aa/79305169d69f7e8b361a175af87353fa.channel' > > could not be opened. > > at > > > org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:69) > > at > > > org.apache.flink.runtime.operators.chaining.ChainedMapDriver.collect(ChainedMapDriver.java:78) > > at org.apache.flink.runtime.operators.MapDriver.run(MapDriver.java:97) > > at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:480) > > at > org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:345) > > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:559) > > at java.lang.Thread.run(Thread.java:745) > > Caused by: java.io.IOException: Channel to path > > > '/data/3/peel/flink/tmp/flink-io-3a97d62c-ada4-44f1-ab72-dd018386f9aa/79305169d69f7e8b361a175af87353fa.channel' > > could not be opened. > > at > > > org.apache.flink.runtime.io.disk.iomanager.AbstractFileIOChannel.<init>(AbstractFileIOChannel.java:61) > > at > > > org.apache.flink.runtime.io.disk.iomanager.AsynchronousFileIOChannel.<init>(AsynchronousFileIOChannel.java:86) > > at > > > org.apache.flink.runtime.io.disk.iomanager.AsynchronousBufferFileWriter.<init>(AsynchronousBufferFileWriter.java:31) > > at > > > org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync.createBufferFileWriter(IOManagerAsync.java:257) > > at > > > org.apache.flink.runtime.io.network.partition.SpillableSubpartition.releaseMemory(SpillableSubpartition.java:151) > > at > > > org.apache.flink.runtime.io.network.partition.ResultPartition.releaseMemory(ResultPartition.java:366) > > at > > > org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestBuffer(LocalBufferPool.java:159) > > at > > > org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestBufferBlocking(LocalBufferPool.java:133) > > at > > > org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:92) > > at > > > org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:65) > > ... 6 more > > Caused by: java.io.FileNotFoundException: > > > /data/3/peel/flink/tmp/flink-io-3a97d62c-ada4-44f1-ab72-dd018386f9aa/79305169d69f7e8b361a175af87353fa.channel > > (No such file or directory) > > at java.io.RandomAccessFile.open0(Native Method) > > at java.io.RandomAccessFile.open(RandomAccessFile.java:316) > > at java.io.RandomAccessFile.<init>(RandomAccessFile.java:243) > > at java.io.RandomAccessFile.<init>(RandomAccessFile.java:124) > > at > > > org.apache.flink.runtime.io.disk.iomanager.AbstractFileIOChannel.<init>(AbstractFileIOChannel.java:57) > > ... 15 more > > > > here the related jobmanager full log. I can't figure out a solution. > > > > Thank you and have a nice day. > > > > -- > > Andrea Spina > > Guest student at DIMA, TU Berlin > > N.Tessera: 74598 > > MAT: 89369 > > Ingegneria Informatica [LM] (D.M. 270) > -- *Andrea Spina* N.Tessera: *74598* MAT: *89369* *Ingegneria Informatica* *[LM] *(D.M. 270)