You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "zhu.qing (JIRA)" <ji...@apache.org> on 2018/02/01 03:25:00 UTC
[jira] [Commented] (FLINK-8534) if insert too much BucketEntry into
one bucket in join of iteration will cause a error (Caused :
java.io.FileNotFoundException release file error)
[ https://issues.apache.org/jira/browse/FLINK-8534?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16347953#comment-16347953 ]
zhu.qing commented on FLINK-8534:
---------------------------------
Just set Xms to 1g will reproduce the bug
> if insert too much BucketEntry into one bucket in join of iteration will cause a error (Caused : java.io.FileNotFoundException release file error)
> --------------------------------------------------------------------------------------------------------------------------------------------------
>
> Key: FLINK-8534
> URL: https://issues.apache.org/jira/browse/FLINK-8534
> Project: Flink
> Issue Type: Bug
> Components: Local Runtime
> Environment: windows, intellij idea, 8g ram, 4core i5 cpu, Flink 1.4.0, and parallelism = 2 will cause problem and others will not.
> Reporter: zhu.qing
> Priority: Major
> Attachments: T2AdjSetBfs.java
>
>
> When insert too much entry into bucket (MutableHashTable insertBucketEntry() line 1054 more than 255) will cause spillPartition() (HashPartition line 317). So
> this.buildSideChannel = ioAccess.createBlockChannelWriter(targetChannel, bufferReturnQueue);
> And in
> prepareNextPartition() of ReOpenableMutableHashTable (line 156)
> furtherPartitioning = true;
> so in
> finalizeProbePhase() in HashPartition (line 367)
> this.probeSideChannel.close();
> //the file will be delete
> this.buildSideChannel.deleteChannel();
> this.probeSideChannel.deleteChannel();
> after deleteChannel the next iteartion will fail.
>
> And I use web-google(SNAP) as dataset.
>
> Exception in thread "main" org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
> at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:897)
> at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:840)
> at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:840)
> 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:39)
> at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)
> 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.io.IOException: Channel to path 'C:\Users\sanquan.qz\AppData\Local\Temp\flink-io-5af23edc-1ec0-4718-87a5-916ee022a8be\fc08af25b6f879b8e7bb24291c47ea1d.000018.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:294)
> at org.apache.flink.runtime.operators.hash.MutableHashTable.buildTableFromSpilledPartition(MutableHashTable.java:880)
> at org.apache.flink.runtime.operators.hash.MutableHashTable.prepareNextPartition(MutableHashTable.java:637)
> at org.apache.flink.runtime.operators.hash.ReOpenableMutableHashTable.prepareNextPartition(ReOpenableMutableHashTable.java:170)
> at org.apache.flink.runtime.operators.hash.MutableHashTable.nextRecord(MutableHashTable.java:675)
> at org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashJoinIterator.callWithNextKey(NonReusingBuildFirstHashJoinIterator.java:117)
> at org.apache.flink.runtime.operators.AbstractCachedBuildSideJoinDriver.run(AbstractCachedBuildSideJoinDriver.java:176)
> at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:490)
> at org.apache.flink.runtime.iterative.task.AbstractIterativeTask.run(AbstractIterativeTask.java:145)
> at org.apache.flink.runtime.iterative.task.IterationIntermediateTask.run(IterationIntermediateTask.java:93)
> at org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:355)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.FileNotFoundException: C:\Users\sanquan.qz\AppData\Local\Temp\flink-io-5af23edc-1ec0-4718-87a5-916ee022a8be\fc08af25b6f879b8e7bb24291c47ea1d.000018.channel (系统找不到指定的文件。)
> 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
> Process finished with exit code 1
> here is the full stack of exception
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)