Hi,

Generally speaking changes of parallelism is supported between checkpoints and 
savepoints. Other changes to the job’s topology, like adding/changing/removing 
operators, changing types in the job graph are only officially supported via 
savepoints.

But in reality, as for now, there is no difference between checkpoints and 
savepoints, but that’s subject to change, so it’s better not to relay this 
behaviour. For example with unaligned checkpoints [1] (hopefully in 1.11), 
there will be a difference between those two concepts.

Piotrek

[1] 
https://cwiki.apache.org/confluence/display/FLINK/FLIP-76%3A+Unaligned+Checkpoints
 
<https://cwiki.apache.org/confluence/display/FLINK/FLIP-76:+Unaligned+Checkpoints>

> On 12 Mar 2020, at 12:16, Aaron Levin <aaronle...@stripe.com> wrote:
> 
> Hi,
> 
> What's the expected behaviour of:
> 
> * changing an operator's parallelism
> * deploying this change from an incremental (RocksDB) checkpoint instead of a 
> savepoint
> 
> The flink docs[0][1] are a little unclear on what the expected behaviour is 
> here. I understand that the key-space is being changed because parallelism is 
> changed. I've seen instances where this happens and a job does not fail. But 
> how does it treat potentially missing state for a given key? 
> 
> I know I can test this, but I'm curious what the _expected_ behaviour is? 
> I.e. what behaviour can I rely on, which won't change between versions or 
> releases? Do we expect the job to fail? Do we expect missing keys to just be 
> considered empty? 
> 
> Thanks!
> 
> [0] 
> https://ci.apache.org/projects/flink/flink-docs-stable/ops/state/savepoints.html#what-is-a-savepoint-how-is-a-savepoint-different-from-a-checkpoint
>  
> <https://ci.apache.org/projects/flink/flink-docs-stable/ops/state/savepoints.html#what-is-a-savepoint-how-is-a-savepoint-different-from-a-checkpoint>
> [1] 
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/upgrading.html
>  
> <https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/upgrading.html>
> 
> Aaron Levin

Reply via email to