80
80
import java .util .ArrayList ;
81
81
import java .util .Collection ;
82
82
import java .util .concurrent .CompletableFuture ;
83
+ import java .util .concurrent .ExecutionException ;
83
84
import java .util .concurrent .ExecutorService ;
84
85
import java .util .concurrent .Executors ;
85
86
import java .util .concurrent .TimeUnit ;
92
93
* constructs the related components (network, I/O manager, memory manager, RPC service, HA service)
93
94
* and starts them.
94
95
*/
95
- public class TaskManagerRunner implements FatalErrorHandler , AutoCloseableAsync {
96
+ public class TaskManagerRunner implements FatalErrorHandler {
96
97
97
98
private static final Logger LOG = LoggerFactory .getLogger (TaskManagerRunner .class );
98
99
99
100
private static final long FATAL_ERROR_SHUTDOWN_TIMEOUT_MS = 10000L ;
100
101
101
- private static final int STARTUP_FAILURE_RETURN_CODE = 1 ;
102
-
103
- @ VisibleForTesting static final int RUNTIME_FAILURE_RETURN_CODE = 2 ;
102
+ private static final int SUCCESS_EXIT_CODE = 0 ;
103
+ @ VisibleForTesting static final int FAILURE_EXIT_CODE = 1 ;
104
104
105
105
private final Object lock = new Object ();
106
106
@@ -123,7 +123,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync
123
123
124
124
private final TaskExecutorService taskExecutorService ;
125
125
126
- private final CompletableFuture <Void > terminationFuture ;
126
+ private final CompletableFuture <Result > terminationFuture ;
127
127
128
128
private boolean shutdown ;
129
129
@@ -193,7 +193,8 @@ public TaskManagerRunner(
193
193
this .shutdown = false ;
194
194
handleUnexpectedTaskExecutorServiceTermination ();
195
195
196
- MemoryLogger .startIfConfigured (LOG , configuration , terminationFuture );
196
+ MemoryLogger .startIfConfigured (
197
+ LOG , configuration , terminationFuture .thenAccept (ignored -> {}));
197
198
}
198
199
199
200
private void handleUnexpectedTaskExecutorServiceTermination () {
@@ -220,8 +221,19 @@ public void start() throws Exception {
220
221
taskExecutorService .start ();
221
222
}
222
223
223
- @ Override
224
- public CompletableFuture <Void > closeAsync () {
224
+ public void close () throws Exception {
225
+ try {
226
+ closeAsync ().get ();
227
+ } catch (ExecutionException e ) {
228
+ ExceptionUtils .rethrowException (ExceptionUtils .stripExecutionException (e ));
229
+ }
230
+ }
231
+
232
+ public CompletableFuture <Result > closeAsync () {
233
+ return closeAsync (Result .SUCCESS );
234
+ }
235
+
236
+ private CompletableFuture <Result > closeAsync (Result terminationResult ) {
225
237
synchronized (lock ) {
226
238
if (!shutdown ) {
227
239
shutdown = true ;
@@ -238,7 +250,7 @@ public CompletableFuture<Void> closeAsync() {
238
250
if (throwable != null ) {
239
251
terminationFuture .completeExceptionally (throwable );
240
252
} else {
241
- terminationFuture .complete (null );
253
+ terminationFuture .complete (terminationResult );
242
254
}
243
255
});
244
256
}
@@ -291,7 +303,7 @@ private CompletableFuture<Void> shutDownServices() {
291
303
}
292
304
293
305
// export the termination future for caller to know it is terminated
294
- public CompletableFuture <Void > getTerminationFuture () {
306
+ public CompletableFuture <Result > getTerminationFuture () {
295
307
return terminationFuture ;
296
308
}
297
309
@@ -314,17 +326,15 @@ public void onFatalError(Throwable exception) {
314
326
&& !ExceptionUtils .isMetaspaceOutOfMemoryError (exception )) {
315
327
terminateJVM ();
316
328
} else {
317
- closeAsync ();
329
+ closeAsync (Result . FAILURE );
318
330
319
331
FutureUtils .orTimeout (
320
332
terminationFuture , FATAL_ERROR_SHUTDOWN_TIMEOUT_MS , TimeUnit .MILLISECONDS );
321
-
322
- terminationFuture .whenComplete ((Void ignored , Throwable throwable ) -> terminateJVM ());
323
333
}
324
334
}
325
335
326
336
private void terminateJVM () {
327
- System .exit (RUNTIME_FAILURE_RETURN_CODE );
337
+ System .exit (FAILURE_EXIT_CODE );
328
338
}
329
339
330
340
// --------------------------------------------------------------------------------------------
@@ -345,49 +355,78 @@ public static void main(String[] args) throws Exception {
345
355
LOG .info ("Cannot determine the maximum number of open file descriptors" );
346
356
}
347
357
348
- runTaskManagerSecurely (args );
358
+ runTaskManagerProcessSecurely (args );
349
359
}
350
360
351
361
public static Configuration loadConfiguration (String [] args ) throws FlinkParseException {
352
362
return ConfigurationParserUtils .loadCommonConfiguration (
353
363
args , TaskManagerRunner .class .getSimpleName ());
354
364
}
355
365
356
- public static void runTaskManager (Configuration configuration , PluginManager pluginManager )
366
+ public static int runTaskManager (Configuration configuration , PluginManager pluginManager )
357
367
throws Exception {
358
- final TaskManagerRunner taskManagerRunner =
359
- new TaskManagerRunner (
360
- configuration , pluginManager , TaskManagerRunner ::createTaskExecutorService );
368
+ final TaskManagerRunner taskManagerRunner ;
361
369
362
- taskManagerRunner .start ();
363
- }
370
+ try {
371
+ taskManagerRunner =
372
+ new TaskManagerRunner (
373
+ configuration ,
374
+ pluginManager ,
375
+ TaskManagerRunner ::createTaskExecutorService );
376
+ taskManagerRunner .start ();
377
+ } catch (Exception exception ) {
378
+ throw new FlinkException ("Failed to start the TaskManagerRunner." , exception );
379
+ }
364
380
365
- public static void runTaskManagerSecurely (String [] args ) {
366
381
try {
367
- Configuration configuration = loadConfiguration (args );
368
- runTaskManagerSecurely (configuration );
382
+ return taskManagerRunner .getTerminationFuture ().get ().getExitCode ();
369
383
} catch (Throwable t ) {
370
- final Throwable strippedThrowable =
371
- ExceptionUtils .stripException (t , UndeclaredThrowableException .class );
372
- LOG .error ("TaskManager initialization failed." , strippedThrowable );
373
- System .exit (STARTUP_FAILURE_RETURN_CODE );
384
+ throw new FlinkException (
385
+ "Unexpected failure during runtime of TaskManagerRunner." ,
386
+ ExceptionUtils .stripExecutionException (t ));
387
+ }
388
+ }
389
+
390
+ public static void runTaskManagerProcessSecurely (String [] args ) {
391
+ Configuration configuration = null ;
392
+
393
+ try {
394
+ configuration = loadConfiguration (args );
395
+ } catch (FlinkParseException fpe ) {
396
+ LOG .error ("Could not load the configuration." , fpe );
397
+ System .exit (FAILURE_EXIT_CODE );
374
398
}
399
+
400
+ runTaskManagerProcessSecurely (checkNotNull (configuration ));
375
401
}
376
402
377
- public static void runTaskManagerSecurely (Configuration configuration ) throws Exception {
403
+ public static void runTaskManagerProcessSecurely (Configuration configuration ) {
378
404
replaceGracefulExitWithHaltIfConfigured (configuration );
379
405
final PluginManager pluginManager =
380
406
PluginUtils .createPluginManagerFromRootFolder (configuration );
381
407
FileSystem .initialize (configuration , pluginManager );
382
408
383
- SecurityUtils .install (new SecurityConfiguration (configuration ));
409
+ int exitCode ;
410
+ Throwable throwable = null ;
384
411
385
- SecurityUtils .getInstalledContext ()
386
- .runSecured (
387
- () -> {
388
- runTaskManager (configuration , pluginManager );
389
- return null ;
390
- });
412
+ try {
413
+ SecurityUtils .install (new SecurityConfiguration (configuration ));
414
+
415
+ exitCode =
416
+ SecurityUtils .getInstalledContext ()
417
+ .runSecured (() -> runTaskManager (configuration , pluginManager ));
418
+ } catch (Throwable t ) {
419
+ throwable = ExceptionUtils .stripException (t , UndeclaredThrowableException .class );
420
+ exitCode = FAILURE_EXIT_CODE ;
421
+ }
422
+
423
+ if (throwable != null ) {
424
+ LOG .error ("Terminating TaskManagerRunner with exit code {}." , exitCode , throwable );
425
+ } else {
426
+ LOG .info ("Terminating TaskManagerRunner with exit code {}." , exitCode );
427
+ }
428
+
429
+ System .exit (exitCode );
391
430
}
392
431
393
432
// --------------------------------------------------------------------------------------------
@@ -612,4 +651,19 @@ public interface TaskExecutorService extends AutoCloseableAsync {
612
651
613
652
CompletableFuture <Void > getTerminationFuture ();
614
653
}
654
+
655
+ public enum Result {
656
+ SUCCESS (SUCCESS_EXIT_CODE ),
657
+ FAILURE (FAILURE_EXIT_CODE );
658
+
659
+ private final int exitCode ;
660
+
661
+ Result (int exitCode ) {
662
+ this .exitCode = exitCode ;
663
+ }
664
+
665
+ public int getExitCode () {
666
+ return exitCode ;
667
+ }
668
+ }
615
669
}
0 commit comments