afedulov commented on code in PR #656:
URL: 
https://github.com/apache/flink-kubernetes-operator/pull/656#discussion_r1303469585


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java:
##########
@@ -321,7 +336,32 @@ public static String operatorConfigKey(String key) {
                                     + "but to complement them in special 
cases. For instance, a "
                                     + "full checkpoint might need to be 
occasionally triggered to "
                                     + "break the chain of incremental 
checkpoints and consolidate "
-                                    + "the partial incremental files.");
+                                    + "the partial incremental files. "
+                                    + "WARNING: not intended to be used 
together with the cron-based "
+                                    + "periodic checkpoint triggering");
+
+    @Documentation.Section(SECTION_DYNAMIC)
+    public static final ConfigOption<String> PERIODIC_CHECKPOINT_CRON =
+            operatorConfig("periodic.checkpoint.cron")
+                    .stringType()
+                    .defaultValue(
+                            "0 0 0 29 2 ? 1999") // Default value that is 
guaranteed not to trigger (not a leap year)
+                    .withDescription(
+                            "A cron expression that defines when to 
automatically trigger checkpoints. "
+                                    + "The precise triggering schedule is not 
guaranteed, checkpoints will "
+                                    + "be triggered as part of the regular 
reconcile loop. Periodic triggering "
+                                    + "with frequency higher than the 
reconciliation loop "
+                                    + "("
+                                    + OPERATOR_RECONCILE_INTERVAL.key()
+                                    + ") will lead to trigger omissions. "
+                                    + "NOTE: checkpoints are generally managed 
by Flink. This "
+                                    + "setting isn't meant to replace Flink's 
checkpoint settings, "
+                                    + "but to complement them in special 
cases. For instance, a "
+                                    + "full checkpoint might need to be 
occasionally triggered to "
+                                    + "break the chain of incremental 
checkpoints and consolidate "
+                                    + "the partial incremental files. "
+                                    + "WARNING: not intended to be used 
together with the interval-based "
+                                    + "periodic checkpoint triggering");

Review Comment:
   @mxm 
   > For the sake of simplifying this, could we unify both options under 
periodic.{checkpoint,savepoint}.interval? 
   
   I am not against it, but see the following downsides:
   - We won’t be able to tell if the user misspelled something in the duration 
format or her cron expression is invalid.  Since Quartz supports both 6- and 
7-positional expressions, it can get confusing sometimes when `?` or `*` are 
used in the "wrong" place. Some combinations are invalid, but we won't able to 
point at the error with the cron format, since there is a possibility that the 
user wanted to specify Duration to begin with, but made a mistake there.
   - The name would become a bit awkward:  
`periodic.{checkpoint|savepoint}.interval` , whereas cron does not necessarily 
represent an interval, but can also be a point-in-time setting.  This probably 
leads to the necessity to deprecate the already-released 
`periodic.savepoint.interval`
   - Although the absolutely valid intention is to make things simpler for the 
user, the complexity of explaining the "Eierlegende Milchwollsau" functionality 
of the single setting in the docs plus the unclear naming seem to work against 
this intention.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to