forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
/
StickyAllocationAndLocalRecoveryTestJob.java
483 lines (398 loc) · 16.7 KB
/
StickyAllocationAndLocalRecoveryTestJob.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
/*
* 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:https://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.tests;
import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.runtime.state.filesystem.FsStateBackend;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.environment.CheckpointConfig;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.util.Collector;
import org.apache.flink.util.Preconditions;
import org.apache.commons.lang3.RandomStringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
/**
* Automatic end-to-end test for local recovery (including sticky allocation).
*
* <p>List of possible input parameters for this job:
* <ul>
* <li>checkpointDir: the checkpoint directory, required.</li>
* <li>parallelism: the parallelism of the job, default 1.</li>
* <li>maxParallelism: the maximum parallelism of the job, default 1.</li>
* <li>checkpointInterval: the checkpointing interval in milliseconds, default 1000.</li>
* <li>restartDelay: the delay of the fixed delay restart strategy, default 0.</li>
* <li>externalizedCheckpoints: flag to activate externalized checkpoints, default <code>false</code>.</li>
* <li>stateBackend: choice for state backend between <code>file</code> and <code>rocks</code>, default <code>file</code>.</li>
* <li>killJvmOnFail: flag that determines whether or not an artificial failure induced by the test kills the JVM or not.</li>
* <li>asyncCheckpoints: flag for async checkpoints with file state backend, default <code>true</code>.</li>
* <li>incrementalCheckpoints: flag for incremental checkpoint with rocks state backend, default <code>false</code>.</li>
* <li>delay: sleep delay to throttle down the production of the source, default 0.</li>
* <li>maxAttempts: the maximum number of run attempts, before the job finishes with success, default 3.</li>
* <li>valueSize: size of the artificial value for each key in bytes, default 10.</li>
* </ul>
*/
public class StickyAllocationAndLocalRecoveryTestJob {
private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
public static void main(String[] args) throws Exception {
final ParameterTool pt = ParameterTool.fromArgs(args);
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(pt.getInt("parallelism", 1));
env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
if (pt.getBoolean("externalizedCheckpoints", false)) {
env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
}
String stateBackend = pt.get("stateBackend", "file");
String checkpointDir = pt.getRequired("checkpointDir");
boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
if ("file".equals(stateBackend)) {
boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", true);
env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
} else if ("rocks".equals(stateBackend)) {
boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
} else {
throw new IllegalArgumentException("Unknown backend: " + stateBackend);
}
// make parameters available in the web interface
env.getConfig().setGlobalJobParameters(pt);
// delay to throttle down the production of the source
long delay = pt.getLong("delay", 0L);
// the maximum number of attempts, before the job finishes with success
int maxAttempts = pt.getInt("maxAttempts", 3);
// size of one artificial value
int valueSize = pt.getInt("valueSize", 10);
env.addSource(new RandomLongSource(maxAttempts, delay))
.keyBy((KeySelector<Long, Long>) aLong -> aLong)
.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
.addSink(new PrintSinkFunction<>());
env.execute("Sticky Allocation And Local Recovery Test");
}
/**
* Source function that produces a long sequence.
*/
private static final class RandomLongSource extends RichParallelSourceFunction<Long> implements CheckpointedFunction {
private static final long serialVersionUID = 1L;
/**
* Generator delay between two events.
*/
final long delay;
/**
* Maximum restarts before shutting down this source.
*/
final int maxAttempts;
/**
* State that holds the current key for recovery.
*/
transient ListState<Long> sourceCurrentKeyState;
/**
* Generator's current key.
*/
long currentKey;
/**
* Generator runs while this is true.
*/
volatile boolean running;
RandomLongSource(int maxAttempts, long delay) {
this.delay = delay;
this.maxAttempts = maxAttempts;
this.running = true;
}
@Override
public void run(SourceContext<Long> sourceContext) throws Exception {
int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
// the source emits one final event and shuts down once we have reached max attempts.
if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
synchronized (sourceContext.getCheckpointLock()) {
sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
}
return;
}
while (running) {
synchronized (sourceContext.getCheckpointLock()) {
sourceContext.collect(currentKey);
currentKey += numberOfParallelSubtasks;
}
if (delay > 0) {
Thread.sleep(delay);
}
}
}
@Override
public void cancel() {
running = false;
}
@Override
public void snapshotState(FunctionSnapshotContext context) throws Exception {
sourceCurrentKeyState.clear();
sourceCurrentKeyState.add(currentKey);
}
@Override
public void initializeState(FunctionInitializationContext context) throws Exception {
ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
currentKey = getRuntimeContext().getIndexOfThisSubtask();
Iterable<Long> iterable = sourceCurrentKeyState.get();
if (iterable != null) {
Iterator<Long> iterator = iterable.iterator();
if (iterator.hasNext()) {
currentKey = iterator.next();
Preconditions.checkState(!iterator.hasNext());
}
}
}
}
/**
* Stateful map function. Failure creation and checks happen here.
*/
private static final class StateCreatingFlatMap
extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
private static final long serialVersionUID = 1L;
/**
* User configured size of the generated artificial values in the keyed state.
*/
final int valueSize;
/**
* Holds the user configuration if the artificial test failure is killing the JVM.
*/
final boolean killTaskOnFailure;
/**
* This state is used to create artificial keyed state in the backend.
*/
transient ValueState<String> valueState;
/**
* This state is used to persist the schedulingAndFailureInfo to state.
*/
transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
/**
* This contains the current scheduling and failure meta data.
*/
transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
/**
* Message to indicate that recovery detected a failure with sticky allocation.
*/
transient volatile String allocationFailureMessage;
/**
* If this flag is true, the next invocation of the map function introduces a test failure.
*/
transient volatile boolean failTask;
StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
this.valueSize = valueSize;
this.failTask = false;
this.killTaskOnFailure = killTaskOnFailure;
this.allocationFailureMessage = null;
}
@Override
public void flatMap(Long key, Collector<String> collector) throws IOException {
if (allocationFailureMessage != null) {
// Report the failure downstream, so that we can get the message from the output.
collector.collect(allocationFailureMessage);
allocationFailureMessage = null;
}
if (failTask) {
// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
if (killTaskOnFailure) {
Runtime.getRuntime().halt(-1);
} else {
throw new RuntimeException("Artificial user code exception.");
}
}
// sanity check
if (null != valueState.value()) {
throw new IllegalStateException("This should never happen, keys are generated monotonously.");
}
// store artificial data to blow up the state
valueState.update(RandomStringUtils.random(valueSize, true, true));
}
@Override
public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
}
@Override
public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
ValueStateDescriptor<String> stateDescriptor =
new ValueStateDescriptor<>("state", String.class);
valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
schedulingAndFailureState =
functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
String allocationID = runtimeContext.getAllocationIDAsString();
// Pattern of the name: "Flat Map -> Sink: Unnamed (4/4)#0". Remove "#0" part:
String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks().split("#")[0];
final int thisJvmPid = getJvmPid();
final Set<Integer> killedJvmPids = new HashSet<>();
// here we check if the sticky scheduling worked as expected
if (functionInitializationContext.isRestored()) {
Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
MapperSchedulingAndFailureInfo infoForThisTask = null;
List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
if (iterable != null) {
for (MapperSchedulingAndFailureInfo testInfo : iterable) {
completeInfo.add(testInfo);
if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
infoForThisTask = testInfo;
}
if (testInfo.killedJvm) {
killedJvmPids.add(testInfo.jvmPid);
}
}
}
Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
allocationFailureMessage = String.format(
"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
"Complete information from before the crash: %s.",
taskNameWithSubtasks,
runtimeContext.getAttemptNumber(),
infoForThisTask.allocationId,
infoForThisTask.jvmPid,
allocationID,
thisJvmPid,
completeInfo);
}
}
// We determine which of the subtasks will produce the artificial failure
boolean failingTask = shouldTaskFailForThisAttempt();
// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
failingTask,
failingTask && killTaskOnFailure,
thisJvmPid,
taskNameWithSubtasks,
allocationID);
schedulingAndFailureState.clear();
schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
}
@Override
public void notifyCheckpointComplete(long checkpointId) {
// we can only fail the task after at least one checkpoint is completed to record progress.
failTask = currentSchedulingAndFailureInfo.failingTask;
}
@Override
public void notifyCheckpointAborted(long checkpointId) {
}
private boolean shouldTaskFailForThisAttempt() {
RuntimeContext runtimeContext = getRuntimeContext();
int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
int attempt = runtimeContext.getAttemptNumber();
return (attempt % numSubtasks) == subtaskIdx;
}
private boolean isScheduledToCorrectAllocation(
MapperSchedulingAndFailureInfo infoForThisTask,
String allocationID,
Set<Integer> killedJvmPids) {
return (infoForThisTask.allocationId.equals(allocationID)
|| killedJvmPids.contains(infoForThisTask.jvmPid));
}
}
/**
* This code is copied from Stack Overflow.
*
* <p><a href="https://stackoverflow.com/questions/35842">https://stackoverflow.com/questions/35842</a>, answer
* <a href="https://stackoverflow.com/a/12066696/9193881">https://stackoverflow.com/a/12066696/9193881</a>
*
* <p>Author: <a href="https://stackoverflow.com/users/446591/brad-mace">Brad Mace</a>)
*/
private static int getJvmPid() throws Exception {
java.lang.management.RuntimeMXBean runtime =
java.lang.management.ManagementFactory.getRuntimeMXBean();
java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
jvm.setAccessible(true);
sun.management.VMManagement mgmt =
(sun.management.VMManagement) jvm.get(runtime);
java.lang.reflect.Method pidMethod =
mgmt.getClass().getDeclaredMethod("getProcessId");
pidMethod.setAccessible(true);
return (int) (Integer) pidMethod.invoke(mgmt);
}
/**
* Records the information required to check sticky scheduling after a restart.
*/
public static class MapperSchedulingAndFailureInfo implements Serializable {
private static final long serialVersionUID = 1L;
/**
* True iff this task inflicts a test failure.
*/
final boolean failingTask;
/**
* True iff this task kills its JVM.
*/
final boolean killedJvm;
/**
* PID of the task JVM.
*/
final int jvmPid;
/**
* Name and subtask index of the task.
*/
final String taskNameWithSubtask;
/**
* The current allocation id of this task.
*/
final String allocationId;
MapperSchedulingAndFailureInfo(
boolean failingTask,
boolean killedJvm,
int jvmPid,
String taskNameWithSubtask,
String allocationId) {
this.failingTask = failingTask;
this.killedJvm = killedJvm;
this.jvmPid = jvmPid;
this.taskNameWithSubtask = taskNameWithSubtask;
this.allocationId = allocationId;
}
@Override
public String toString() {
return "MapperTestInfo{" +
"failingTask=" + failingTask +
", killedJvm=" + killedJvm +
", jvmPid=" + jvmPid +
", taskNameWithSubtask='" + taskNameWithSubtask + '\'' +
", allocationId='" + allocationId + '\'' +
'}';
}
}
}