Thanks a lot Stephan, as far as we observed the error, it always occurred in an iteration later than the first one (mostly iteration 2). I filed a bug report here: https://issues.apache.org/jira/browse/FLINK-2334 <https://issues.apache.org/jira/browse/FLINK-2334>.
And thanks for pointing out the workaround. We will try forcing a sort-merge join. > On 09 Jul 2015, at 10:37, Stephan Ewen <se...@apache.org > <mailto:se...@apache.org>> wrote: > > Can you verify that this always happens in an iteration later than the first > one? It may be that the cached build-side files are accidentally removed too > early. > > Can you file a bug report for this? Please make sure you add the stack trace > and the suspicion that cached build partitions are deleted to early. > > A workaround could be to force a sort-merge join for the time being. > > On Thu, Jul 9, 2015 at 12:17 AM, Heller, David <david.hel...@student.hpi.de > <mailto:david.hel...@student.hpi.de>> wrote: > Hello, > > we are implementing a text mining algorithm on Flink (0.9.0) and encountered > a mysterious IOException (see below for detailed stacktrace). Flink seems to > have problems finding some temporary file even though there is enough free > space on the disk. Do you have an idea why the file cannot be found? > > We've encountered the exception both in local and cluster execution and on > MacOS as well as on linux. > Data size does not seem to be the reason: the error occurred on a 6.5GB > dataset as well as on a small 400MB dataset > Our algorithm works iteratively and uses Bulk iterations. Interestingly, on > one occasion the error disappeared consistently when setting the iteration > number higher (from 2 to 6). > On another occasion, the exception appeared when adding a simple map operator > at the end (holding the identity function). > Generally, the error is quite hard to reproduce. > > Thanks in advance for any ideas and your time! > > David > > > Stacktrace: > > java.io.IOException: Channel to path > '/var/folders/xx/0dd3w4jd7fbb4ytmhqxm157h0000gn/T/flink-io-f5061483-ff59-43dc-883f-79af813d5804/19a70637e025c7ee3919b30239060895.000023.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.AsynchronousBulkBlockReader.<init>(AsynchronousBulkBlockReader.java:46) > at > org.apache.flink.runtime.io.disk.iomanager.AsynchronousBulkBlockReader.<init>(AsynchronousBulkBlockReader.java:39) > at > org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync.createBulkBlockChannelReader(IOManagerAsync.java:263) > at > org.apache.flink.runtime.operators.hash.MutableHashTable.buildTableFromSpilledPartition(MutableHashTable.java:751) > at > org.apache.flink.runtime.operators.hash.MutableHashTable.prepareNextPartition(MutableHashTable.java:508) > at > org.apache.flink.runtime.operators.hash.ReOpenableMutableHashTable.prepareNextPartition(ReOpenableMutableHashTable.java:167) > at > org.apache.flink.runtime.operators.hash.MutableHashTable.nextRecord(MutableHashTable.java:544) > at > org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator.callWithNextKey(NonReusingBuildFirstHashMatchIterator.java:104) > at > org.apache.flink.runtime.operators.AbstractCachedBuildSideMatchDriver.run(AbstractCachedBuildSideMatchDriver.java:155) > at > org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:496) > at > org.apache.flink.runtime.iterative.task.AbstractIterativePactTask.run(AbstractIterativePactTask.java:139) > at > org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask.run(IterationIntermediatePactTask.java:92) > at > org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:559) > at java.lang.Thread.run(Thread.java:745) > > Caused by: java.io.FileNotFoundException: > /var/folders/xx/0dd3w4jd7fbb4ytmhqxm157h0000gn/T/flink-io-f5061483-ff59-43dc-883f-79af813d5804/19a70637e025c7ee3919b30239060895.000023.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) > ... 16 more >