-
Notifications
You must be signed in to change notification settings - Fork 13k
/
Utils.java
822 lines (727 loc) · 35.5 KB
/
Utils.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
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
/*
* 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.yarn;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigUtils;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
import org.apache.flink.runtime.util.HadoopUtils;
import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils;
import org.apache.flink.util.StringUtils;
import org.apache.flink.util.function.FunctionWithException;
import org.apache.flink.yarn.configuration.YarnConfigOptions;
import org.apache.flink.yarn.configuration.YarnResourceManagerDriverConfiguration;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.util.Records;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.yarn.YarnConfigKeys.ENV_FLINK_CLASSPATH;
import static org.apache.flink.yarn.YarnConfigKeys.LOCAL_RESOURCE_DESCRIPTOR_SEPARATOR;
import static org.apache.flink.yarn.configuration.YarnConfigOptions.YARN_CONTAINER_START_COMMAND_TEMPLATE;
/** Utility class that provides helper methods to work with Apache Hadoop YARN. */
public final class Utils {
private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
/** KRB5 file name populated in YARN container for secure IT run. */
public static final String KRB5_FILE_NAME = "krb5.conf";
/** Yarn site xml file name populated in YARN container for secure IT run. */
public static final String YARN_SITE_FILE_NAME = "yarn-site.xml";
/** Constant representing a wildcard access control list. */
private static final String WILDCARD_ACL = "*";
/** The prefixes that Flink adds to the YARN config. */
private static final String[] FLINK_CONFIG_PREFIXES = {"flink.yarn."};
@VisibleForTesting
static final String YARN_RM_FAIR_SCHEDULER_CLAZZ =
"org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler";
@VisibleForTesting
static final String YARN_RM_SLS_FAIR_SCHEDULER_CLAZZ =
"org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler";
@VisibleForTesting
static final String YARN_RM_INCREMENT_ALLOCATION_MB_KEY =
"yarn.resource-types.memory-mb.increment-allocation";
@VisibleForTesting
static final String YARN_RM_INCREMENT_ALLOCATION_MB_LEGACY_KEY =
"yarn.scheduler.increment-allocation-mb";
private static final int DEFAULT_YARN_RM_INCREMENT_ALLOCATION_MB = 1024;
@VisibleForTesting
static final String YARN_RM_INCREMENT_ALLOCATION_VCORES_KEY =
"yarn.resource-types.vcores.increment-allocation";
@VisibleForTesting
static final String YARN_RM_INCREMENT_ALLOCATION_VCORES_LEGACY_KEY =
"yarn.scheduler.increment-allocation-vcores";
@VisibleForTesting
static final String IGNORE_UNRECOGNIZED_VM_OPTIONS = "-XX:+IgnoreUnrecognizedVMOptions";
private static final int DEFAULT_YARN_RM_INCREMENT_ALLOCATION_VCORES = 1;
public static void setupYarnClassPath(Configuration conf, Map<String, String> appMasterEnv) {
addToEnvironment(
appMasterEnv, Environment.CLASSPATH.name(), appMasterEnv.get(ENV_FLINK_CLASSPATH));
String[] applicationClassPathEntries =
conf.getStrings(
YarnConfiguration.YARN_APPLICATION_CLASSPATH,
YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH);
for (String c : applicationClassPathEntries) {
addToEnvironment(appMasterEnv, Environment.CLASSPATH.name(), c.trim());
}
}
/**
* Deletes the YARN application files, e.g., Flink binaries, libraries, etc., from the remote
* filesystem.
*
* @param applicationFilesDir The application files directory.
*/
public static void deleteApplicationFiles(final String applicationFilesDir) {
if (!StringUtils.isNullOrWhitespaceOnly(applicationFilesDir)) {
final org.apache.flink.core.fs.Path path =
new org.apache.flink.core.fs.Path(applicationFilesDir);
try {
final org.apache.flink.core.fs.FileSystem fileSystem = path.getFileSystem();
if (!fileSystem.delete(path, true)) {
LOG.error(
"Deleting yarn application files under {} was unsuccessful.",
applicationFilesDir);
}
} catch (final IOException e) {
LOG.error(
"Could not properly delete yarn application files directory {}.",
applicationFilesDir,
e);
}
} else {
LOG.debug(
"No yarn application files directory set. Therefore, cannot clean up the data.");
}
}
/**
* Creates a YARN resource for the remote object at the given location.
*
* @param remoteRsrcPath remote location of the resource
* @param resourceSize size of the resource
* @param resourceModificationTime last modification time of the resource
* @return YARN resource
*/
static LocalResource registerLocalResource(
Path remoteRsrcPath,
long resourceSize,
long resourceModificationTime,
LocalResourceVisibility resourceVisibility,
LocalResourceType resourceType) {
LocalResource localResource = Records.newRecord(LocalResource.class);
localResource.setResource(URL.fromURI(remoteRsrcPath.toUri()));
localResource.setSize(resourceSize);
localResource.setTimestamp(resourceModificationTime);
localResource.setType(resourceType);
localResource.setVisibility(resourceVisibility);
return localResource;
}
/**
* Creates a YARN resource for the remote object at the given location.
*
* @param fs remote filesystem
* @param remoteRsrcPath resource path to be registered
* @return YARN resource
*/
private static LocalResource registerLocalResource(
FileSystem fs, Path remoteRsrcPath, LocalResourceType resourceType) throws IOException {
FileStatus jarStat = fs.getFileStatus(remoteRsrcPath);
return registerLocalResource(
remoteRsrcPath,
jarStat.getLen(),
jarStat.getModificationTime(),
LocalResourceVisibility.APPLICATION,
resourceType);
}
/**
* Copied method from org.apache.hadoop.yarn.util.Apps. It was broken by YARN-1824 (2.4.0) and
* fixed for 2.4.1 by https://issues.apache.org/jira/browse/YARN-1931
*/
public static void addToEnvironment(
Map<String, String> environment, String variable, String value) {
String val = environment.get(variable);
if (val == null) {
val = value;
} else {
val = val + File.pathSeparator + value;
}
environment.put(StringInterner.weakIntern(variable), StringInterner.weakIntern(val));
}
/**
* Resolve keytab path either as absolute path or relative to working directory.
*
* @param workingDir current working directory
* @param keytabPath configured keytab path.
* @return resolved keytab path, or null if not found.
*/
public static String resolveKeytabPath(String workingDir, String keytabPath) {
String keytab = null;
if (keytabPath != null) {
File f;
f = new File(keytabPath);
if (f.exists()) {
keytab = f.getAbsolutePath();
LOG.info("Resolved keytab path: {}", keytab);
} else {
// try using relative paths, this is the case when the keytab was shipped
// as a local resource
f = new File(workingDir, keytabPath);
if (f.exists()) {
keytab = f.getAbsolutePath();
LOG.info("Resolved keytab path: {}", keytab);
} else {
LOG.warn("Could not resolve keytab path with: {}", keytabPath);
keytab = null;
}
}
}
return keytab;
}
/** Private constructor to prevent instantiation. */
private Utils() {
throw new RuntimeException();
}
/**
* Creates the launch context, which describes how to bring up a TaskExecutor / TaskManager
* process in an allocated YARN container.
*
* <p>This code is extremely YARN specific and registers all the resources that the TaskExecutor
* needs (such as JAR file, config file, ...) and all environment variables in a YARN container
* launch context. The launch context then ensures that those resources will be copied into the
* containers transient working directory.
*
* @param flinkConfig The Flink configuration object.
* @param yarnConfig The YARN configuration object.
* @param configuration The YarnResourceManagerDriver configurations.
* @param tmParams The TaskExecutor container memory parameters.
* @param taskManagerDynamicProperties The dynamic configurations to be updated for the
* TaskExecutors based on client uploaded Flink config.
* @param workingDirectory The current application master container's working directory.
* @param taskManagerMainClass The class with the main method.
* @param log The logger.
* @return The launch context for the TaskManager processes.
* @throws Exception Thrown if the launch context could not be created, for example if the
* resources could not be copied.
*/
static ContainerLaunchContext createTaskExecutorContext(
org.apache.flink.configuration.Configuration flinkConfig,
YarnConfiguration yarnConfig,
YarnResourceManagerDriverConfiguration configuration,
ContaineredTaskManagerParameters tmParams,
String taskManagerDynamicProperties,
String workingDirectory,
Class<?> taskManagerMainClass,
Logger log)
throws Exception {
// get and validate all relevant variables
String remoteFlinkJarPath =
checkNotNull(
configuration.getFlinkDistJar(),
"Environment variable %s not set",
YarnConfigKeys.FLINK_DIST_JAR);
String shipListString =
checkNotNull(
configuration.getClientShipFiles(),
"Environment variable %s not set",
YarnConfigKeys.ENV_CLIENT_SHIP_FILES);
final String remoteKeytabPath = configuration.getRemoteKeytabPath();
final String localKeytabPath = configuration.getLocalKeytabPath();
final String keytabPrincipal = configuration.getKeytabPrinciple();
final String remoteYarnConfPath = configuration.getYarnSiteXMLPath();
final String remoteKrb5Path = configuration.getKrb5Path();
if (log.isDebugEnabled()) {
log.debug("TM:remote keytab path obtained {}", remoteKeytabPath);
log.debug("TM:local keytab path obtained {}", localKeytabPath);
log.debug("TM:keytab principal obtained {}", keytabPrincipal);
log.debug("TM:remote yarn conf path obtained {}", remoteYarnConfPath);
log.debug("TM:remote krb5 path obtained {}", remoteKrb5Path);
}
String classPathString =
checkNotNull(
configuration.getFlinkClasspath(),
"Environment variable %s not set",
YarnConfigKeys.ENV_FLINK_CLASSPATH);
// register keytab
LocalResource keytabResource = null;
if (remoteKeytabPath != null) {
log.info(
"TM:Adding keytab {} to the container local resource bucket", remoteKeytabPath);
Path keytabPath = new Path(remoteKeytabPath);
FileSystem fs = keytabPath.getFileSystem(yarnConfig);
keytabResource = registerLocalResource(fs, keytabPath, LocalResourceType.FILE);
}
// To support Yarn Secure Integration Test Scenario
LocalResource yarnConfResource = null;
if (remoteYarnConfPath != null) {
log.info(
"TM:Adding remoteYarnConfPath {} to the container local resource bucket",
remoteYarnConfPath);
Path yarnConfPath = new Path(remoteYarnConfPath);
FileSystem fs = yarnConfPath.getFileSystem(yarnConfig);
yarnConfResource = registerLocalResource(fs, yarnConfPath, LocalResourceType.FILE);
}
// register krb5.conf
LocalResource krb5ConfResource = null;
boolean hasKrb5 = false;
if (remoteKrb5Path != null) {
log.info(
"Adding remoteKrb5Path {} to the container local resource bucket",
remoteKrb5Path);
Path krb5ConfPath = new Path(remoteKrb5Path);
FileSystem fs = krb5ConfPath.getFileSystem(yarnConfig);
krb5ConfResource = registerLocalResource(fs, krb5ConfPath, LocalResourceType.FILE);
hasKrb5 = true;
}
Map<String, LocalResource> taskManagerLocalResources = new HashMap<>();
// register Flink Jar with remote HDFS
final YarnLocalResourceDescriptor flinkDistLocalResourceDesc =
YarnLocalResourceDescriptor.fromString(remoteFlinkJarPath);
taskManagerLocalResources.put(
flinkDistLocalResourceDesc.getResourceKey(),
flinkDistLocalResourceDesc.toLocalResource());
// To support Yarn Secure Integration Test Scenario
if (yarnConfResource != null) {
taskManagerLocalResources.put(YARN_SITE_FILE_NAME, yarnConfResource);
}
if (krb5ConfResource != null) {
taskManagerLocalResources.put(KRB5_FILE_NAME, krb5ConfResource);
}
if (keytabResource != null) {
taskManagerLocalResources.put(localKeytabPath, keytabResource);
}
// prepare additional files to be shipped
decodeYarnLocalResourceDescriptorListFromString(shipListString)
.forEach(
resourceDesc ->
taskManagerLocalResources.put(
resourceDesc.getResourceKey(),
resourceDesc.toLocalResource()));
// now that all resources are prepared, we can create the launch context
log.info("Creating container launch context for TaskManagers");
boolean hasLogback = new File(workingDirectory, "logback.xml").exists();
boolean hasLog4j = new File(workingDirectory, "log4j.properties").exists();
String launchCommand =
getTaskManagerShellCommand(
flinkConfig,
tmParams,
".",
ApplicationConstants.LOG_DIR_EXPANSION_VAR,
hasLogback,
hasLog4j,
hasKrb5,
taskManagerMainClass,
taskManagerDynamicProperties);
if (log.isDebugEnabled()) {
log.debug("Starting TaskManagers with command: " + launchCommand);
} else {
log.info("Starting TaskManagers");
}
ContainerLaunchContext ctx = Records.newRecord(ContainerLaunchContext.class);
ctx.setCommands(Collections.singletonList(launchCommand));
ctx.setLocalResources(taskManagerLocalResources);
Map<String, String> containerEnv = new HashMap<>();
containerEnv.putAll(tmParams.taskManagerEnv());
// add YARN classpath, etc to the container environment
containerEnv.put(ENV_FLINK_CLASSPATH, classPathString);
setupYarnClassPath(yarnConfig, containerEnv);
containerEnv.put(
YarnConfigKeys.ENV_HADOOP_USER_NAME,
UserGroupInformation.getCurrentUser().getUserName());
if (remoteKeytabPath != null && localKeytabPath != null && keytabPrincipal != null) {
containerEnv.put(YarnConfigKeys.REMOTE_KEYTAB_PATH, remoteKeytabPath);
containerEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localKeytabPath);
containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, keytabPrincipal);
} else if (localKeytabPath != null && keytabPrincipal != null) {
containerEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localKeytabPath);
containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, keytabPrincipal);
}
ctx.setEnvironment(containerEnv);
setAclsFor(ctx, flinkConfig);
// For TaskManager YARN container context, read the tokens from the jobmanager yarn
// container local file.
// NOTE: must read the tokens from the local file, not from the UGI context, because if UGI
// is login
// using Kerberos keytabs, there is no HDFS delegation token in the UGI context.
final String fileLocation = System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
if (fileLocation != null) {
log.debug("Adding security tokens to TaskExecutor's container launch context.");
try (DataOutputBuffer dob = new DataOutputBuffer()) {
Credentials cred =
Credentials.readTokenStorageFile(
new File(fileLocation),
HadoopUtils.getHadoopConfiguration(flinkConfig));
// Filter out AMRMToken before setting the tokens to the TaskManager container
// context.
Credentials taskManagerCred = new Credentials();
Collection<Token<? extends TokenIdentifier>> userTokens = cred.getAllTokens();
for (Token<? extends TokenIdentifier> token : userTokens) {
if (!token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
taskManagerCred.addToken(token.getService(), token);
}
}
taskManagerCred.writeTokenStorageToStream(dob);
ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
ctx.setTokens(securityTokens);
} catch (Throwable t) {
log.error("Failed to add Hadoop's security tokens.", t);
}
} else {
log.info(
"Could not set security tokens because Hadoop's token file location is unknown.");
}
return ctx;
}
/**
* Generates the shell command to start a task manager.
*
* @param flinkConfig The Flink configuration.
* @param tmParams Parameters for the task manager.
* @param configDirectory The configuration directory for the config.yaml
* @param logDirectory The log directory.
* @param hasLogback Uses logback?
* @param hasLog4j Uses log4j?
* @param mainClass The main class to start with.
* @return A String containing the task manager startup command.
*/
public static String getTaskManagerShellCommand(
org.apache.flink.configuration.Configuration flinkConfig,
ContaineredTaskManagerParameters tmParams,
String configDirectory,
String logDirectory,
boolean hasLogback,
boolean hasLog4j,
boolean hasKrb5,
Class<?> mainClass,
String mainArgs) {
final Map<String, String> startCommandValues = new HashMap<>();
startCommandValues.put("java", "$JAVA_HOME/bin/java");
final TaskExecutorProcessSpec taskExecutorProcessSpec =
tmParams.getTaskExecutorProcessSpec();
startCommandValues.put(
"jvmmem", ProcessMemoryUtils.generateJvmParametersStr(taskExecutorProcessSpec));
List<ConfigOption<String>> jvmOptions =
Arrays.asList(
CoreOptions.FLINK_DEFAULT_JVM_OPTIONS,
CoreOptions.FLINK_JVM_OPTIONS,
CoreOptions.FLINK_DEFAULT_TM_JVM_OPTIONS,
CoreOptions.FLINK_TM_JVM_OPTIONS);
startCommandValues.put("jvmopts", generateJvmOptsString(flinkConfig, jvmOptions, hasKrb5));
String logging = "";
if (hasLogback || hasLog4j) {
logging = "-Dlog.file=" + logDirectory + "/taskmanager.log";
if (hasLogback) {
logging += " -Dlogback.configurationFile=file:" + configDirectory + "/logback.xml";
}
if (hasLog4j) {
logging += " -Dlog4j.configuration=file:" + configDirectory + "/log4j.properties";
logging +=
" -Dlog4j.configurationFile=file:" + configDirectory + "/log4j.properties";
}
}
startCommandValues.put("logging", logging);
startCommandValues.put("class", mainClass.getName());
startCommandValues.put(
"redirects",
"1> "
+ logDirectory
+ "/taskmanager.out "
+ "2> "
+ logDirectory
+ "/taskmanager.err");
String argsStr =
TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec)
+ " --configDir "
+ configDirectory;
if (!mainArgs.isEmpty()) {
argsStr += " " + mainArgs;
}
startCommandValues.put("args", argsStr);
final String commandTemplate = flinkConfig.get(YARN_CONTAINER_START_COMMAND_TEMPLATE);
String startCommand = getStartCommand(commandTemplate, startCommandValues);
LOG.debug("TaskManager start command: " + startCommand);
return startCommand;
}
/**
* Replaces placeholders in the template start command with values from startCommandValues.
*
* <p>If the default template {@link
* ConfigConstants#DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE} is used, the following keys
* must be present in the map or the resulting command will still contain placeholders:
*
* <ul>
* <li><tt>java</tt> = path to the Java executable
* <li><tt>jvmmem</tt> = JVM memory limits and tweaks
* <li><tt>jvmopts</tt> = misc options for the Java VM
* <li><tt>logging</tt> = logging-related configuration settings
* <li><tt>class</tt> = main class to execute
* <li><tt>args</tt> = arguments for the main class
* <li><tt>redirects</tt> = output redirects
* </ul>
*
* @param template a template start command with placeholders
* @param startCommandValues a replacement map <tt>placeholder -> value</tt>
* @return the start command with placeholders filled in
*/
public static String getStartCommand(String template, Map<String, String> startCommandValues) {
for (Map.Entry<String, String> variable : startCommandValues.entrySet()) {
template =
template.replace("%" + variable.getKey() + "%", variable.getValue())
.replace(" ", " ")
.trim();
}
return template;
}
public static String generateJvmOptsString(
org.apache.flink.configuration.Configuration conf,
List<ConfigOption<String>> jvmOptions,
boolean hasKrb5) {
StringBuilder javaOptsSb = new StringBuilder();
for (ConfigOption<String> option : jvmOptions) {
concatWithSpace(javaOptsSb, conf.get(option));
}
concatWithSpace(javaOptsSb, IGNORE_UNRECOGNIZED_VM_OPTIONS);
// krb5.conf file will be available as local resource in JM/TM container
if (hasKrb5) {
concatWithSpace(javaOptsSb, "-Djava.security.krb5.conf=krb5.conf");
}
return javaOptsSb.toString().trim();
}
static boolean isRemotePath(String path) throws IOException {
org.apache.flink.core.fs.Path flinkPath = new org.apache.flink.core.fs.Path(path);
return flinkPath.getFileSystem().isDistributedFS();
}
private static List<YarnLocalResourceDescriptor>
decodeYarnLocalResourceDescriptorListFromString(String resources) throws Exception {
final List<YarnLocalResourceDescriptor> resourceDescriptors = new ArrayList<>();
for (String shipResourceDescStr : resources.split(LOCAL_RESOURCE_DESCRIPTOR_SEPARATOR)) {
if (!shipResourceDescStr.isEmpty()) {
resourceDescriptors.add(
YarnLocalResourceDescriptor.fromString(shipResourceDescStr));
}
}
return resourceDescriptors;
}
@VisibleForTesting
static Resource getUnitResource(YarnConfiguration yarnConfig) {
final int unitMemMB, unitVcore;
final String yarnRmSchedulerClazzName = yarnConfig.get(YarnConfiguration.RM_SCHEDULER);
if (Objects.equals(yarnRmSchedulerClazzName, YARN_RM_FAIR_SCHEDULER_CLAZZ)
|| Objects.equals(yarnRmSchedulerClazzName, YARN_RM_SLS_FAIR_SCHEDULER_CLAZZ)) {
String propMem = yarnConfig.get(YARN_RM_INCREMENT_ALLOCATION_MB_KEY);
String propVcore = yarnConfig.get(YARN_RM_INCREMENT_ALLOCATION_VCORES_KEY);
unitMemMB =
propMem != null
? Integer.parseInt(propMem)
: yarnConfig.getInt(
YARN_RM_INCREMENT_ALLOCATION_MB_LEGACY_KEY,
DEFAULT_YARN_RM_INCREMENT_ALLOCATION_MB);
unitVcore =
propVcore != null
? Integer.parseInt(propVcore)
: yarnConfig.getInt(
YARN_RM_INCREMENT_ALLOCATION_VCORES_LEGACY_KEY,
DEFAULT_YARN_RM_INCREMENT_ALLOCATION_VCORES);
} else {
unitMemMB =
yarnConfig.getInt(
YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
unitVcore =
yarnConfig.getInt(
YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
}
return Resource.newInstance(unitMemMB, unitVcore);
}
public static List<Path> getQualifiedRemoteProvidedLibDirs(
org.apache.flink.configuration.Configuration configuration,
YarnConfiguration yarnConfiguration)
throws IOException {
return getRemoteSharedLibPaths(
configuration,
pathStr -> {
final Path path = new Path(pathStr);
return path.getFileSystem(yarnConfiguration).makeQualified(path);
});
}
private static List<Path> getRemoteSharedLibPaths(
org.apache.flink.configuration.Configuration configuration,
FunctionWithException<String, Path, IOException> strToPathMapper)
throws IOException {
final List<Path> providedLibDirs =
ConfigUtils.decodeListFromConfig(
configuration, YarnConfigOptions.PROVIDED_LIB_DIRS, strToPathMapper);
for (Path path : providedLibDirs) {
if (!Utils.isRemotePath(path.toString())) {
throw new IllegalArgumentException(
"The \""
+ YarnConfigOptions.PROVIDED_LIB_DIRS.key()
+ "\" should only contain"
+ " dirs accessible from all worker nodes, while the \""
+ path
+ "\" is local.");
}
}
return providedLibDirs;
}
public static boolean isUsrLibDirectory(final FileSystem fileSystem, final Path path)
throws IOException {
final FileStatus fileStatus = fileSystem.getFileStatus(path);
// Use the Path obj from fileStatus to get rid of trailing slash
return fileStatus.isDirectory()
&& ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR.equals(fileStatus.getPath().getName());
}
public static Optional<Path> getQualifiedRemoteProvidedUsrLib(
org.apache.flink.configuration.Configuration configuration,
YarnConfiguration yarnConfiguration)
throws IOException, IllegalArgumentException {
String usrlib = configuration.get(YarnConfigOptions.PROVIDED_USRLIB_DIR);
if (usrlib == null) {
return Optional.empty();
}
final Path qualifiedUsrLibPath =
FileSystem.get(yarnConfiguration).makeQualified(new Path(usrlib));
checkArgument(
isRemotePath(qualifiedUsrLibPath.toString()),
"The \"%s\" must point to a remote dir "
+ "which is accessible from all worker nodes.",
YarnConfigOptions.PROVIDED_USRLIB_DIR.key());
checkArgument(
isUsrLibDirectory(FileSystem.get(yarnConfiguration), qualifiedUsrLibPath),
"The \"%s\" should be named with \"%s\".",
YarnConfigOptions.PROVIDED_USRLIB_DIR.key(),
ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR);
return Optional.of(qualifiedUsrLibPath);
}
public static YarnConfiguration getYarnAndHadoopConfiguration(
org.apache.flink.configuration.Configuration flinkConfig) {
final YarnConfiguration yarnConfig = getYarnConfiguration(flinkConfig);
yarnConfig.addResource(HadoopUtils.getHadoopConfiguration(flinkConfig));
return yarnConfig;
}
/**
* Add additional config entries from the flink config to the yarn config.
*
* @param flinkConfig The Flink configuration object.
* @return The yarn configuration.
*/
public static YarnConfiguration getYarnConfiguration(
org.apache.flink.configuration.Configuration flinkConfig) {
final YarnConfiguration yarnConfig = new YarnConfiguration();
for (String key : flinkConfig.keySet()) {
for (String prefix : FLINK_CONFIG_PREFIXES) {
if (key.startsWith(prefix)) {
String newKey = key.substring("flink.".length());
String value = flinkConfig.getString(key, null);
yarnConfig.set(newKey, value);
LOG.debug(
"Adding Flink config entry for {} as {}={} to Yarn config",
key,
newKey,
value);
}
}
}
return yarnConfig;
}
/**
* Sets the application ACLs for the given ContainerLaunchContext based on the values specified
* in the given Flink configuration. Only ApplicationAccessType.VIEW_APP and
* ApplicationAccessType.MODIFY_APP ACLs are set, and only if they are configured in the Flink
* configuration. If the viewAcls or modifyAcls string contains the WILDCARD_ACL constant, it
* will replace the entire string with the WILDCARD_ACL. The resulting map is then set as the
* application acls for the given container launch context.
*
* @param amContainer the ContainerLaunchContext to set the ACLs for.
* @param flinkConfig the Flink configuration to read the ACL values from.
*/
public static void setAclsFor(
ContainerLaunchContext amContainer,
org.apache.flink.configuration.Configuration flinkConfig) {
Map<ApplicationAccessType, String> acls = new HashMap<>();
final String viewAcls = flinkConfig.get(YarnConfigOptions.APPLICATION_VIEW_ACLS);
final String modifyAcls = flinkConfig.get(YarnConfigOptions.APPLICATION_MODIFY_ACLS);
validateAclString(viewAcls);
validateAclString(modifyAcls);
if (viewAcls != null && !viewAcls.isEmpty()) {
acls.put(ApplicationAccessType.VIEW_APP, viewAcls);
}
if (modifyAcls != null && !modifyAcls.isEmpty()) {
acls.put(ApplicationAccessType.MODIFY_APP, modifyAcls);
}
if (!acls.isEmpty()) {
amContainer.setApplicationACLs(acls);
}
}
/* Validates the ACL string to ensure that it is either null or the wildcard ACL. */
private static void validateAclString(String acl) {
if (acl != null && acl.contains("*") && !acl.equals("*")) {
throw new IllegalArgumentException(
String.format(
"Invalid wildcard ACL %s. The ACL wildcard does not support regex. The only valid wildcard ACL is '*'.",
acl));
}
}
public static Path getPathFromLocalFile(File localFile) {
return new Path(localFile.toURI());
}
public static Path getPathFromLocalFilePathStr(String localPathStr) {
return getPathFromLocalFile(new File(localPathStr));
}
public static void concatWithSpace(StringBuilder sb, String value) {
if (value == null || value.isEmpty()) {
return;
}
sb.append(' ');
sb.append(value);
}
}