Also +1 in general. I have a few questions though:
- does it only apply to the logic in org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory#createRestartStrategyFactory, which is only the cluster side configuration? Or do you want to change the logic also on the job side in ExecutionConfig? - if the latter, does that mean deprecated methods in ExecutionConfig like: setNumberOfExecutionRetries, setExecutionRetryDelay will have no effect? I think this would be a good idea, but would suggest to remove the corresponding fields and methods. This is not that simple though. I tried to do that for other parameters that have no effect already like codeAnalysisMode & failTaskOnCheckpointError. The are two problems: 1) setNumberOfExecutionRetires are effectively marked with @Public annotation (the codeAnalysisMode & failTaskOnCheckpointError don't have this problem). Therefore this would be a binary incompatible change. 2) ExecutionConfig is stored in state as part of PojoSerializer in pre flink 1.7. It should not be a problem for numberOfExecutionRetries & executionRetryDelays as they are of primitive types. It is a problem for codeAnalysisMode (we cannot remove the class, as this breaks serialization). I wanted to mention that anyway, just to be aware of that. Best, Dawid On 30/08/2019 14:48, Stephan Ewen wrote: > +1 in general > > What is the default in batch, though? No restarts? I always found that > somewhat uncommon. > Should we also change that part, if we are changing the default anyways? > > > On Fri, Aug 30, 2019 at 2:35 PM Till Rohrmann <trohrm...@apache.org> wrote: > >> Hi everyone, >> >> I wanted to discuss how to simplify Flink's cluster level RestartStrategy >> configuration [1]. Currently, Flink's behaviour with respect to configuring >> the {{RestartStrategies}} is quite complicated and convoluted. The reason >> for this is that we evolved the way it has been configured and wanted to >> keep it backwards compatible. Due to this, we have currently the following >> behaviour: >> >> * If the config option `restart-strategy` is configured, then Flink uses >> this `RestartStrategy` (so far so simple) >> * If the config option `restart-strategy` is not configured, then >> ** If `restart-strategy.fixed-delay.attempts` or >> `restart-strategy.fixed-delay.delay` are defined, then instantiate >> `FixedDelayRestartStrategy(restart-strategy.fixed-delay.attempts, >> restart-strategy.fixed-delay.delay)` >> ** If `restart-strategy.fixed-delay.attempts` and >> `restart-strategy.fixed-delay.delay` are not defined, then >> *** If checkpointing is disabled, then choose `NoRestartStrategy` >> *** If checkpointing is enabled, then choose >> `FixedDelayRestartStrategy(Integer.MAX_VALUE, "0 s")` >> >> I would like to simplify the configuration by removing the "If >> `restart-strategy.fixed-delay.attempts` or >> `restart-strategy.fixed-delay.delay`, then" condition. That way, the logic >> would be the following: >> >> * If the config option `restart-strategy` is configured, then Flink uses >> this `RestartStrategy` >> * If the config option `restart-strategy` is not configured, then >> ** If checkpointing is disabled, then choose `NoRestartStrategy` >> ** If checkpointing is enabled, then choose >> `FixedDelayRestartStrategy(Integer.MAX_VALUE, "0 s")` >> >> That way we retain the user friendliness that jobs restart if the user >> enabled checkpointing and we make it clear that any ` >> restart-strategy.fixed-delay.xyz` setting will only be respected if >> `restart-strategy` has been set to `fixed-delay`. >> >> This simplification would, however, change Flink's behaviour and might >> break existing setups. Since we introduced `RestartStrategies` with Flink >> 1.0.0 and deprecated the prior configuration mechanism which enables >> restarting if either the `attempts` or the `delay` has been set, I think >> that the number of broken jobs should be minimal if not non-existent. >> >> I'm sure that one can simplify the way RestartStrategies are >> programmatically configured as well but for the sake of simplicity/scoping >> I'd like to not touch it right away. >> >> What do you think about this behaviour change? >> >> [1] https://issues.apache.org/jira/browse/FLINK-13921 >> >> Cheers, >> Till >>
signature.asc
Description: OpenPGP digital signature