Re: Checkpoints very slow with high backpressure

2019-07-31 Thread Piotr Nowojski
Hi, For Flink 1.8 (and 1.9) the only thing that you can do, is to try to limit amount of data buffered between the nodes (check Flink network configuration [1] for number of buffers and or buffer pool sizes). This can reduce maximal throughput (but only if the network transfer is a significant

Checkpoints very slow with high backpressure

2019-07-30 Thread Mohammad Hosseinian
Hi, I'm still facing the same issue under 1.8. Our pipeline uses end-to-end exactly-once semantic, which means the consumer program cannot read the messages until they are committed. So in case of an outage, the whole runtime delay is passed over to the next stream processor application and create

回复:Checkpoints very slow with high backpressure

2018-04-07 Thread Zhijiang(wangzhijiang999)
slow with high backpressure Thanks for the explanation. I hope that either 1.5 will solve your issue (please let us know if it  doesn’t!) or if you can’t wait, that decreasing memory buffers can mitigate the  problem. Piotrek > On 5 Apr 2018, at 08:13, Edward  wrote: >  > Thanks for t

Re: Checkpoints very slow with high backpressure

2018-04-05 Thread Piotr Nowojski
Thanks for the explanation. I hope that either 1.5 will solve your issue (please let us know if it doesn’t!) or if you can’t wait, that decreasing memory buffers can mitigate the problem. Piotrek > On 5 Apr 2018, at 08:13, Edward wrote: > > Thanks for the update Piotr. > > The reason it pre

Re: Checkpoints very slow with high backpressure

2018-04-05 Thread Edward
Thanks for the update Piotr. The reason it prevents us from using checkpoints is this: We are relying on the checkpoints to trigger commit of Kafka offsets for our source (kafka consumers). When there is no backpressure this works fine. When there is backpressure, checkpoints fail because they tak

Re: Checkpoints very slow with high backpressure

2018-04-05 Thread Piotr Nowojski
Hi, If I’m not mistaken this is a known issue, that we were working to resolve for Flink 1.5 release. The problem is that with back pressure, data are being buffered between nodes and on checkpoint, all of those data must be processed before checkpoint can be completed. This is especially probl

Re: Checkpoints very slow with high backpressure

2018-04-05 Thread Edward
I read through this thread and didn't see any resolution to the slow checkpoint issue (just that someone resolved their backpressure issue). We are experiencing the same problem: - When there is no backpressure, checkpoints take less than 100ms - When there is high backpressure, checkpoints take

Re: Checkpoints very slow with high backpressure

2017-06-02 Thread rhashmi
Nvm i found it. Backpressure caused by aws RDS instance of mysql. -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoints-very-slow-with-high-backpressure-tp12762p13468.html Sent from the Apache Flink User Mailing List archive. mailing

Re: Checkpoints very slow with high backpressure

2017-06-01 Thread rhashmi
or - Received late message for now expired checkpoint attempt 1 from c601dd04affa7da13a226daa222062e7 of job 303656ace348131ed7a38bb02b4fe374. -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoints-very-slow-with-high-backpressure-tp12762p13422.html Se

Re: Checkpoints very slow with high backpressure

2017-06-01 Thread rhashmi
Anyidea? -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoints-very-slow-with-high-backpressure-tp12762p13419.html Sent from the Apache Flink User Mailing List archive. mailing list archive at Nabble.com.

Re: Checkpoints very slow with high backpressure

2017-06-01 Thread rhashmi
e in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoints-very-slow-with-high-backpressure-tp12762p13418.html Sent from the Apache Flink User Mailing List archive. mailing list archive at Nabble.com.

Re: Checkpoints very slow with high backpressure

2017-05-31 Thread Chen Qin
ne >> database. >> >> is there recommended approach to handle backpressure? >> >> Version Flink 1.2. >> >> >> >> >> >> >> -- >> View this message in context: http://apache-flink-user-maili >> ng-list-archive.2336050.n4.nabble.com/Checkpoints-very- >> slow-with-high-backpressure-tp12762p13411.html >> Sent from the Apache Flink User Mailing List archive. mailing list >> archive at Nabble.com. >> > >

Re: Checkpoints very slow with high backpressure

2017-05-31 Thread SHI Xiaogang
e backpressure? > > Version Flink 1.2. > > > > > > > -- > View this message in context: http://apache-flink-user- > mailing-list-archive.2336050.n4.nabble.com/Checkpoints- > very-slow-with-high-backpressure-tp12762p13411.html > Sent from the Apache Flink User Mailing List archive. mailing list archive > at Nabble.com. >

Re: Checkpoints very slow with high backpressure

2017-05-31 Thread rhashmi
are also not firing so no update being made to down line database. is there recommended approach to handle backpressure? Version Flink 1.2. -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoints-very-slow-with-high-backpressure

Re: Checkpoints very slow with high backpressure

2017-04-24 Thread Yassine MARZOUGUI
Hi Ufuk, The ProcessFunction receives elements and buffers them into a MapState, and periodically (for example every x seconds) register processing time timers (according to some rules which it gets from a connected rule stream). When a timer fires, I pop next element from state, request an extern

Re: Checkpoints very slow with high backpressure

2017-04-24 Thread Ufuk Celebi
@Yessine: no, there is no way to disable the back pressure mechanism. Do you have more details about the two last operators? What do you mean with the process function is slow on purpose? @Rune: with 1.3 Flink will configure the internal buffers in a way that not too much data is buffered in the i

Re: Checkpoints very slow with high backpressure

2017-04-24 Thread Rune Skou Larsen
Sorry I cant help you, but we're also experiencing slow checkpointing, when having backpressure from sink. I tried HDFS, S3, and RocksDB state backends, but to no avail - checkpointing always times out with backpressure. Can we somehow reduce Flink's internal buffer sizes, so checkpointing w

Checkpoints very slow with high backpressure

2017-04-24 Thread Yassine Marzougui
Hi all, I have a streaming pipeline as follows: 1 - read a folder continuousely from HDFS 2 - filter duplicates (using keyby(x->x) and keeping a state per key indicating whether its is seen) 3 - schedule some future actions on the stream using ProcessFunction and processing time timers (elements a

Fwd: Checkpoints very slow with high backpressure

2017-04-24 Thread Yassine MARZOUGUI
-- Forwarded message -- From: "Yassine MARZOUGUI" Date: Apr 23, 2017 20:53 Subject: Checkpoints very slow with high backpressure To: Cc: Hi all, I have a Streaming pipeline as follows: 1 - read a folder continuousely from HDFS 2 - filter duplicates (using keyby

Re: Checkpoints very slow with high backpressure

2017-04-24 Thread Yassine MARZOUGUI
Im sorry guys if you received multiple instances of this mail, I kept trying to send it yesterday, but looks like the mailing list was stuck and didn't dispatch it until now. Sorry for the disturb. On Apr 23, 2017 20:53, "Yassine MARZOUGUI" wrote: > Hi all, > > I have a Streaming pipeline as fol

Checkpoints very slow with high backpressure

2017-04-24 Thread Yassine MARZOUGUI
Hi all, I have a streaming pipeline as follows: 1 - read a folder continuousely from HDFS 2 - filter duplicates (using keyby(x->x) and keeping a state per key indicating whether its is seen) 3 - schedule some future actions on the stream using ProcessFunction and processing time timers (elements a

Checkpoints very slow with high backpressure

2017-04-24 Thread Yassine MARZOUGUI
Re-sending as it looks like the previous mail wasn't correctly sent --- Hi all, I have a streaming pipeline as follows: 1 - read a folder continuousely from HDFS 2 - filter duplicates (using keyby(x->x) and keeping a state per key indicating whether its is seen) 3

Checkpoints very slow with high backpressure

2017-04-24 Thread Yassine MARZOUGUI

Checkpoints very slow with high backpressure

2017-04-23 Thread Yassine Marzougui

Checkpoints very slow with high backpressure

2017-04-23 Thread Yassine MARZOUGUI
Hi all, I have a Streaming pipeline as follows: 1 - read a folder continuousely from HDFS 2 - filter duplicates (using keyby(x->x) and keeping a state per key indicating whether its is seen) 3 - schedule some future actions on the stream using ProcessFunction and processing time timers (elements a