29
29
import org .apache .flink .configuration .ReadableConfig ;
30
30
import org .apache .flink .configuration .StateRecoveryOptions ;
31
31
import org .apache .flink .configuration .description .InlineElement ;
32
+ import org .apache .flink .core .execution .CheckpointingMode ;
32
33
import org .apache .flink .core .fs .Path ;
33
34
import org .apache .flink .runtime .jobgraph .tasks .CheckpointCoordinatorConfiguration ;
34
35
import org .apache .flink .runtime .state .CheckpointStorage ;
35
36
import org .apache .flink .runtime .state .StateBackend ;
36
37
import org .apache .flink .runtime .state .storage .FileSystemCheckpointStorage ;
37
- import org .apache .flink .streaming .api .CheckpointingMode ;
38
38
import org .apache .flink .util .Preconditions ;
39
39
40
40
import org .slf4j .Logger ;
@@ -72,9 +72,9 @@ public class CheckpointConfig implements java.io.Serializable {
72
72
* The default checkpoint mode: exactly once.
73
73
*
74
74
* @deprecated This field is no longer used. Please use {@link
75
- * ExecutionCheckpointingOptions.CHECKPOINTING_MODE } instead.
75
+ * ExecutionCheckpointingOptions.CHECKPOINTING_CONSISTENCY_MODE } instead.
76
76
*/
77
- public static final CheckpointingMode DEFAULT_MODE =
77
+ public static final org . apache . flink . streaming . api . CheckpointingMode DEFAULT_MODE =
78
78
ExecutionCheckpointingOptions .CHECKPOINTING_MODE .defaultValue ();
79
79
80
80
/**
@@ -177,41 +177,42 @@ public boolean isCheckpointingEnabled() {
177
177
* Gets the checkpointing mode (exactly-once vs. at-least-once).
178
178
*
179
179
* @return The checkpointing mode.
180
- * @deprecated Use {@link #getCheckpointMode } instead.
180
+ * @deprecated Use {@link #getConsistencyMode } instead.
181
181
*/
182
182
@ Deprecated
183
- public CheckpointingMode getCheckpointingMode () {
183
+ public org . apache . flink . streaming . api . CheckpointingMode getCheckpointingMode () {
184
184
return configuration .get (ExecutionCheckpointingOptions .CHECKPOINTING_MODE );
185
185
}
186
186
187
187
/**
188
188
* Sets the checkpointing mode (exactly-once vs. at-least-once).
189
189
*
190
190
* @param checkpointingMode The checkpointing mode.
191
- * @deprecated Use {@link #setCheckpointMode } instead.
191
+ * @deprecated Use {@link #setConsistencyMode } instead.
192
192
*/
193
193
@ Deprecated
194
- public void setCheckpointingMode (CheckpointingMode checkpointingMode ) {
194
+ public void setCheckpointingMode (
195
+ org .apache .flink .streaming .api .CheckpointingMode checkpointingMode ) {
195
196
configuration .set (ExecutionCheckpointingOptions .CHECKPOINTING_MODE , checkpointingMode );
196
197
}
197
198
198
199
/**
199
- * Gets the checkpointing mode (exactly-once vs. at-least-once).
200
+ * Gets the checkpointing consistency mode (exactly-once vs. at-least-once).
200
201
*
201
202
* @return The checkpointing mode.
202
203
*/
203
- public org . apache . flink . core . execution . CheckpointingMode getCheckpointMode () {
204
- return configuration .get (ExecutionCheckpointingOptions .CHECKPOINTING_MODE_V2 );
204
+ public CheckpointingMode getConsistencyMode () {
205
+ return configuration .get (ExecutionCheckpointingOptions .CHECKPOINTING_CONSISTENCY_MODE );
205
206
}
206
207
207
208
/**
208
- * Sets the checkpointing mode (exactly-once vs. at-least-once).
209
+ * Sets the checkpointing consistency mode (exactly-once vs. at-least-once).
209
210
*
210
211
* @param checkpointingMode The checkpointing mode.
211
212
*/
212
- public void setCheckpointMode (
213
- org . apache . flink . core . execution . CheckpointingMode checkpointingMode ) {
214
- configuration . set ( ExecutionCheckpointingOptions .CHECKPOINTING_MODE_V2 , checkpointingMode );
213
+ public void setConsistencyMode ( CheckpointingMode checkpointingMode ) {
214
+ configuration . set (
215
+ ExecutionCheckpointingOptions .CHECKPOINTING_CONSISTENCY_MODE , checkpointingMode );
215
216
}
216
217
217
218
/**
@@ -598,7 +599,8 @@ public boolean isExternalizedCheckpointsEnabled() {
598
599
* embedded into the stream of data anymore.
599
600
*
600
601
* <p>Unaligned checkpoints can only be enabled if {@link
601
- * ExecutionCheckpointingOptions#CHECKPOINTING_MODE} is {@link CheckpointingMode#EXACTLY_ONCE}.
602
+ * ExecutionCheckpointingOptions#CHECKPOINTING_CONSISTENCY_MODE} is {@link
603
+ * CheckpointingMode#EXACTLY_ONCE}.
602
604
*
603
605
* @param enabled Flag to indicate whether unaligned are enabled.
604
606
*/
@@ -616,7 +618,8 @@ public void enableUnalignedCheckpoints(boolean enabled) {
616
618
* embedded into the stream of data anymore.
617
619
*
618
620
* <p>Unaligned checkpoints can only be enabled if {@link
619
- * ExecutionCheckpointingOptions#CHECKPOINTING_MODE} is {@link CheckpointingMode#EXACTLY_ONCE}.
621
+ * ExecutionCheckpointingOptions#CHECKPOINTING_CONSISTENCY_MODE} is {@link
622
+ * CheckpointingMode#EXACTLY_ONCE}.
620
623
*/
621
624
@ PublicEvolving
622
625
public void enableUnalignedCheckpoints () {
@@ -986,7 +989,7 @@ public InlineElement getDescription() {
986
989
987
990
/**
988
991
* Sets all relevant options contained in the {@link ReadableConfig} such as e.g. {@link
989
- * ExecutionCheckpointingOptions#CHECKPOINTING_MODE }.
992
+ * ExecutionCheckpointingOptions#CHECKPOINTING_CONSISTENCY_MODE }.
990
993
*
991
994
* <p>It will change the value of a setting only if a corresponding option was set in the {@code
992
995
* configuration}. If a key is not present, the current value of a field will remain untouched.
@@ -995,8 +998,8 @@ public InlineElement getDescription() {
995
998
*/
996
999
public void configure (ReadableConfig configuration ) {
997
1000
configuration
998
- .getOptional (ExecutionCheckpointingOptions .CHECKPOINTING_MODE )
999
- .ifPresent (this ::setCheckpointingMode );
1001
+ .getOptional (ExecutionCheckpointingOptions .CHECKPOINTING_CONSISTENCY_MODE )
1002
+ .ifPresent (this ::setConsistencyMode );
1000
1003
configuration
1001
1004
.getOptional (ExecutionCheckpointingOptions .CHECKPOINTING_INTERVAL )
1002
1005
.ifPresent (i -> this .setCheckpointInterval (i .toMillis ()));
0 commit comments