Hi, We are currently using BucketingSink to save data into HDFS in parquet format. But when the flink job was cancelled, we always got Exception in BucketingSink's close method. The datailed exception info is as below: [ERROR] [2017-09-26 20:51:58,893] [org.apache.flink.streaming.runtime.tasks.StreamTask] - Error during disposal of stream operator. java.io.InterruptedIOException: Interrupted while waiting for data to be acknowledged by pipeline at org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:2151) at org.apache.hadoop.hdfs.DFSOutputStream.flushInternal(DFSOutputStream.java:2130) at org.apache.hadoop.hdfs.DFSOutputStream.closeImpl(DFSOutputStream.java:2266) at org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:2236) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) at org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:643) at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:117) at org.apache.parquet.hadoop.ParquetWriter.close(ParquetWriter.java:301) ....... at org.apache.flink.api.common.functions.util.FunctionUtils.closeFunction(FunctionUtils.java:43) at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.dispose(AbstractUdfStreamOperator.java:126) at org.apache.flink.streaming.runtime.tasks.StreamTask.disposeAllOperators(StreamTask.java:429) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:334) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702) at java.lang.Thread.run(Thread.java:745) It seems that DFSOutputStream haven't been closed before task thread is force terminated. We found a similar problem in http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Changing-timeout-for-cancel-command-td12601.html , but setting "akka.ask.timeout" to a larger value does not work for us. So how can we make sure the stream is safely closed when cacelling a job? Best, wangsan |
After digging into the source code, we found that when Flink job is canceled, a The
Notice that TaskCanceler first send interrupt signal to task thread, and following with join method. And since the task thread is now try to close DFSOutputStream, which is waiting for ack, thus InterruptedException is throwed out in task thread.
I was confused why
|
Hi,
I would speculate that the reason for this order is that we want to shutdown the tasks quickly by interrupting blocking calls in the event of failure, so that recover can begin as fast as possible. I am looping in Stephan who might give more details about this code. Best, Stefan
|
Hi, 'Join' method can be call with a timeout (as is called in TaskCanceler), so it won't be block forever if the respective thread is in deadlock state. Maybe calling 'interrupt()' after 'join(timeout)' is more reasonable, altought it still can not make sure operations inside 'close()' method is finished. Best, wangsan 在2017年09月29 01时52分, "Stephan Ewen"<[hidden email]>写道:
|
In reply to this post by wangsan
Hi Wangsan, We were struggling with this for many days as well. In the end we found a work around. Well work-around, this for sure qualifies as one of the ugliest hacks I have ever contemplated. Our work-around for Flink immediately interrupting the close, is to continue closing on another thread! Here is an example in Scala: class MyBucketingSink[A](basePath: String) extends BucketingSink[A](basePath) { Obviously, if the close hangs, the entire job will hang and Flink will need to be fully restarted. Please let us know if you see any other problems with this approach. Kind regards, Erik.
|
Free forum by Nabble | Edit this page |