Re: [flink-connector-filesystem] OutOfMemory in checkpointless environment

2018-06-08 Thread Rinat
Chesnay, thx for your reply, I’ve created one https://issues.apache.org/jira/browse/FLINK-9558 > On 8 Jun 2018, at 12:58, Chesnay Schepler wrote: > > I agree, if the sink doesn't properly work without checkpointing we should > make sure that

Re: [flink-connector-filesystem] OutOfMemory in checkpointless environment

2018-06-08 Thread Chesnay Schepler
I agree, if the sink doesn't properly work without checkpointing we should make sure that it fails early if it used that way. It would be great if you could open a JIRA. On 08.06.2018 10:08, Rinat wrote: Piotr, thx for your reply, for now everything is pretty clear. But from my point of view,

Re: [flink-connector-filesystem] OutOfMemory in checkpointless environment

2018-06-08 Thread Rinat
Piotr, thx for your reply, for now everything is pretty clear. But from my point of view, it’s better to add some information about leaks in case of disabled checkpointing into BucketingSink documentation > On 8 Jun 2018, at 10:35, Piotr Nowojski wrote: > > Hi, > > BucketingSink is designed t

Re: [flink-connector-filesystem] OutOfMemory in checkpointless environment

2018-06-08 Thread Piotr Nowojski
Hi, BucketingSink is designed to provide exactly-once writes to file system, which is inherently tied to checkpointing. As you just saw, without checkpointing, BucketingSink is never notified that it can commit pending files. If you do not want to use checkpointing for some reasons, you could

[flink-connector-filesystem] OutOfMemory in checkpointless environment

2018-06-07 Thread Rinat
Hi mates, we got some Flink jobs, that are writing data from kafka into hdfs, using Bucketing-Sink. For some reasons, those jobs are running without checkpointing. For now, it not a big problem for us, if some files are remained opened in case of job reloading. Periodically, those jobs fail wit