19
19
package org .apache .flink .runtime .checkpoint ;
20
20
21
21
import org .apache .flink .api .common .JobStatus ;
22
- import org .apache .flink .api .java .tuple .Tuple2 ;
23
22
import org .apache .flink .runtime .persistence .PossibleInconsistentStateException ;
24
23
import org .apache .flink .runtime .persistence .ResourceVersion ;
25
24
import org .apache .flink .runtime .persistence .StateHandleStore ;
26
- import org .apache .flink .runtime .state .RetrievableStateHandle ;
27
- import org .apache .flink .util .FlinkException ;
25
+ import org .apache .flink .util .Preconditions ;
28
26
29
27
import org .slf4j .Logger ;
30
28
import org .slf4j .LoggerFactory ;
31
29
32
- import java .io .IOException ;
33
30
import java .util .ArrayDeque ;
34
31
import java .util .ArrayList ;
35
- import java .util .Comparator ;
32
+ import java .util .Collection ;
36
33
import java .util .List ;
37
- import java .util .Set ;
38
34
import java .util .concurrent .Executor ;
39
- import java .util .stream . Collectors ;
35
+ import java .util .concurrent . atomic . AtomicBoolean ;
40
36
41
37
import static org .apache .flink .util .Preconditions .checkArgument ;
42
38
import static org .apache .flink .util .Preconditions .checkNotNull ;
@@ -61,9 +57,6 @@ public class DefaultCompletedCheckpointStore<R extends ResourceVersion<R>>
61
57
private static final Logger LOG =
62
58
LoggerFactory .getLogger (DefaultCompletedCheckpointStore .class );
63
59
64
- private static final Comparator <Tuple2 <RetrievableStateHandle <CompletedCheckpoint >, String >>
65
- STRING_COMPARATOR = Comparator .comparing (o -> o .f1 );
66
-
67
60
/** Completed checkpoints state handle store. */
68
61
private final StateHandleStore <CompletedCheckpoint , R > checkpointStateHandleStore ;
69
62
@@ -81,6 +74,9 @@ public class DefaultCompletedCheckpointStore<R extends ResourceVersion<R>>
81
74
82
75
private final CheckpointStoreUtil completedCheckpointStoreUtil ;
83
76
77
+ /** False if store has been shutdown. */
78
+ private final AtomicBoolean running = new AtomicBoolean (true );
79
+
84
80
/**
85
81
* Creates a {@link DefaultCompletedCheckpointStore} instance.
86
82
*
@@ -95,18 +91,14 @@ public DefaultCompletedCheckpointStore(
95
91
int maxNumberOfCheckpointsToRetain ,
96
92
StateHandleStore <CompletedCheckpoint , R > stateHandleStore ,
97
93
CheckpointStoreUtil completedCheckpointStoreUtil ,
94
+ Collection <CompletedCheckpoint > completedCheckpoints ,
98
95
Executor executor ) {
99
-
100
96
checkArgument (maxNumberOfCheckpointsToRetain >= 1 , "Must retain at least one checkpoint." );
101
-
102
97
this .maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain ;
103
-
104
98
this .checkpointStateHandleStore = checkNotNull (stateHandleStore );
105
-
106
99
this .completedCheckpoints = new ArrayDeque <>(maxNumberOfCheckpointsToRetain + 1 );
107
-
100
+ this . completedCheckpoints . addAll ( completedCheckpoints );
108
101
this .ioExecutor = checkNotNull (executor );
109
-
110
102
this .completedCheckpointStoreUtil = checkNotNull (completedCheckpointStoreUtil );
111
103
}
112
104
@@ -115,55 +107,13 @@ public boolean requiresExternalizedCheckpoints() {
115
107
return true ;
116
108
}
117
109
118
- /**
119
- * Recover all the valid checkpoints from state handle store. All the successfully recovered
120
- * checkpoints will be added to {@link #completedCheckpoints} sorted by checkpoint id.
121
- */
122
- @ Override
123
- public void recover () throws Exception {
124
- LOG .info ("Recovering checkpoints from {}." , checkpointStateHandleStore );
125
-
126
- // Get all there is first
127
- final List <Tuple2 <RetrievableStateHandle <CompletedCheckpoint >, String >> initialCheckpoints =
128
- checkpointStateHandleStore .getAllAndLock ();
129
-
130
- initialCheckpoints .sort (STRING_COMPARATOR );
131
-
132
- final int numberOfInitialCheckpoints = initialCheckpoints .size ();
133
-
134
- LOG .info (
135
- "Found {} checkpoints in {}." ,
136
- numberOfInitialCheckpoints ,
137
- checkpointStateHandleStore );
138
- if (haveAllDownloaded (initialCheckpoints )) {
139
- LOG .info (
140
- "All {} checkpoints found are already downloaded." , numberOfInitialCheckpoints );
141
- return ;
142
- }
143
-
144
- final List <CompletedCheckpoint > retrievedCheckpoints =
145
- new ArrayList <>(numberOfInitialCheckpoints );
146
- LOG .info ("Trying to fetch {} checkpoints from storage." , numberOfInitialCheckpoints );
147
-
148
- for (Tuple2 <RetrievableStateHandle <CompletedCheckpoint >, String > checkpointStateHandle :
149
- initialCheckpoints ) {
150
- retrievedCheckpoints .add (
151
- checkNotNull (retrieveCompletedCheckpoint (checkpointStateHandle )));
152
- }
153
-
154
- // Clear local handles in order to prevent duplicates on recovery. The local handles should
155
- // reflect the state handle store contents.
156
- completedCheckpoints .clear ();
157
- completedCheckpoints .addAll (retrievedCheckpoints );
158
- }
159
-
160
110
/**
161
111
* Synchronously writes the new checkpoints to state handle store and asynchronously removes
162
112
* older ones.
163
113
*
164
114
* @param checkpoint Completed checkpoint to add.
165
115
* @throws PossibleInconsistentStateException if adding the checkpoint failed and leaving the
166
- * system in an possibly inconsistent state, i.e. it's uncertain whether the checkpoint
116
+ * system in a possibly inconsistent state, i.e. it's uncertain whether the checkpoint
167
117
* metadata was fully written to the underlying systems or not.
168
118
*/
169
119
@ Override
@@ -172,13 +122,13 @@ public void addCheckpoint(
172
122
CheckpointsCleaner checkpointsCleaner ,
173
123
Runnable postCleanup )
174
124
throws Exception {
175
-
125
+ Preconditions . checkState ( running . get (), "Checkpoint store has already been shutdown." );
176
126
checkNotNull (checkpoint , "Checkpoint" );
177
127
178
128
final String path =
179
129
completedCheckpointStoreUtil .checkpointIDToName (checkpoint .getCheckpointID ());
180
130
181
- // Now add the new one. If it fails, we don't want to loose existing data.
131
+ // Now add the new one. If it fails, we don't want to lose existing data.
182
132
checkpointStateHandleStore .addAndLock (path , checkpoint );
183
133
184
134
completedCheckpoints .addLast (checkpoint );
@@ -214,30 +164,28 @@ public int getMaxNumberOfRetainedCheckpoints() {
214
164
@ Override
215
165
public void shutdown (JobStatus jobStatus , CheckpointsCleaner checkpointsCleaner )
216
166
throws Exception {
217
- if (jobStatus .isGloballyTerminalState ()) {
218
- LOG .info ("Shutting down" );
219
-
220
- for (CompletedCheckpoint checkpoint : completedCheckpoints ) {
221
- try {
222
- tryRemoveCompletedCheckpoint (
223
- checkpoint ,
224
- checkpoint .shouldBeDiscardedOnShutdown (jobStatus ),
225
- checkpointsCleaner ,
226
- () -> {});
227
- } catch (Exception e ) {
228
- LOG .warn ("Fail to remove checkpoint during shutdown." , e );
167
+ if (running .compareAndSet (true , false )) {
168
+ if (jobStatus .isGloballyTerminalState ()) {
169
+ LOG .info ("Shutting down" );
170
+ for (CompletedCheckpoint checkpoint : completedCheckpoints ) {
171
+ try {
172
+ tryRemoveCompletedCheckpoint (
173
+ checkpoint ,
174
+ checkpoint .shouldBeDiscardedOnShutdown (jobStatus ),
175
+ checkpointsCleaner ,
176
+ () -> {});
177
+ } catch (Exception e ) {
178
+ LOG .warn ("Fail to remove checkpoint during shutdown." , e );
179
+ }
229
180
}
181
+ completedCheckpoints .clear ();
182
+ checkpointStateHandleStore .clearEntries ();
183
+ } else {
184
+ LOG .info ("Suspending" );
185
+ // Clear the local handles, but don't remove any state
186
+ completedCheckpoints .clear ();
187
+ checkpointStateHandleStore .releaseAll ();
230
188
}
231
-
232
- completedCheckpoints .clear ();
233
- checkpointStateHandleStore .clearEntries ();
234
- } else {
235
- LOG .info ("Suspending" );
236
-
237
- // Clear the local handles, but don't remove any state
238
- completedCheckpoints .clear ();
239
-
240
- checkpointStateHandleStore .releaseAll ();
241
189
}
242
190
}
243
191
@@ -257,25 +205,6 @@ private void tryRemoveCompletedCheckpoint(
257
205
}
258
206
}
259
207
260
- private boolean haveAllDownloaded (
261
- List <Tuple2 <RetrievableStateHandle <CompletedCheckpoint >, String >> checkpointPointers ) {
262
- if (completedCheckpoints .size () != checkpointPointers .size ()) {
263
- return false ;
264
- }
265
- Set <Long > localIds =
266
- completedCheckpoints .stream ()
267
- .map (CompletedCheckpoint ::getCheckpointID )
268
- .collect (Collectors .toSet ());
269
- for (Tuple2 <RetrievableStateHandle <CompletedCheckpoint >, String > initialCheckpoint :
270
- checkpointPointers ) {
271
- if (!localIds .contains (
272
- completedCheckpointStoreUtil .nameToCheckpointID (initialCheckpoint .f1 ))) {
273
- return false ;
274
- }
275
- }
276
- return true ;
277
- }
278
-
279
208
/**
280
209
* Tries to remove the checkpoint identified by the given checkpoint id.
281
210
*
@@ -286,34 +215,4 @@ private boolean tryRemove(long checkpointId) throws Exception {
286
215
return checkpointStateHandleStore .releaseAndTryRemove (
287
216
completedCheckpointStoreUtil .checkpointIDToName (checkpointId ));
288
217
}
289
-
290
- private CompletedCheckpoint retrieveCompletedCheckpoint (
291
- Tuple2 <RetrievableStateHandle <CompletedCheckpoint >, String > stateHandle )
292
- throws FlinkException {
293
- long checkpointId = completedCheckpointStoreUtil .nameToCheckpointID (stateHandle .f1 );
294
-
295
- LOG .info ("Trying to retrieve checkpoint {}." , checkpointId );
296
-
297
- try {
298
- return stateHandle .f0 .retrieveState ();
299
- } catch (ClassNotFoundException cnfe ) {
300
- throw new FlinkException (
301
- "Could not retrieve checkpoint "
302
- + checkpointId
303
- + " from state handle under "
304
- + stateHandle .f1
305
- + ". This indicates that you are trying to recover from state written by an "
306
- + "older Flink version which is not compatible. Try cleaning the state handle store." ,
307
- cnfe );
308
- } catch (IOException ioe ) {
309
- throw new FlinkException (
310
- "Could not retrieve checkpoint "
311
- + checkpointId
312
- + " from state handle under "
313
- + stateHandle .f1
314
- + ". This indicates that the retrieved state handle is broken. Try cleaning the "
315
- + "state handle store." ,
316
- ioe );
317
- }
318
- }
319
218
}
0 commit comments