aljoscha commented on a change in pull request #10222: [FLINK-14788][configuration] Add configure method to CheckpointConfig URL: https://github.com/apache/flink/pull/10222#discussion_r347901695
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java ########## @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.environment; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.configuration.description.TextElement; +import org.apache.flink.streaming.api.CheckpointingMode; + +import java.time.Duration; + +/** + * Execution {@link ConfigOption} for configuring checkpointing related parameters. + * + * @see CheckpointConfig + */ +@PublicEvolving +public class ExecutionCheckpointingOptions { + public static final ConfigOption<CheckpointingMode> CHECKPOINTING_MODE = + ConfigOptions.key("execution.checkpointing.mode") + .enumType(CheckpointingMode.class) + .defaultValue(CheckpointingMode.EXACTLY_ONCE) + .withDescription("The checkpointing mode (exactly-once vs. at-least-once)."); + + public static final ConfigOption<Duration> CHECKPOINTING_TIMEOUT = + ConfigOptions.key("execution.checkpointing.timeout") + .durationType() + .defaultValue(Duration.ofMinutes(10)) + .withDescription("The maximum time that a checkpoint may take before being discarded."); + + public static final ConfigOption<Integer> MAX_CONCURRENT_CHECKPOINTS = + ConfigOptions.key("execution.checkpointing.max-concurrent-checkpoints") + .intType() + .defaultValue(1) + .withDescription("The maximum number of checkpoint attempts that may be in progress at the same time. If " + + "this value is n, then no checkpoints will be triggered while n checkpoint attempts are currently in " + + "flight. For the next checkpoint to be triggered, one checkpoint attempt would need to finish or " + + "expire."); + + public static final ConfigOption<Duration> MIN_PAUSE_BETWEEN_CHECKPOINTS = + ConfigOptions.key("execution.checkpointing.min-pause") + .durationType() + .defaultValue(Duration.ZERO) + .withDescription(Description.builder() + .text("The minimal pause between checkpointing attempts. This setting defines how soon the" + + "checkpoint coordinator may trigger another checkpoint after it becomes possible to trigger" + + "another checkpoint with respect to the maximum number of concurrent checkpoints" + + "(see %s).", TextElement.code(MAX_CONCURRENT_CHECKPOINTS.key())) + .linebreak() + .linebreak() + .text("If the maximum number of concurrent checkpoints is set to one, this setting makes effectively " + + "sure that a minimum amount of time passes where no checkpoint is in progress at all.") + .build()); + + public static final ConfigOption<Boolean> PREFER_CHECKPOINT_FOR_RECOVERY = + ConfigOptions.key("execution.checkpointing.prefer-checkpoint-for-recovery") + .booleanType() + .defaultValue(false) + .withDescription("If enabled, a job recovery should fallback to checkpoint when there is a more recent " + + "savepoint."); + + public static final ConfigOption<Integer> TOLERABLE_FAILURE_NUMBER = + ConfigOptions.key("execution.checkpointing.tolerable-failed-checkpoints") + .intType() + .noDefaultValue() + .withDescription("The tolerable checkpoint failure number. If set to 0, that means" + + "we do not tolerance any checkpoint failure."); + + public static final ConfigOption<CheckpointConfig.ExternalizedCheckpointCleanup> EXTERNALIZED_CHECKPOINT = + ConfigOptions.key("execution.checkpointing.externalized-checkpoint") Review comment: Seeing this now in code I think the option is slightly misnamed. It should be something like `externalized-checkpoint-cleanup` or `externalized-checkpoint-retention`. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
