Re: High back-pressure after recovering from a save point

2017-07-16 Thread Kien Truong
Hi, We have been testing with the FsStateBackend for the last few days and have not encountered this issue anymore. However, we will evaluate the rocksdb backend again soon because we want incremental checkpoint. I will report back if I have more updates. Best regards, Kien On Jul 15, 2017,

Re: High back-pressure after recovering from a save point

2017-07-14 Thread Gyula Fóra
It will work if you assign a new uid to the Kafka source. Gyula On Fri, Jul 14, 2017, 18:42 Tzu-Li (Gordon) Tai wrote: > One thing: do note that `FlinkKafkaConsumer#setStartFromLatest()` does not > have any effect when starting from savepoints. > i.e., the consumer will still start from whateve

Re: High back-pressure after recovering from a save point

2017-07-14 Thread Tzu-Li (Gordon) Tai
One thing: do note that `FlinkKafkaConsumer#setStartFromLatest()` does not have any effect when starting from savepoints. i.e., the consumer will still start from whatever offset is written in the savepoint. On 15 July 2017 at 12:38:10 AM, Tzu-Li (Gordon) Tai (tzuli...@apache.org) wrote: Can yo

Re: High back-pressure after recovering from a save point

2017-07-14 Thread Tzu-Li (Gordon) Tai
Can you try starting from the savepoint, but telling Kafka to start from the latest offset? (@gordon: Is that possible in Flink 1.3.1 or only in 1.4-SNAPSHOT ?) This is already possible in Flink 1.3.x. `FlinkKafkaConsumer#setStartFromLatest()` would be it. On 15 July 2017 at 12:33:53 AM, Steph

Re: High back-pressure after recovering from a save point

2017-07-14 Thread Stephan Ewen
Can you try starting from the savepoint, but telling Kafka to start from the latest offset? (@gordon: Is that possible in Flink 1.3.1 or only in 1.4-SNAPSHOT ?) On Fri, Jul 14, 2017 at 11:18 AM, Kien Truong wrote: > Hi, > > Sorry for the version typo, I'm running 1.3.1. I did not test with 1.2.

Re: High back-pressure after recovering from a save point

2017-07-14 Thread Kien Truong
Hi, Sorry for the version typo, I'm running 1.3.1. I did not test with 1.2.x. The jobs runs fine with almost 0 back-pressure if it's started from scratch or if I reuse the kafka consumers group id without specifying the safe point. Best regards, Kien On Jul 14, 2017, 15:59, at 15:59, Stephan E

Re: High back-pressure after recovering from a save point

2017-07-14 Thread Stephan Ewen
Hi! Flink 1.3.2 does not yet exist. Do you mean 1.3.1 or latest master? Can you tell us whether this occurs only in 1.3.x and worked well in 1.2.x? If you just keep the job running without savepoint/restore, you do not get into backpressure situations? Thanks, Stephan On Fri, Jul 14, 2017 at 1

Re: High back-pressure after recovering from a save point

2017-07-13 Thread Kien Truong
Hi Fabian, This happens to me even when the restore is immediate, so there's not much data in Kafka to catch up (5 minutes max) Regards Kien On Jul 13, 2017, 23:40, at 23:40, Fabian Hueske wrote: >I would guess that this is quite usual because the job has to >"catch-up" >work. >For example, if

Re: High back-pressure after recovering from a save point

2017-07-13 Thread Fabian Hueske
I would guess that this is quite usual because the job has to "catch-up" work. For example, if you took a save point two days ago and restore the job today, the input data of the last two days has been written to Kafka (assuming Kafka as source) and needs to be processed. The job will now read as f

High back-pressure after recovering from a save point

2017-07-13 Thread Kien Truong
Hi all, I have one job where back-pressure is significantly higher after resuming from a save point. Because that job makes heavy use of stateful functions with RocksDBStateBackend , I'm suspecting that this is the cause of performance degradation. Does anyone encounter simillar issues or