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 02:32:00 UTC

[jira] [Updated] (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:all-tabpanel ]

zhu.qing updated FLINK-8534:
----------------------------
    Description: 
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

  was:
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. 


> 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)