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 problematic if 
processing a single record takes/can take significant amount of time. 

With Flink 1.5 we introduced mechanism to better control the amount of buffered 
data and it should address this issue (Flink 1.5 should be released within 
couple of weeks).

In the mean time, you could try out Flink 1.5 release candidate that has been 
just published or you could try to reduce the number of configured network 
buffers, however keep in mind that at some point this can decrease your maximal 
throughput:

https://ci.apache.org/projects/flink/flink-docs-release-1.4/ops/config.html#configuring-the-network-buffers
 
<https://ci.apache.org/projects/flink/flink-docs-release-1.4/ops/config.html#configuring-the-network-buffers>

On the other hand, why does it prevents you from using a checkpointing at all? 

Piotr Nowojski 

> On 5 Apr 2018, at 06:10, Edward <egb...@hotmail.com> wrote:
> 
> 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 anywhere from 5 minutes
> to 25 minutes.
> 
> This is preventing us from using the checkpointing feature at all, since
> periodic backpressure is unavoidable.
> 
> We are experiencing this when running on Flink 1.4.0.
> We are retaining only a single checkpoint, and the size of retained
> checkpoint is less than 250KB, so there's not a lot of state.
>   state.backend: jobmanager
>   state.backend.async: true
>   state.backend.fs.checkpointdir: hdfs://checkpoints
>   state.checkpoints.num-retained: 1
>   max concurrent checkpoints: 1
>   checkpointing mode: AT_LEAST_ONCE
> 
> One other data point: if I rewrite the job to allow chaining all steps (i.e.
> same parallelism on all steps, so they fit in 1 task slot), the checkpoints
> are still slow under backpressure, but are an order of magnitude faster --
> they take about 60 seconds rather than 15 minutes.
> 
> 
> 
> --
> Sent from: 
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Reply via email to