001/**
002* Licensed to the Apache Software Foundation (ASF) under one
003* or more contributor license agreements.  See the NOTICE file
004* distributed with this work for additional information
005* regarding copyright ownership.  The ASF licenses this file
006* to you under the Apache License, Version 2.0 (the
007* "License"); you may not use this file except in compliance
008* with the License.  You may obtain a copy of the License at
009*
010*     http://www.apache.org/licenses/LICENSE-2.0
011*
012* Unless required by applicable law or agreed to in writing, software
013* distributed under the License is distributed on an "AS IS" BASIS,
014* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015* See the License for the specific language governing permissions and
016* limitations under the License.
017*/
018
019package org.apache.hadoop.yarn.conf;
020
021import java.net.InetSocketAddress;
022import java.util.Arrays;
023import java.util.Collections;
024import java.util.List;
025
026import org.apache.hadoop.HadoopIllegalArgumentException;
027import org.apache.hadoop.classification.InterfaceAudience.Private;
028import org.apache.hadoop.classification.InterfaceAudience.Public;
029import org.apache.hadoop.classification.InterfaceStability.Evolving;
030import org.apache.hadoop.classification.InterfaceStability.Unstable;
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.http.HttpConfig;
033import org.apache.hadoop.net.NetUtils;
034import org.apache.hadoop.util.StringUtils;
035import org.apache.hadoop.yarn.api.ApplicationConstants;
036
037@Public
038@Evolving
039public class YarnConfiguration extends Configuration {
040
041  @Private
042  public static final String CS_CONFIGURATION_FILE= "capacity-scheduler.xml";
043
044  @Private
045  public static final String HADOOP_POLICY_CONFIGURATION_FILE =
046      "hadoop-policy.xml";
047
048  @Private
049  public static final String YARN_SITE_CONFIGURATION_FILE = "yarn-site.xml";
050
051  private static final String YARN_DEFAULT_CONFIGURATION_FILE =
052      "yarn-default.xml";
053
054  @Private
055  public static final String CORE_SITE_CONFIGURATION_FILE = "core-site.xml";
056
057  @Private
058  public static final List<String> RM_CONFIGURATION_FILES =
059      Collections.unmodifiableList(Arrays.asList(
060          CS_CONFIGURATION_FILE,
061          HADOOP_POLICY_CONFIGURATION_FILE,
062          YARN_SITE_CONFIGURATION_FILE,
063          CORE_SITE_CONFIGURATION_FILE));
064
065  @Evolving
066  public static final int APPLICATION_MAX_TAGS = 10;
067
068  @Evolving
069  public static final int APPLICATION_MAX_TAG_LENGTH = 100;
070
071  static {
072    addDeprecatedKeys();
073    Configuration.addDefaultResource(YARN_DEFAULT_CONFIGURATION_FILE);
074    Configuration.addDefaultResource(YARN_SITE_CONFIGURATION_FILE);
075  }
076
077  private static void addDeprecatedKeys() {
078    Configuration.addDeprecations(new DeprecationDelta[] {
079        new DeprecationDelta("yarn.client.max-nodemanagers-proxies",
080            NM_CLIENT_MAX_NM_PROXIES)
081    });
082  }
083
084  //Configurations
085
086  public static final String YARN_PREFIX = "yarn.";
087
088  /** Delay before deleting resource to ease debugging of NM issues */
089  public static final String DEBUG_NM_DELETE_DELAY_SEC =
090    YarnConfiguration.NM_PREFIX + "delete.debug-delay-sec";
091  
092  ////////////////////////////////
093  // IPC Configs
094  ////////////////////////////////
095  public static final String IPC_PREFIX = YARN_PREFIX + "ipc.";
096
097  /** Factory to create client IPC classes.*/
098  public static final String IPC_CLIENT_FACTORY_CLASS =
099    IPC_PREFIX + "client.factory.class";
100  public static final String DEFAULT_IPC_CLIENT_FACTORY_CLASS = 
101      "org.apache.hadoop.yarn.factories.impl.pb.RpcClientFactoryPBImpl";
102
103  /** Factory to create server IPC classes.*/
104  public static final String IPC_SERVER_FACTORY_CLASS = 
105    IPC_PREFIX + "server.factory.class";
106  public static final String DEFAULT_IPC_SERVER_FACTORY_CLASS = 
107      "org.apache.hadoop.yarn.factories.impl.pb.RpcServerFactoryPBImpl";
108
109  /** Factory to create serializeable records.*/
110  public static final String IPC_RECORD_FACTORY_CLASS = 
111    IPC_PREFIX + "record.factory.class";
112  public static final String DEFAULT_IPC_RECORD_FACTORY_CLASS = 
113      "org.apache.hadoop.yarn.factories.impl.pb.RecordFactoryPBImpl";
114
115  /** RPC class implementation*/
116  public static final String IPC_RPC_IMPL =
117    IPC_PREFIX + "rpc.class";
118  public static final String DEFAULT_IPC_RPC_IMPL = 
119    "org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC";
120  
121  ////////////////////////////////
122  // Resource Manager Configs
123  ////////////////////////////////
124  public static final String RM_PREFIX = "yarn.resourcemanager.";
125
126  public static final String RM_CLUSTER_ID = RM_PREFIX + "cluster-id";
127
128  public static final String RM_HOSTNAME = RM_PREFIX + "hostname";
129
130  /** The address of the applications manager interface in the RM.*/
131  public static final String RM_ADDRESS = 
132    RM_PREFIX + "address";
133  public static final int DEFAULT_RM_PORT = 8032;
134  public static final String DEFAULT_RM_ADDRESS =
135    "0.0.0.0:" + DEFAULT_RM_PORT;
136
137  /** The actual bind address for the RM.*/
138  public static final String RM_BIND_HOST =
139    RM_PREFIX + "bind-host";
140
141  /** The number of threads used to handle applications manager requests.*/
142  public static final String RM_CLIENT_THREAD_COUNT =
143    RM_PREFIX + "client.thread-count";
144  public static final int DEFAULT_RM_CLIENT_THREAD_COUNT = 50;
145
146  /** Number of threads used to launch/cleanup AM.*/
147  public static final String RM_AMLAUNCHER_THREAD_COUNT =
148      RM_PREFIX + "amlauncher.thread-count";
149  public static final int DEFAULT_RM_AMLAUNCHER_THREAD_COUNT = 50;
150
151  /** Retry times to connect with NM.*/
152  public static final String RM_NODEMANAGER_CONNECT_RETIRES =
153      RM_PREFIX + "nodemanager-connect-retries";
154  public static final int DEFAULT_RM_NODEMANAGER_CONNECT_RETIRES = 10;
155
156  /** The Kerberos principal for the resource manager.*/
157  public static final String RM_PRINCIPAL =
158    RM_PREFIX + "principal";
159  
160  /** The address of the scheduler interface.*/
161  public static final String RM_SCHEDULER_ADDRESS = 
162    RM_PREFIX + "scheduler.address";
163  public static final int DEFAULT_RM_SCHEDULER_PORT = 8030;
164  public static final String DEFAULT_RM_SCHEDULER_ADDRESS = "0.0.0.0:" +
165    DEFAULT_RM_SCHEDULER_PORT;
166
167  /** Miniumum request grant-able by the RM scheduler. */
168  public static final String RM_SCHEDULER_MINIMUM_ALLOCATION_MB =
169    YARN_PREFIX + "scheduler.minimum-allocation-mb";
170  public static final int DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB = 1024;
171  public static final String RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES =
172      YARN_PREFIX + "scheduler.minimum-allocation-vcores";
173    public static final int DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES = 1;
174
175  /** Maximum request grant-able by the RM scheduler. */
176  public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_MB =
177    YARN_PREFIX + "scheduler.maximum-allocation-mb";
178  public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB = 8192;
179  public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES =
180      YARN_PREFIX + "scheduler.maximum-allocation-vcores";
181  public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES = 4;
182
183  /** Number of threads to handle scheduler interface.*/
184  public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT =
185    RM_PREFIX + "scheduler.client.thread-count";
186  public static final int DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT = 50;
187
188  /** If the port should be included or not in the node name. The node name
189   * is used by the scheduler for resource requests allocation location 
190   * matching. Typically this is just the hostname, using the port is needed
191   * when using minicluster and specific NM are required.*/
192  public static final String RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME =
193      YARN_PREFIX + "scheduler.include-port-in-node-name";
194  public static final boolean DEFAULT_RM_SCHEDULER_USE_PORT_FOR_NODE_NAME = 
195      false;
196
197  /** Enable Resource Manager webapp ui actions */
198  public static final String RM_WEBAPP_UI_ACTIONS_ENABLED =
199    RM_PREFIX + "webapp.ui-actions.enabled";
200  public static final boolean DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED =
201    true;
202
203  /** Whether the RM should enable Reservation System */
204  public static final String RM_RESERVATION_SYSTEM_ENABLE = RM_PREFIX
205      + "reservation-system.enable";
206  public static final boolean DEFAULT_RM_RESERVATION_SYSTEM_ENABLE = false;
207
208  /** The class to use as the Reservation System. */
209  public static final String RM_RESERVATION_SYSTEM_CLASS = RM_PREFIX
210      + "reservation-system.class";
211
212  /** The PlanFollower for the Reservation System. */
213  public static final String RM_RESERVATION_SYSTEM_PLAN_FOLLOWER = RM_PREFIX
214      + "reservation-system.plan.follower";
215
216  /** The step size of the Reservation System. */
217  public static final String RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
218      RM_PREFIX + "reservation-system.planfollower.time-step";
219  public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
220      1000L;
221
222  /**
223   * Enable periodic monitor threads.
224   * @see #RM_SCHEDULER_MONITOR_POLICIES
225   */
226  public static final String RM_SCHEDULER_ENABLE_MONITORS =
227    RM_PREFIX + "scheduler.monitor.enable";
228  public static final boolean DEFAULT_RM_SCHEDULER_ENABLE_MONITORS = false;
229
230  /** List of SchedulingEditPolicy classes affecting the scheduler. */
231  public static final String RM_SCHEDULER_MONITOR_POLICIES =
232    RM_PREFIX + "scheduler.monitor.policies";
233
234  /** The address of the RM web application.*/
235  public static final String RM_WEBAPP_ADDRESS = 
236    RM_PREFIX + "webapp.address";
237
238  public static final int DEFAULT_RM_WEBAPP_PORT = 8088;
239  public static final String DEFAULT_RM_WEBAPP_ADDRESS = "0.0.0.0:" +
240    DEFAULT_RM_WEBAPP_PORT;
241  
242  /** The https address of the RM web application.*/
243  public static final String RM_WEBAPP_HTTPS_ADDRESS =
244      RM_PREFIX + "webapp.https.address";
245  public static final boolean YARN_SSL_CLIENT_HTTPS_NEED_AUTH_DEFAULT = false;
246  public static final String YARN_SSL_SERVER_RESOURCE_DEFAULT = "ssl-server.xml";
247  
248  public static final int DEFAULT_RM_WEBAPP_HTTPS_PORT = 8090;
249  public static final String DEFAULT_RM_WEBAPP_HTTPS_ADDRESS = "0.0.0.0:"
250      + DEFAULT_RM_WEBAPP_HTTPS_PORT;
251  
252  public static final String RM_RESOURCE_TRACKER_ADDRESS =
253    RM_PREFIX + "resource-tracker.address";
254  public static final int DEFAULT_RM_RESOURCE_TRACKER_PORT = 8031;
255  public static final String DEFAULT_RM_RESOURCE_TRACKER_ADDRESS =
256    "0.0.0.0:" + DEFAULT_RM_RESOURCE_TRACKER_PORT;
257
258  /** The expiry interval for application master reporting.*/
259  public static final String RM_AM_EXPIRY_INTERVAL_MS = 
260    YARN_PREFIX  + "am.liveness-monitor.expiry-interval-ms";
261  public static final int DEFAULT_RM_AM_EXPIRY_INTERVAL_MS = 600000;
262
263  /** How long to wait until a node manager is considered dead.*/
264  public static final String RM_NM_EXPIRY_INTERVAL_MS = 
265    YARN_PREFIX + "nm.liveness-monitor.expiry-interval-ms";
266  public static final int DEFAULT_RM_NM_EXPIRY_INTERVAL_MS = 600000;
267
268  /** Are acls enabled.*/
269  public static final String YARN_ACL_ENABLE = 
270    YARN_PREFIX + "acl.enable";
271  public static final boolean DEFAULT_YARN_ACL_ENABLE = false;
272  
273  /** ACL of who can be admin of YARN cluster.*/
274  public static final String YARN_ADMIN_ACL = 
275    YARN_PREFIX + "admin.acl";
276  public static final String DEFAULT_YARN_ADMIN_ACL = "*";
277  
278  /** ACL used in case none is found. Allows nothing. */
279  public static final String DEFAULT_YARN_APP_ACL = " ";
280
281  /**
282   * Enable/disable intermediate-data encryption at YARN level. For now, this
283   * only is used by the FileSystemRMStateStore to setup right file-system
284   * security attributes.
285   */
286  @Private
287  public static final String YARN_INTERMEDIATE_DATA_ENCRYPTION = YARN_PREFIX
288      + "intermediate-data-encryption.enable";
289
290  @Private
291  public static final Boolean DEFAULT_YARN_INTERMEDIATE_DATA_ENCRYPTION = false;
292
293  /** The address of the RM admin interface.*/
294  public static final String RM_ADMIN_ADDRESS = 
295    RM_PREFIX + "admin.address";
296  public static final int DEFAULT_RM_ADMIN_PORT = 8033;
297  public static final String DEFAULT_RM_ADMIN_ADDRESS = "0.0.0.0:" +
298      DEFAULT_RM_ADMIN_PORT;
299  
300  /**Number of threads used to handle RM admin interface.*/
301  public static final String RM_ADMIN_CLIENT_THREAD_COUNT =
302    RM_PREFIX + "admin.client.thread-count";
303  public static final int DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT = 1;
304  
305  /**
306   * The maximum number of application attempts.
307   * It's a global setting for all application masters.
308   */
309  public static final String RM_AM_MAX_ATTEMPTS =
310    RM_PREFIX + "am.max-attempts";
311  public static final int DEFAULT_RM_AM_MAX_ATTEMPTS = 2;
312  
313  /** The keytab for the resource manager.*/
314  public static final String RM_KEYTAB = 
315    RM_PREFIX + "keytab";
316
317  /**The kerberos principal to be used for spnego filter for RM.*/
318  public static final String RM_WEBAPP_SPNEGO_USER_NAME_KEY =
319      RM_PREFIX + "webapp.spnego-principal";
320  
321  /**The kerberos keytab to be used for spnego filter for RM.*/
322  public static final String RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
323      RM_PREFIX + "webapp.spnego-keytab-file";
324
325  /**
326   * Flag to enable override of the default kerberos authentication filter with
327   * the RM authentication filter to allow authentication using delegation
328   * tokens(fallback to kerberos if the tokens are missing). Only applicable
329   * when the http authentication type is kerberos.
330   */
331  public static final String RM_WEBAPP_DELEGATION_TOKEN_AUTH_FILTER = RM_PREFIX
332      + "webapp.delegation-token-auth-filter.enabled";
333  public static final boolean DEFAULT_RM_WEBAPP_DELEGATION_TOKEN_AUTH_FILTER =
334      true;
335
336  /** Enable cross origin (CORS) support. **/
337  public static final String RM_WEBAPP_ENABLE_CORS_FILTER =
338      RM_PREFIX + "webapp.cross-origin.enabled";
339  public static final boolean DEFAULT_RM_WEBAPP_ENABLE_CORS_FILTER = false;
340
341  /** How long to wait until a container is considered dead.*/
342  public static final String RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS = 
343    RM_PREFIX + "rm.container-allocation.expiry-interval-ms";
344  public static final int DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS = 600000;
345  
346  /** Path to file with nodes to include.*/
347  public static final String RM_NODES_INCLUDE_FILE_PATH = 
348    RM_PREFIX + "nodes.include-path";
349  public static final String DEFAULT_RM_NODES_INCLUDE_FILE_PATH = "";
350  
351  /** Path to file with nodes to exclude.*/
352  public static final String RM_NODES_EXCLUDE_FILE_PATH = 
353    RM_PREFIX + "nodes.exclude-path";
354  public static final String DEFAULT_RM_NODES_EXCLUDE_FILE_PATH = "";
355  
356  /** Number of threads to handle resource tracker calls.*/
357  public static final String RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT =
358    RM_PREFIX + "resource-tracker.client.thread-count";
359  public static final int DEFAULT_RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT = 50;
360  
361  /** The class to use as the resource scheduler.*/
362  public static final String RM_SCHEDULER = 
363    RM_PREFIX + "scheduler.class";
364 
365  public static final String DEFAULT_RM_SCHEDULER = 
366      "org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler";
367
368  /** RM set next Heartbeat interval for NM */
369  public static final String RM_NM_HEARTBEAT_INTERVAL_MS =
370      RM_PREFIX + "nodemanagers.heartbeat-interval-ms";
371  public static final long DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS = 1000;
372
373  /** Number of worker threads that write the history data. */
374  public static final String RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE =
375      RM_PREFIX + "history-writer.multi-threaded-dispatcher.pool-size";
376  public static final int DEFAULT_RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE =
377      10;
378
379  /**
380   *  The setting that controls whether yarn system metrics is published on the
381   *  timeline server or not by RM.
382   */
383  public static final String RM_SYSTEM_METRICS_PUBLISHER_ENABLED =
384      RM_PREFIX + "system-metrics-publisher.enabled";
385  public static final boolean DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED = false;
386
387  public static final String RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE =
388      RM_PREFIX + "system-metrics-publisher.dispatcher.pool-size";
389  public static final int DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE =
390      10;
391
392  //RM delegation token related keys
393  public static final String RM_DELEGATION_KEY_UPDATE_INTERVAL_KEY =
394    RM_PREFIX + "delegation.key.update-interval";
395  public static final long RM_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT =
396    24*60*60*1000; // 1 day
397  public static final String RM_DELEGATION_TOKEN_RENEW_INTERVAL_KEY =
398    RM_PREFIX + "delegation.token.renew-interval";
399  public static final long RM_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT =
400    24*60*60*1000;  // 1 day
401  public static final String RM_DELEGATION_TOKEN_MAX_LIFETIME_KEY =
402     RM_PREFIX + "delegation.token.max-lifetime";
403  public static final long RM_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT =
404    7*24*60*60*1000; // 7 days
405  
406  public static final String RECOVERY_ENABLED = RM_PREFIX + "recovery.enabled";
407  public static final boolean DEFAULT_RM_RECOVERY_ENABLED = false;
408
409  public static final String YARN_FAIL_FAST = YARN_PREFIX + "fail-fast";
410  public static final boolean DEFAULT_YARN_FAIL_FAST = false;
411
412  public static final String RM_FAIL_FAST = RM_PREFIX + "fail-fast";
413
414  @Private
415  public static final String RM_WORK_PRESERVING_RECOVERY_ENABLED = RM_PREFIX
416      + "work-preserving-recovery.enabled";
417  @Private
418  public static final boolean DEFAULT_RM_WORK_PRESERVING_RECOVERY_ENABLED =
419      true;
420
421  public static final String RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS =
422      RM_PREFIX + "work-preserving-recovery.scheduling-wait-ms";
423  public static final long DEFAULT_RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS =
424      10000;
425
426  /** Zookeeper interaction configs */
427  public static final String RM_ZK_PREFIX = RM_PREFIX + "zk-";
428
429  public static final String RM_ZK_ADDRESS = RM_ZK_PREFIX + "address";
430
431  public static final String RM_ZK_NUM_RETRIES = RM_ZK_PREFIX + "num-retries";
432  public static final int DEFAULT_ZK_RM_NUM_RETRIES = 1000;
433
434  public static final String RM_ZK_RETRY_INTERVAL_MS =
435      RM_ZK_PREFIX + "retry-interval-ms";
436  public static final long DEFAULT_RM_ZK_RETRY_INTERVAL_MS = 1000;
437
438  public static final String RM_ZK_TIMEOUT_MS = RM_ZK_PREFIX + "timeout-ms";
439  public static final int DEFAULT_RM_ZK_TIMEOUT_MS = 10000;
440
441  public static final String RM_ZK_ACL = RM_ZK_PREFIX + "acl";
442  public static final String DEFAULT_RM_ZK_ACL = "world:anyone:rwcda";
443
444  public static final String RM_ZK_AUTH = RM_ZK_PREFIX + "auth";
445
446  public static final String ZK_STATE_STORE_PREFIX =
447      RM_PREFIX + "zk-state-store.";
448
449  /** Parent znode path under which ZKRMStateStore will create znodes */
450  public static final String ZK_RM_STATE_STORE_PARENT_PATH =
451      ZK_STATE_STORE_PREFIX + "parent-path";
452  public static final String DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH = "/rmstore";
453
454  /** Root node ACLs for fencing */
455  public static final String ZK_RM_STATE_STORE_ROOT_NODE_ACL =
456      ZK_STATE_STORE_PREFIX + "root-node.acl";
457
458  /** HA related configs */
459  public static final String RM_HA_PREFIX = RM_PREFIX + "ha.";
460  public static final String RM_HA_ENABLED = RM_HA_PREFIX + "enabled";
461  public static final boolean DEFAULT_RM_HA_ENABLED = false;
462
463  public static final String RM_HA_IDS = RM_HA_PREFIX + "rm-ids";
464  public static final String RM_HA_ID = RM_HA_PREFIX + "id";
465
466  /** Store the related configuration files in File System */
467  public static final String FS_BASED_RM_CONF_STORE = RM_PREFIX
468      + "configuration.file-system-based-store";
469  public static final String DEFAULT_FS_BASED_RM_CONF_STORE = "/yarn/conf";
470
471  public static final String RM_CONFIGURATION_PROVIDER_CLASS = RM_PREFIX
472      + "configuration.provider-class";
473  public static final String DEFAULT_RM_CONFIGURATION_PROVIDER_CLASS =
474      "org.apache.hadoop.yarn.LocalConfigurationProvider";
475
476  public static final String YARN_AUTHORIZATION_PROVIDER = YARN_PREFIX
477      + "authorization-provider";
478  private static final List<String> RM_SERVICES_ADDRESS_CONF_KEYS_HTTP =
479      Collections.unmodifiableList(Arrays.asList(
480          RM_ADDRESS,
481          RM_SCHEDULER_ADDRESS,
482          RM_ADMIN_ADDRESS,
483          RM_RESOURCE_TRACKER_ADDRESS,
484          RM_WEBAPP_ADDRESS));
485
486  private static final List<String> RM_SERVICES_ADDRESS_CONF_KEYS_HTTPS =
487      Collections.unmodifiableList(Arrays.asList(
488          RM_ADDRESS,
489          RM_SCHEDULER_ADDRESS,
490          RM_ADMIN_ADDRESS,
491          RM_RESOURCE_TRACKER_ADDRESS,
492          RM_WEBAPP_HTTPS_ADDRESS));
493
494  public static final String AUTO_FAILOVER_PREFIX =
495      RM_HA_PREFIX + "automatic-failover.";
496
497  public static final String AUTO_FAILOVER_ENABLED =
498      AUTO_FAILOVER_PREFIX + "enabled";
499  public static final boolean DEFAULT_AUTO_FAILOVER_ENABLED = true;
500
501  public static final String AUTO_FAILOVER_EMBEDDED =
502      AUTO_FAILOVER_PREFIX + "embedded";
503  public static final boolean DEFAULT_AUTO_FAILOVER_EMBEDDED = true;
504
505  public static final String AUTO_FAILOVER_ZK_BASE_PATH =
506      AUTO_FAILOVER_PREFIX + "zk-base-path";
507  public static final String DEFAULT_AUTO_FAILOVER_ZK_BASE_PATH =
508      "/yarn-leader-election";
509
510  public static final String CLIENT_FAILOVER_PREFIX =
511      YARN_PREFIX + "client.failover-";
512  public static final String CLIENT_FAILOVER_PROXY_PROVIDER =
513      CLIENT_FAILOVER_PREFIX + "proxy-provider";
514  public static final String DEFAULT_CLIENT_FAILOVER_PROXY_PROVIDER =
515      "org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider";
516
517  public static final String CLIENT_FAILOVER_MAX_ATTEMPTS =
518      CLIENT_FAILOVER_PREFIX + "max-attempts";
519
520  public static final String CLIENT_FAILOVER_SLEEPTIME_BASE_MS =
521      CLIENT_FAILOVER_PREFIX + "sleep-base-ms";
522
523  public static final String CLIENT_FAILOVER_SLEEPTIME_MAX_MS =
524      CLIENT_FAILOVER_PREFIX + "sleep-max-ms";
525
526  public static final String CLIENT_FAILOVER_RETRIES =
527      CLIENT_FAILOVER_PREFIX + "retries";
528  public static final int DEFAULT_CLIENT_FAILOVER_RETRIES = 0;
529
530  public static final String CLIENT_FAILOVER_RETRIES_ON_SOCKET_TIMEOUTS =
531      CLIENT_FAILOVER_PREFIX + "retries-on-socket-timeouts";
532  public static final int
533      DEFAULT_CLIENT_FAILOVER_RETRIES_ON_SOCKET_TIMEOUTS = 0;
534
535  ////////////////////////////////
536  // RM state store configs
537  ////////////////////////////////
538  /** The class to use as the persistent store.*/
539  public static final String RM_STORE = RM_PREFIX + "store.class";
540  
541  /** URI for FileSystemRMStateStore */
542  public static final String FS_RM_STATE_STORE_URI = RM_PREFIX
543      + "fs.state-store.uri";
544  public static final String FS_RM_STATE_STORE_RETRY_POLICY_SPEC = RM_PREFIX
545      + "fs.state-store.retry-policy-spec";
546  public static final String DEFAULT_FS_RM_STATE_STORE_RETRY_POLICY_SPEC =
547      "2000, 500";
548
549  public static final String FS_RM_STATE_STORE_NUM_RETRIES =
550      RM_PREFIX + "fs.state-store.num-retries";
551  public static final int DEFAULT_FS_RM_STATE_STORE_NUM_RETRIES = 0;
552
553  public static final String FS_RM_STATE_STORE_RETRY_INTERVAL_MS =
554      RM_PREFIX + "fs.state-store.retry-interval-ms";
555  public static final long DEFAULT_FS_RM_STATE_STORE_RETRY_INTERVAL_MS =
556      1000L;
557
558  public static final String RM_LEVELDB_STORE_PATH = RM_PREFIX
559      + "leveldb-state-store.path";
560
561  /** The maximum number of completed applications RM keeps. */ 
562  public static final String RM_MAX_COMPLETED_APPLICATIONS =
563    RM_PREFIX + "max-completed-applications";
564  public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 10000;
565
566  /**
567   * The maximum number of completed applications RM state store keeps, by
568   * default equals to DEFAULT_RM_MAX_COMPLETED_APPLICATIONS
569   */
570  public static final String RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS =
571      RM_PREFIX + "state-store.max-completed-applications";
572  public static final int DEFAULT_RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS =
573      DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
574
575  /** Default application name */
576  public static final String DEFAULT_APPLICATION_NAME = "N/A";
577
578  /** Default application type */
579  public static final String DEFAULT_APPLICATION_TYPE = "YARN";
580
581  /** Default application type length */
582  public static final int APPLICATION_TYPE_LENGTH = 20;
583  
584  /** Default queue name */
585  public static final String DEFAULT_QUEUE_NAME = "default";
586
587  /**
588   * Buckets (in minutes) for the number of apps running in each queue.
589   */
590  public static final String RM_METRICS_RUNTIME_BUCKETS =
591    RM_PREFIX + "metrics.runtime.buckets";
592
593  /**
594   * Default sizes of the runtime metric buckets in minutes.
595   */
596  public static final String DEFAULT_RM_METRICS_RUNTIME_BUCKETS = 
597    "60,300,1440";
598
599  public static final String RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS = RM_PREFIX
600      + "am-rm-tokens.master-key-rolling-interval-secs";
601
602  public static final long DEFAULT_RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
603      24 * 60 * 60;
604
605  public static final String RM_CONTAINER_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
606      RM_PREFIX + "container-tokens.master-key-rolling-interval-secs";
607
608  public static final long DEFAULT_RM_CONTAINER_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
609      24 * 60 * 60;
610
611  public static final String RM_NMTOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
612      RM_PREFIX + "nm-tokens.master-key-rolling-interval-secs";
613  
614  public static final long DEFAULT_RM_NMTOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
615      24 * 60 * 60;
616
617  public static final String RM_NODEMANAGER_MINIMUM_VERSION =
618      RM_PREFIX + "nodemanager.minimum.version";
619
620  public static final String DEFAULT_RM_NODEMANAGER_MINIMUM_VERSION =
621      "NONE";
622
623  /**
624   * RM proxy users' prefix
625   */
626  public static final String RM_PROXY_USER_PREFIX = RM_PREFIX + "proxyuser.";
627
628  ////////////////////////////////
629  // Node Manager Configs
630  ////////////////////////////////
631  
632  /** Prefix for all node manager configs.*/
633  public static final String NM_PREFIX = "yarn.nodemanager.";
634
635  /** Environment variables that will be sent to containers.*/
636  public static final String NM_ADMIN_USER_ENV = NM_PREFIX + "admin-env";
637  public static final String DEFAULT_NM_ADMIN_USER_ENV = "MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX";
638
639  /** Environment variables that containers may override rather than use NodeManager's default.*/
640  public static final String NM_ENV_WHITELIST = NM_PREFIX + "env-whitelist";
641  public static final String DEFAULT_NM_ENV_WHITELIST = StringUtils.join(",",
642    Arrays.asList(ApplicationConstants.Environment.JAVA_HOME.key(),
643      ApplicationConstants.Environment.HADOOP_COMMON_HOME.key(),
644      ApplicationConstants.Environment.HADOOP_HDFS_HOME.key(),
645      ApplicationConstants.Environment.HADOOP_CONF_DIR.key(),
646      ApplicationConstants.Environment.CLASSPATH_PREPEND_DISTCACHE.key(),
647      ApplicationConstants.Environment.HADOOP_YARN_HOME.key()));
648  
649  /** address of node manager IPC.*/
650  public static final String NM_ADDRESS = NM_PREFIX + "address";
651  public static final int DEFAULT_NM_PORT = 0;
652  public static final String DEFAULT_NM_ADDRESS = "0.0.0.0:"
653      + DEFAULT_NM_PORT;
654  
655  /** The actual bind address or the NM.*/
656  public static final String NM_BIND_HOST =
657    NM_PREFIX + "bind-host";
658
659  /** who will execute(launch) the containers.*/
660  public static final String NM_CONTAINER_EXECUTOR = 
661    NM_PREFIX + "container-executor.class";
662
663  /**  
664   * Adjustment to make to the container os scheduling priority.
665   * The valid values for this could vary depending on the platform.
666   * On Linux, higher values mean run the containers at a less 
667   * favorable priority than the NM. 
668   * The value specified is an int.
669   */
670  public static final String NM_CONTAINER_EXECUTOR_SCHED_PRIORITY = 
671    NM_PREFIX + "container-executor.os.sched.priority.adjustment";
672  public static final int DEFAULT_NM_CONTAINER_EXECUTOR_SCHED_PRIORITY = 0;
673  
674  /** Number of threads container manager uses.*/
675  public static final String NM_CONTAINER_MGR_THREAD_COUNT =
676    NM_PREFIX + "container-manager.thread-count";
677  public static final int DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT = 20;
678  
679  /** Number of threads used in cleanup.*/
680  public static final String NM_DELETE_THREAD_COUNT = 
681    NM_PREFIX +  "delete.thread-count";
682  public static final int DEFAULT_NM_DELETE_THREAD_COUNT = 4;
683  
684  /** Keytab for NM.*/
685  public static final String NM_KEYTAB = NM_PREFIX + "keytab";
686  
687  /**List of directories to store localized files in.*/
688  public static final String NM_LOCAL_DIRS = NM_PREFIX + "local-dirs";
689  public static final String DEFAULT_NM_LOCAL_DIRS = "/tmp/nm-local-dir";
690
691  /**
692   * Number of files in each localized directories
693   * Avoid tuning this too low. 
694   */
695  public static final String NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY =
696    NM_PREFIX + "local-cache.max-files-per-directory";
697  public static final int DEFAULT_NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY = 8192;
698
699  /** Address where the localizer IPC is.*/
700  public static final String NM_LOCALIZER_ADDRESS =
701    NM_PREFIX + "localizer.address";
702  public static final int DEFAULT_NM_LOCALIZER_PORT = 8040;
703  public static final String DEFAULT_NM_LOCALIZER_ADDRESS = "0.0.0.0:" +
704    DEFAULT_NM_LOCALIZER_PORT;
705  
706  /** Interval in between cache cleanups.*/
707  public static final String NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS =
708    NM_PREFIX + "localizer.cache.cleanup.interval-ms";
709  public static final long DEFAULT_NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = 
710    10 * 60 * 1000;
711  
712  /**
713   * Target size of localizer cache in MB, per nodemanager. It is a target
714   * retention size that only includes resources with PUBLIC and PRIVATE
715   * visibility and excludes resources with APPLICATION visibility
716   */
717  public static final String NM_LOCALIZER_CACHE_TARGET_SIZE_MB =
718    NM_PREFIX + "localizer.cache.target-size-mb";
719  public static final long DEFAULT_NM_LOCALIZER_CACHE_TARGET_SIZE_MB = 10 * 1024;
720  
721  /** Number of threads to handle localization requests.*/
722  public static final String NM_LOCALIZER_CLIENT_THREAD_COUNT =
723    NM_PREFIX + "localizer.client.thread-count";
724  public static final int DEFAULT_NM_LOCALIZER_CLIENT_THREAD_COUNT = 5;
725  
726  /** Number of threads to use for localization fetching.*/
727  public static final String NM_LOCALIZER_FETCH_THREAD_COUNT = 
728    NM_PREFIX + "localizer.fetch.thread-count";
729  public static final int DEFAULT_NM_LOCALIZER_FETCH_THREAD_COUNT = 4;
730
731  /** Where to store container logs.*/
732  public static final String NM_LOG_DIRS = NM_PREFIX + "log-dirs";
733  public static final String DEFAULT_NM_LOG_DIRS = "/tmp/logs";
734
735  public static final String NM_RESOURCEMANAGER_MINIMUM_VERSION =
736      NM_PREFIX + "resourcemanager.minimum.version";
737  public static final String DEFAULT_NM_RESOURCEMANAGER_MINIMUM_VERSION = "NONE";
738
739  /** Interval at which the delayed token removal thread runs */
740  public static final String RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
741      RM_PREFIX + "delayed.delegation-token.removal-interval-ms";
742  public static final long DEFAULT_RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
743      30000l;
744  
745  /** Delegation Token renewer thread count */
746  public static final String RM_DELEGATION_TOKEN_RENEWER_THREAD_COUNT =
747      RM_PREFIX + "delegation-token-renewer.thread-count";
748  public static final int DEFAULT_RM_DELEGATION_TOKEN_RENEWER_THREAD_COUNT = 50;
749
750  public static final String RM_PROXY_USER_PRIVILEGES_ENABLED = RM_PREFIX
751      + "proxy-user-privileges.enabled";
752  public static boolean DEFAULT_RM_PROXY_USER_PRIVILEGES_ENABLED = false;
753
754  /** Whether to enable log aggregation */
755  public static final String LOG_AGGREGATION_ENABLED = YARN_PREFIX
756      + "log-aggregation-enable";
757  public static final boolean DEFAULT_LOG_AGGREGATION_ENABLED = false;
758  
759  /** 
760   * How long to wait before deleting aggregated logs, -1 disables.
761   * Be careful set this too small and you will spam the name node.
762   */
763  public static final String LOG_AGGREGATION_RETAIN_SECONDS = YARN_PREFIX
764      + "log-aggregation.retain-seconds";
765  public static final long DEFAULT_LOG_AGGREGATION_RETAIN_SECONDS = -1;
766  
767  /**
768   * How long to wait between aggregated log retention checks. If set to
769   * a value {@literal <=} 0 then the value is computed as one-tenth of the
770   * log retention setting. Be careful set this too small and you will spam
771   * the name node.
772   */
773  public static final String LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS =
774      YARN_PREFIX + "log-aggregation.retain-check-interval-seconds";
775  public static final long DEFAULT_LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS = -1;
776
777  /**
778   * Number of seconds to retain logs on the NodeManager. Only applicable if Log
779   * aggregation is disabled
780   */
781  public static final String NM_LOG_RETAIN_SECONDS = NM_PREFIX
782      + "log.retain-seconds";
783  public static final long DEFAULT_NM_LOG_RETAIN_SECONDS = 3 * 60 * 60;
784
785  /**
786   * Define how often NMs wake up and upload log files
787   */
788  public static final String NM_LOG_AGGREGATION_ROLL_MONITORING_INTERVAL_SECONDS =
789      NM_PREFIX + "log-aggregation.roll-monitoring-interval-seconds";
790  public static final long
791      DEFAULT_NM_LOG_AGGREGATION_ROLL_MONITORING_INTERVAL_SECONDS = -1;
792  /**
793   * Number of threads used in log cleanup. Only applicable if Log aggregation
794   * is disabled
795   */
796  public static final String NM_LOG_DELETION_THREADS_COUNT = 
797    NM_PREFIX +  "log.deletion-threads-count";
798  public static final int DEFAULT_NM_LOG_DELETE_THREAD_COUNT = 4;
799
800  /** Where to aggregate logs to.*/
801  public static final String NM_REMOTE_APP_LOG_DIR = 
802    NM_PREFIX + "remote-app-log-dir";
803  public static final String DEFAULT_NM_REMOTE_APP_LOG_DIR = "/tmp/logs";
804
805  /**
806   * The remote log dir will be created at
807   * NM_REMOTE_APP_LOG_DIR/${user}/NM_REMOTE_APP_LOG_DIR_SUFFIX/${appId}
808   */
809  public static final String NM_REMOTE_APP_LOG_DIR_SUFFIX = 
810    NM_PREFIX + "remote-app-log-dir-suffix";
811  public static final String DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX="logs";
812
813  public static final String YARN_LOG_SERVER_URL =
814    YARN_PREFIX + "log.server.url";
815  
816  public static final String YARN_TRACKING_URL_GENERATOR = 
817      YARN_PREFIX + "tracking.url.generator";
818
819  /** Amount of memory in GB that can be allocated for containers.*/
820  public static final String NM_PMEM_MB = NM_PREFIX + "resource.memory-mb";
821  public static final int DEFAULT_NM_PMEM_MB = 8 * 1024;
822
823  /** Specifies whether physical memory check is enabled. */
824  public static final String NM_PMEM_CHECK_ENABLED = NM_PREFIX
825      + "pmem-check-enabled";
826  public static final boolean DEFAULT_NM_PMEM_CHECK_ENABLED = true;
827
828  /** Specifies whether physical memory check is enabled. */
829  public static final String NM_VMEM_CHECK_ENABLED = NM_PREFIX
830      + "vmem-check-enabled";
831  public static final boolean DEFAULT_NM_VMEM_CHECK_ENABLED = true;
832
833  /** Conversion ratio for physical memory to virtual memory. */
834  public static final String NM_VMEM_PMEM_RATIO =
835    NM_PREFIX + "vmem-pmem-ratio";
836  public static final float DEFAULT_NM_VMEM_PMEM_RATIO = 2.1f;
837  
838  /** Number of Virtual CPU Cores which can be allocated for containers.*/
839  public static final String NM_VCORES = NM_PREFIX + "resource.cpu-vcores";
840  public static final int DEFAULT_NM_VCORES = 8;
841
842  /** Percentage of overall CPU which can be allocated for containers. */
843  public static final String NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT =
844      NM_PREFIX + "resource.percentage-physical-cpu-limit";
845  public static final int DEFAULT_NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT =
846      100;
847  
848  /** NM Webapp address.**/
849  public static final String NM_WEBAPP_ADDRESS = NM_PREFIX + "webapp.address";
850  public static final int DEFAULT_NM_WEBAPP_PORT = 8042;
851  public static final String DEFAULT_NM_WEBAPP_ADDRESS = "0.0.0.0:" +
852    DEFAULT_NM_WEBAPP_PORT;
853  
854  /** NM Webapp https address.**/
855  public static final String NM_WEBAPP_HTTPS_ADDRESS = NM_PREFIX
856      + "webapp.https.address";
857  public static final int DEFAULT_NM_WEBAPP_HTTPS_PORT = 8044;
858  public static final String DEFAULT_NM_WEBAPP_HTTPS_ADDRESS = "0.0.0.0:"
859      + DEFAULT_NM_WEBAPP_HTTPS_PORT; 
860
861  /** Enable/disable CORS filter. */
862  public static final String NM_WEBAPP_ENABLE_CORS_FILTER =
863      NM_PREFIX + "webapp.cross-origin.enabled";
864  public static final boolean DEFAULT_NM_WEBAPP_ENABLE_CORS_FILTER = false;
865
866  /** How often to monitor containers.*/
867  public final static String NM_CONTAINER_MON_INTERVAL_MS =
868    NM_PREFIX + "container-monitor.interval-ms";
869  public final static int DEFAULT_NM_CONTAINER_MON_INTERVAL_MS = 3000;
870
871  /** Class that calculates containers current resource utilization.*/
872  public static final String NM_CONTAINER_MON_RESOURCE_CALCULATOR =
873    NM_PREFIX + "container-monitor.resource-calculator.class";
874  /** Class that calculates process tree resource utilization.*/
875  public static final String NM_CONTAINER_MON_PROCESS_TREE =
876    NM_PREFIX + "container-monitor.process-tree.class";
877  public static final String PROCFS_USE_SMAPS_BASED_RSS_ENABLED = NM_PREFIX +
878      "container-monitor.procfs-tree.smaps-based-rss.enabled";
879  public static final boolean DEFAULT_PROCFS_USE_SMAPS_BASED_RSS_ENABLED =
880      false;
881
882  /** Enable/disable container metrics. */
883  @Private
884  public static final String NM_CONTAINER_METRICS_ENABLE =
885      NM_PREFIX + "container-metrics.enable";
886  @Private
887  public static final boolean DEFAULT_NM_CONTAINER_METRICS_ENABLE = true;
888
889  /** Container metrics flush period. -1 for flush on completion. */
890  @Private
891  public static final String NM_CONTAINER_METRICS_PERIOD_MS =
892      NM_PREFIX + "container-metrics.period-ms";
893  @Private
894  public static final int DEFAULT_NM_CONTAINER_METRICS_PERIOD_MS = -1;
895
896  /** The delay time ms to unregister container metrics after completion. */
897  @Private
898  public static final String NM_CONTAINER_METRICS_UNREGISTER_DELAY_MS =
899      NM_PREFIX + "container-metrics.unregister-delay-ms";
900  @Private
901  public static final int DEFAULT_NM_CONTAINER_METRICS_UNREGISTER_DELAY_MS =
902      10000;
903
904  /** Prefix for all node manager disk health checker configs. */
905  private static final String NM_DISK_HEALTH_CHECK_PREFIX =
906      "yarn.nodemanager.disk-health-checker.";
907  /**
908   * Enable/Disable disks' health checker. Default is true. An expert level
909   * configuration property.
910   */
911  public static final String NM_DISK_HEALTH_CHECK_ENABLE =
912      NM_DISK_HEALTH_CHECK_PREFIX + "enable";
913  /** Frequency of running disks' health checker. */
914  public static final String NM_DISK_HEALTH_CHECK_INTERVAL_MS =
915      NM_DISK_HEALTH_CHECK_PREFIX + "interval-ms";
916  /** By default, disks' health is checked every 2 minutes. */
917  public static final long DEFAULT_NM_DISK_HEALTH_CHECK_INTERVAL_MS =
918      2 * 60 * 1000;
919
920  /**
921   * The minimum fraction of number of disks to be healthy for the nodemanager
922   * to launch new containers. This applies to nm-local-dirs and nm-log-dirs.
923   */
924  public static final String NM_MIN_HEALTHY_DISKS_FRACTION =
925      NM_DISK_HEALTH_CHECK_PREFIX + "min-healthy-disks";
926  /**
927   * By default, at least 25% of disks are to be healthy to say that the node is
928   * healthy in terms of disks.
929   */
930  public static final float DEFAULT_NM_MIN_HEALTHY_DISKS_FRACTION = 0.25F;
931
932  /**
933   * The maximum percentage of disk space that can be used after which a disk is
934   * marked as offline. Values can range from 0.0 to 100.0. If the value is
935   * greater than or equal to 100, NM will check for full disk. This applies to
936   * nm-local-dirs and nm-log-dirs.
937   */
938  public static final String NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE =
939      NM_DISK_HEALTH_CHECK_PREFIX + "max-disk-utilization-per-disk-percentage";
940  /**
941   * By default, 90% of the disk can be used before it is marked as offline.
942   */
943  public static final float DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE =
944      90.0F;
945
946  /**
947   * The minimum space that must be available on a local dir for it to be used.
948   * This applies to nm-local-dirs and nm-log-dirs.
949   */
950  public static final String NM_MIN_PER_DISK_FREE_SPACE_MB =
951      NM_DISK_HEALTH_CHECK_PREFIX + "min-free-space-per-disk-mb";
952  /**
953   * By default, all of the disk can be used before it is marked as offline.
954   */
955  public static final long DEFAULT_NM_MIN_PER_DISK_FREE_SPACE_MB = 0;
956
957  /** Frequency of running node health script.*/
958  public static final String NM_HEALTH_CHECK_INTERVAL_MS = 
959    NM_PREFIX + "health-checker.interval-ms";
960  public static final long DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS = 10 * 60 * 1000;
961
962  /** Health check script time out period.*/  
963  public static final String NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS = 
964    NM_PREFIX + "health-checker.script.timeout-ms";
965  public static final long DEFAULT_NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS = 
966    2 * DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS;
967  
968  /** The health check script to run.*/
969  public static final String NM_HEALTH_CHECK_SCRIPT_PATH = 
970    NM_PREFIX + "health-checker.script.path";
971  
972  /** The arguments to pass to the health check script.*/
973  public static final String NM_HEALTH_CHECK_SCRIPT_OPTS = 
974    NM_PREFIX + "health-checker.script.opts";
975
976  /** The Docker image name(For DockerContainerExecutor).*/
977  public static final String NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME =
978    NM_PREFIX + "docker-container-executor.image-name";
979
980  /** The name of the docker executor (For DockerContainerExecutor).*/
981  public static final String NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME =
982    NM_PREFIX + "docker-container-executor.exec-name";
983
984  /** The default docker executor (For DockerContainerExecutor).*/
985  public static final String NM_DEFAULT_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME =
986          "/usr/bin/docker";
987
988  /** The path to the Linux container executor.*/
989  public static final String NM_LINUX_CONTAINER_EXECUTOR_PATH =
990    NM_PREFIX + "linux-container-executor.path";
991  
992  /** 
993   * The UNIX group that the linux-container-executor should run as.
994   * This is intended to be set as part of container-executor.cfg. 
995   */
996  public static final String NM_LINUX_CONTAINER_GROUP =
997    NM_PREFIX + "linux-container-executor.group";
998
999  /**
1000   * If linux-container-executor should limit itself to one user
1001   * when running in non-secure mode.
1002   */
1003  public static final String NM_NONSECURE_MODE_LIMIT_USERS= NM_PREFIX +
1004     "linux-container-executor.nonsecure-mode.limit-users";
1005
1006  public static final boolean DEFAULT_NM_NONSECURE_MODE_LIMIT_USERS = true; 
1007
1008  /**
1009   * The UNIX user that containers will run as when Linux-container-executor
1010   * is used in nonsecure mode (a use case for this is using cgroups).
1011   */
1012  public static final String NM_NONSECURE_MODE_LOCAL_USER_KEY = NM_PREFIX +
1013      "linux-container-executor.nonsecure-mode.local-user";
1014
1015  public static final String DEFAULT_NM_NONSECURE_MODE_LOCAL_USER = "nobody";
1016
1017  /**
1018   * The allowed pattern for UNIX user names enforced by 
1019   * Linux-container-executor when used in nonsecure mode (use case for this 
1020   * is using cgroups). The default value is taken from /usr/sbin/adduser
1021   */
1022  public static final String NM_NONSECURE_MODE_USER_PATTERN_KEY = NM_PREFIX +
1023      "linux-container-executor.nonsecure-mode.user-pattern";
1024
1025  public static final String DEFAULT_NM_NONSECURE_MODE_USER_PATTERN = 
1026      "^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$";
1027
1028  /** The type of resource enforcement to use with the
1029   *  linux container executor.
1030   */
1031  public static final String NM_LINUX_CONTAINER_RESOURCES_HANDLER = 
1032  NM_PREFIX + "linux-container-executor.resources-handler.class";
1033  
1034  /** The path the linux container executor should use for cgroups */
1035  public static final String NM_LINUX_CONTAINER_CGROUPS_HIERARCHY =
1036    NM_PREFIX + "linux-container-executor.cgroups.hierarchy";
1037  
1038  /** Whether the linux container executor should mount cgroups if not found */
1039  public static final String NM_LINUX_CONTAINER_CGROUPS_MOUNT =
1040    NM_PREFIX + "linux-container-executor.cgroups.mount";
1041  
1042  /** Where the linux container executor should mount cgroups if not found */
1043  public static final String NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH =
1044    NM_PREFIX + "linux-container-executor.cgroups.mount-path";
1045
1046  /**
1047   * Whether the apps should run in strict resource usage mode(not allowed to
1048   * use spare CPU)
1049   */
1050  public static final String NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE =
1051      NM_PREFIX + "linux-container-executor.cgroups.strict-resource-usage";
1052  public static final boolean DEFAULT_NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE =
1053      false;
1054
1055
1056
1057  /**
1058   * Interval of time the linux container executor should try cleaning up
1059   * cgroups entry when cleaning up a container. This is required due to what 
1060   * it seems a race condition because the SIGTERM/SIGKILL is asynch.
1061   */
1062  public static final String NM_LINUX_CONTAINER_CGROUPS_DELETE_TIMEOUT =
1063   NM_PREFIX + "linux-container-executor.cgroups.delete-timeout-ms";
1064
1065  public static final long DEFAULT_NM_LINUX_CONTAINER_CGROUPS_DELETE_TIMEOUT =
1066      1000;
1067
1068  /**
1069   * Delay between attempts to remove linux cgroup.
1070   */
1071  public static final String NM_LINUX_CONTAINER_CGROUPS_DELETE_DELAY =
1072      NM_PREFIX + "linux-container-executor.cgroups.delete-delay-ms";
1073
1074  public static final long DEFAULT_NM_LINUX_CONTAINER_CGROUPS_DELETE_DELAY =
1075      20;
1076
1077  /**
1078   * Indicates if memory and CPU limits will be set for the Windows Job
1079   * Object for the containers launched by the default container executor.
1080   */
1081  public static final String NM_WINDOWS_CONTAINER_MEMORY_LIMIT_ENABLED =
1082      NM_PREFIX + "windows-container.memory-limit.enabled";
1083  public static final boolean DEFAULT_NM_WINDOWS_CONTAINER_MEMORY_LIMIT_ENABLED = false;
1084
1085  public static final String NM_WINDOWS_CONTAINER_CPU_LIMIT_ENABLED =
1086      NM_PREFIX + "windows-container.cpu-limit.enabled";
1087  public static final boolean DEFAULT_NM_WINDOWS_CONTAINER_CPU_LIMIT_ENABLED = false;
1088
1089  /** 
1090  /* The Windows group that the windows-secure-container-executor should run as.
1091  */
1092  public static final String NM_WINDOWS_SECURE_CONTAINER_GROUP =
1093      NM_PREFIX + "windows-secure-container-executor.group";
1094
1095  /** T-file compression types used to compress aggregated logs.*/
1096  public static final String NM_LOG_AGG_COMPRESSION_TYPE = 
1097    NM_PREFIX + "log-aggregation.compression-type";
1098  public static final String DEFAULT_NM_LOG_AGG_COMPRESSION_TYPE = "none";
1099  
1100  /** The kerberos principal for the node manager.*/
1101  public static final String NM_PRINCIPAL =
1102    NM_PREFIX + "principal";
1103  
1104  public static final String NM_AUX_SERVICES = 
1105    NM_PREFIX + "aux-services";
1106  
1107  public static final String NM_AUX_SERVICE_FMT =
1108    NM_PREFIX + "aux-services.%s.class";
1109
1110  public static final String NM_USER_HOME_DIR =
1111      NM_PREFIX + "user-home-dir";
1112  
1113  /**The kerberos principal to be used for spnego filter for NM.*/
1114  public static final String NM_WEBAPP_SPNEGO_USER_NAME_KEY =
1115      NM_PREFIX + "webapp.spnego-principal";
1116  
1117  /**The kerberos keytab to be used for spnego filter for NM.*/
1118  public static final String NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
1119      NM_PREFIX + "webapp.spnego-keytab-file";
1120  
1121  public static final String DEFAULT_NM_USER_HOME_DIR= "/home/";
1122
1123  public static final String NM_RECOVERY_PREFIX = NM_PREFIX + "recovery.";
1124  public static final String NM_RECOVERY_ENABLED =
1125      NM_RECOVERY_PREFIX + "enabled";
1126  public static final boolean DEFAULT_NM_RECOVERY_ENABLED = false;
1127
1128  public static final String NM_RECOVERY_DIR = NM_RECOVERY_PREFIX + "dir";
1129
1130  ////////////////////////////////
1131  // Web Proxy Configs
1132  ////////////////////////////////
1133  public static final String PROXY_PREFIX = "yarn.web-proxy.";
1134  
1135  /** The kerberos principal for the proxy.*/
1136  public static final String PROXY_PRINCIPAL =
1137    PROXY_PREFIX + "principal";
1138  
1139  /** Keytab for Proxy.*/
1140  public static final String PROXY_KEYTAB = PROXY_PREFIX + "keytab";
1141  
1142  /** The address for the web proxy.*/
1143  public static final String PROXY_ADDRESS =
1144    PROXY_PREFIX + "address";
1145  public static final int DEFAULT_PROXY_PORT = 9099;
1146  public static final String DEFAULT_PROXY_ADDRESS =
1147    "0.0.0.0:" + DEFAULT_PROXY_PORT;
1148  
1149  /**
1150   * YARN Service Level Authorization
1151   */
1152  public static final String 
1153  YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL =
1154      "security.resourcetracker.protocol.acl";
1155  public static final String 
1156  YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONCLIENT_PROTOCOL =
1157      "security.applicationclient.protocol.acl";
1158  public static final String 
1159  YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCEMANAGER_ADMINISTRATION_PROTOCOL =
1160      "security.resourcemanager-administration.protocol.acl";
1161  public static final String 
1162  YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_PROTOCOL =
1163      "security.applicationmaster.protocol.acl";
1164
1165  public static final String 
1166  YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL =
1167      "security.containermanagement.protocol.acl";
1168  public static final String 
1169  YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER =
1170      "security.resourcelocalizer.protocol.acl";
1171
1172  public static final String
1173  YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONHISTORY_PROTOCOL =
1174      "security.applicationhistory.protocol.acl";
1175
1176  /** No. of milliseconds to wait between sending a SIGTERM and SIGKILL
1177   * to a running container */
1178  public static final String NM_SLEEP_DELAY_BEFORE_SIGKILL_MS =
1179      NM_PREFIX + "sleep-delay-before-sigkill.ms";
1180  public static final long DEFAULT_NM_SLEEP_DELAY_BEFORE_SIGKILL_MS =
1181      250;
1182
1183  /** Max time to wait for a process to come up when trying to cleanup
1184   * container resources */
1185  public static final String NM_PROCESS_KILL_WAIT_MS =
1186      NM_PREFIX + "process-kill-wait.ms";
1187  public static final long DEFAULT_NM_PROCESS_KILL_WAIT_MS =
1188      2000;
1189
1190  /** Max time to wait to establish a connection to RM */
1191  public static final String RESOURCEMANAGER_CONNECT_MAX_WAIT_MS =
1192      RM_PREFIX + "connect.max-wait.ms";
1193  public static final long DEFAULT_RESOURCEMANAGER_CONNECT_MAX_WAIT_MS =
1194      15 * 60 * 1000;
1195
1196  /** Time interval between each attempt to connect to RM */
1197  public static final String RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS =
1198      RM_PREFIX + "connect.retry-interval.ms";
1199  public static final long DEFAULT_RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS
1200      = 30 * 1000;
1201
1202  public static final String DISPATCHER_DRAIN_EVENTS_TIMEOUT =
1203      YARN_PREFIX + "dispatcher.drain-events.timeout";
1204
1205  public static final long DEFAULT_DISPATCHER_DRAIN_EVENTS_TIMEOUT = 300000;
1206
1207  /**
1208   * CLASSPATH for YARN applications. A comma-separated list of CLASSPATH
1209   * entries
1210   */
1211  public static final String YARN_APPLICATION_CLASSPATH = YARN_PREFIX
1212      + "application.classpath";
1213
1214  /**
1215   * Default platform-agnostic CLASSPATH for YARN applications. A
1216   * comma-separated list of CLASSPATH entries. The parameter expansion marker
1217   * will be replaced with real parameter expansion marker ('%' for Windows and
1218   * '$' for Linux) by NodeManager on container launch. For example: {{VAR}}
1219   * will be replaced as $VAR on Linux, and %VAR% on Windows.
1220   */
1221  @Public
1222  @Unstable
1223  public static final String[] DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH= {
1224      ApplicationConstants.Environment.HADOOP_CONF_DIR.$$(),
1225      ApplicationConstants.Environment.HADOOP_COMMON_HOME.$$()
1226          + "/share/hadoop/common/*",
1227      ApplicationConstants.Environment.HADOOP_COMMON_HOME.$$()
1228          + "/share/hadoop/common/lib/*",
1229      ApplicationConstants.Environment.HADOOP_HDFS_HOME.$$()
1230          + "/share/hadoop/hdfs/*",
1231      ApplicationConstants.Environment.HADOOP_HDFS_HOME.$$()
1232          + "/share/hadoop/hdfs/lib/*",
1233      ApplicationConstants.Environment.HADOOP_YARN_HOME.$$()
1234          + "/share/hadoop/yarn/*",
1235      ApplicationConstants.Environment.HADOOP_YARN_HOME.$$()
1236          + "/share/hadoop/yarn/lib/*" };
1237  /**
1238   * <p>
1239   * Default platform-specific CLASSPATH for YARN applications. A
1240   * comma-separated list of CLASSPATH entries constructed based on the client
1241   * OS environment expansion syntax.
1242   * </p>
1243   * <p>
1244   * Note: Use {@link #DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH} for
1245   * cross-platform practice i.e. submit an application from a Windows client to
1246   * a Linux/Unix server or vice versa.
1247   * </p>
1248   */
1249  public static final String[] DEFAULT_YARN_APPLICATION_CLASSPATH = {
1250      ApplicationConstants.Environment.HADOOP_CONF_DIR.$(),
1251      ApplicationConstants.Environment.HADOOP_COMMON_HOME.$()
1252          + "/share/hadoop/common/*",
1253      ApplicationConstants.Environment.HADOOP_COMMON_HOME.$()
1254          + "/share/hadoop/common/lib/*",
1255      ApplicationConstants.Environment.HADOOP_HDFS_HOME.$()
1256          + "/share/hadoop/hdfs/*",
1257      ApplicationConstants.Environment.HADOOP_HDFS_HOME.$()
1258          + "/share/hadoop/hdfs/lib/*",
1259      ApplicationConstants.Environment.HADOOP_YARN_HOME.$()
1260          + "/share/hadoop/yarn/*",
1261      ApplicationConstants.Environment.HADOOP_YARN_HOME.$()
1262          + "/share/hadoop/yarn/lib/*" };
1263
1264  /** Container temp directory */
1265  public static final String DEFAULT_CONTAINER_TEMP_DIR = "./tmp";
1266
1267  public static final String IS_MINI_YARN_CLUSTER = YARN_PREFIX
1268      + "is.minicluster";
1269
1270  public static final String YARN_MC_PREFIX = YARN_PREFIX + "minicluster.";
1271
1272  /** Whether to use fixed ports with the minicluster. */
1273  public static final String YARN_MINICLUSTER_FIXED_PORTS =
1274      YARN_MC_PREFIX + "fixed.ports";
1275
1276  /**
1277   * Default is false to be able to run tests concurrently without port
1278   * conflicts.
1279   */
1280  public static final boolean DEFAULT_YARN_MINICLUSTER_FIXED_PORTS = false;
1281
1282  /**
1283   * Whether the NM should use RPC to connect to the RM. Default is false.
1284   * Can be set to true only when using fixed ports.
1285   */
1286  public static final String YARN_MINICLUSTER_USE_RPC = YARN_MC_PREFIX + "use-rpc";
1287  public static final boolean DEFAULT_YARN_MINICLUSTER_USE_RPC = false;
1288
1289  /**
1290   * Whether users are explicitly trying to control resource monitoring
1291   * configuration for the MiniYARNCluster. Disabled by default.
1292   */
1293  public static final String YARN_MINICLUSTER_CONTROL_RESOURCE_MONITORING =
1294      YARN_MC_PREFIX + "control-resource-monitoring";
1295  public static final boolean
1296      DEFAULT_YARN_MINICLUSTER_CONTROL_RESOURCE_MONITORING = false;
1297
1298  /** Allow changing the memory for the NodeManager in the MiniYARNCluster */
1299  public static final String YARN_MINICLUSTER_NM_PMEM_MB =
1300      YARN_MC_PREFIX + YarnConfiguration.NM_PMEM_MB;
1301  public static final int DEFAULT_YARN_MINICLUSTER_NM_PMEM_MB = 4 * 1024;
1302
1303  /** The log directory for the containers */
1304  public static final String YARN_APP_CONTAINER_LOG_DIR =
1305      YARN_PREFIX + "app.container.log.dir";
1306
1307  public static final String YARN_APP_CONTAINER_LOG_SIZE =
1308      YARN_PREFIX + "app.container.log.filesize";
1309
1310  public static final String YARN_APP_CONTAINER_LOG_BACKUPS =
1311      YARN_PREFIX + "app.container.log.backups";
1312
1313  ////////////////////////////////
1314  // Timeline Service Configs
1315  ////////////////////////////////
1316
1317  public static final String TIMELINE_SERVICE_PREFIX =
1318      YARN_PREFIX + "timeline-service.";
1319
1320  /**
1321   * Comma seperated list of names for UIs hosted in the timeline server
1322   * (For pluggable UIs).
1323   */
1324  public static final String TIMELINE_SERVICE_UI_NAMES =
1325      TIMELINE_SERVICE_PREFIX + "ui-names";
1326
1327  /** Relative web path that will serve up this UI (For pluggable UIs). */
1328  public static final String TIMELINE_SERVICE_UI_WEB_PATH_PREFIX =
1329      TIMELINE_SERVICE_PREFIX + "ui-web-path.";
1330
1331  /**
1332   * Path to war file or static content directory for this UI
1333   * (For pluggable UIs).
1334   */
1335  public static final String TIMELINE_SERVICE_UI_ON_DISK_PATH_PREFIX =
1336      TIMELINE_SERVICE_PREFIX + "ui-on-disk-path.";
1337
1338  // mark app-history related configs @Private as application history is going
1339  // to be integrated into the timeline service
1340  @Private
1341  public static final String APPLICATION_HISTORY_PREFIX =
1342      TIMELINE_SERVICE_PREFIX + "generic-application-history.";
1343
1344  /**
1345   *  The setting that controls whether application history service is
1346   *  enabled or not.
1347   */
1348  @Private
1349  public static final String APPLICATION_HISTORY_ENABLED =
1350      APPLICATION_HISTORY_PREFIX + "enabled";
1351  @Private
1352  public static final boolean DEFAULT_APPLICATION_HISTORY_ENABLED = false;
1353
1354  /** Application history store class */
1355  @Private
1356  public static final String APPLICATION_HISTORY_STORE =
1357      APPLICATION_HISTORY_PREFIX + "store-class";
1358
1359  /** Save container meta-info in the application history store. */
1360  @Private
1361  public static final String
1362      APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO =
1363        APPLICATION_HISTORY_PREFIX + "save-non-am-container-meta-info";
1364  @Private
1365  public static final boolean
1366            DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO = true;
1367
1368  /** URI for FileSystemApplicationHistoryStore */
1369  @Private
1370  public static final String FS_APPLICATION_HISTORY_STORE_URI =
1371      APPLICATION_HISTORY_PREFIX + "fs-history-store.uri";
1372
1373  /** T-file compression types used to compress history data.*/
1374  @Private
1375  public static final String FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE =
1376      APPLICATION_HISTORY_PREFIX + "fs-history-store.compression-type";
1377  @Private
1378  public static final String DEFAULT_FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE =
1379      "none";
1380
1381  /** The setting that controls whether timeline service is enabled or not. */
1382  public static final String TIMELINE_SERVICE_ENABLED =
1383      TIMELINE_SERVICE_PREFIX + "enabled";
1384  public static final boolean DEFAULT_TIMELINE_SERVICE_ENABLED = false;
1385
1386  /** host:port address for timeline service RPC APIs. */
1387  public static final String TIMELINE_SERVICE_ADDRESS =
1388      TIMELINE_SERVICE_PREFIX + "address";
1389  public static final int DEFAULT_TIMELINE_SERVICE_PORT = 10200;
1390  public static final String DEFAULT_TIMELINE_SERVICE_ADDRESS = "0.0.0.0:"
1391      + DEFAULT_TIMELINE_SERVICE_PORT;
1392
1393  /** The listening endpoint for the timeline service application.*/
1394  public static final String TIMELINE_SERVICE_BIND_HOST =
1395      TIMELINE_SERVICE_PREFIX + "bind-host";
1396
1397  /** The number of threads to handle client RPC API requests. */
1398  public static final String TIMELINE_SERVICE_HANDLER_THREAD_COUNT =
1399      TIMELINE_SERVICE_PREFIX + "handler-thread-count";
1400  public static final int DEFAULT_TIMELINE_SERVICE_CLIENT_THREAD_COUNT = 10;
1401  
1402
1403  /** The address of the timeline service web application.*/
1404  public static final String TIMELINE_SERVICE_WEBAPP_ADDRESS =
1405      TIMELINE_SERVICE_PREFIX  + "webapp.address";
1406
1407  public static final int DEFAULT_TIMELINE_SERVICE_WEBAPP_PORT = 8188;
1408  public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS =
1409      "0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_PORT;
1410
1411  /** The https address of the timeline service web application.*/
1412  public static final String TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
1413      TIMELINE_SERVICE_PREFIX + "webapp.https.address";
1414
1415  public static final int DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT = 8190;
1416  public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
1417      "0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT;
1418
1419  /**
1420   * Defines the max number of applications could be fetched using
1421   * REST API or application history protocol and shown in timeline
1422   * server web ui.
1423   */
1424  public static final String APPLICATION_HISTORY_MAX_APPS =
1425      APPLICATION_HISTORY_PREFIX + "max-applications";
1426  public static final long DEFAULT_APPLICATION_HISTORY_MAX_APPS = 10000;
1427
1428  /** Timeline service store class */
1429  public static final String TIMELINE_SERVICE_STORE =
1430      TIMELINE_SERVICE_PREFIX + "store-class";
1431
1432  /** Timeline service enable data age off */
1433  public static final String TIMELINE_SERVICE_TTL_ENABLE =
1434      TIMELINE_SERVICE_PREFIX + "ttl-enable";
1435
1436  /** Timeline service length of time to retain data */
1437  public static final String TIMELINE_SERVICE_TTL_MS =
1438      TIMELINE_SERVICE_PREFIX + "ttl-ms";
1439
1440  public static final long DEFAULT_TIMELINE_SERVICE_TTL_MS =
1441      1000 * 60 * 60 * 24 * 7;
1442
1443  public static final String TIMELINE_SERVICE_LEVELDB_PREFIX =
1444      TIMELINE_SERVICE_PREFIX + "leveldb-timeline-store.";
1445
1446  /** Timeline service leveldb path */
1447  public static final String TIMELINE_SERVICE_LEVELDB_PATH =
1448      TIMELINE_SERVICE_LEVELDB_PREFIX + "path";
1449
1450  /** Timeline service leveldb read cache (uncompressed blocks) */
1451  public static final String TIMELINE_SERVICE_LEVELDB_READ_CACHE_SIZE =
1452      TIMELINE_SERVICE_LEVELDB_PREFIX + "read-cache-size";
1453
1454  public static final long DEFAULT_TIMELINE_SERVICE_LEVELDB_READ_CACHE_SIZE =
1455      100 * 1024 * 1024;
1456
1457  /** Timeline service leveldb start time read cache (number of entities) */
1458  public static final String
1459      TIMELINE_SERVICE_LEVELDB_START_TIME_READ_CACHE_SIZE =
1460      TIMELINE_SERVICE_LEVELDB_PREFIX + "start-time-read-cache-size";
1461
1462  public static final int
1463      DEFAULT_TIMELINE_SERVICE_LEVELDB_START_TIME_READ_CACHE_SIZE = 10000;
1464
1465  /** Timeline service leveldb start time write cache (number of entities) */
1466  public static final String
1467      TIMELINE_SERVICE_LEVELDB_START_TIME_WRITE_CACHE_SIZE =
1468      TIMELINE_SERVICE_LEVELDB_PREFIX + "start-time-write-cache-size";
1469
1470  public static final int
1471      DEFAULT_TIMELINE_SERVICE_LEVELDB_START_TIME_WRITE_CACHE_SIZE = 10000;
1472
1473  /** Timeline service leveldb interval to wait between deletion rounds */
1474  public static final String TIMELINE_SERVICE_LEVELDB_TTL_INTERVAL_MS =
1475      TIMELINE_SERVICE_LEVELDB_PREFIX + "ttl-interval-ms";
1476
1477  public static final long DEFAULT_TIMELINE_SERVICE_LEVELDB_TTL_INTERVAL_MS =
1478      1000 * 60 * 5;
1479
1480  /** The Kerberos principal for the timeline server.*/
1481  public static final String TIMELINE_SERVICE_PRINCIPAL =
1482      TIMELINE_SERVICE_PREFIX + "principal";
1483
1484  /** The Kerberos keytab for the timeline server.*/
1485  public static final String TIMELINE_SERVICE_KEYTAB =
1486      TIMELINE_SERVICE_PREFIX + "keytab";
1487
1488  /** Enables cross origin support for timeline server.*/
1489  public static final String TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED =
1490      TIMELINE_SERVICE_PREFIX + "http-cross-origin.enabled";
1491
1492  /** Default value for cross origin support for timeline server.*/
1493  public static final boolean
1494      TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT = false;
1495
1496  /** Timeline client settings */
1497  public static final String TIMELINE_SERVICE_CLIENT_PREFIX =
1498      TIMELINE_SERVICE_PREFIX + "client.";
1499
1500  /** Timeline client call, max retries (-1 means no limit) */
1501  public static final String TIMELINE_SERVICE_CLIENT_MAX_RETRIES =
1502      TIMELINE_SERVICE_CLIENT_PREFIX + "max-retries";
1503
1504  public static final int DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES = 30;
1505
1506  /** Timeline client call, retry interval */
1507  public static final String TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS =
1508      TIMELINE_SERVICE_CLIENT_PREFIX + "retry-interval-ms";
1509
1510  public static final long
1511      DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS = 1000;
1512
1513  /** Timeline client policy for whether connections are fatal */
1514  public static final String TIMELINE_SERVICE_CLIENT_BEST_EFFORT =
1515      TIMELINE_SERVICE_CLIENT_PREFIX + "best-effort";
1516
1517  public static final boolean
1518      DEFAULT_TIMELINE_SERVICE_CLIENT_BEST_EFFORT = false;
1519
1520  /** Flag to enable recovery of timeline service */
1521  public static final String TIMELINE_SERVICE_RECOVERY_ENABLED =
1522      TIMELINE_SERVICE_PREFIX + "recovery.enabled";
1523  public static final boolean DEFAULT_TIMELINE_SERVICE_RECOVERY_ENABLED = false;
1524
1525  /** Timeline service state store class */
1526  public static final String TIMELINE_SERVICE_STATE_STORE_CLASS =
1527      TIMELINE_SERVICE_PREFIX + "state-store-class";
1528
1529  public static final String TIMELINE_SERVICE_LEVELDB_STATE_STORE_PREFIX =
1530      TIMELINE_SERVICE_PREFIX + "leveldb-state-store.";
1531
1532  /** Timeline service state store leveldb path */
1533  public static final String TIMELINE_SERVICE_LEVELDB_STATE_STORE_PATH =
1534      TIMELINE_SERVICE_LEVELDB_STATE_STORE_PREFIX + "path";
1535
1536  // Timeline delegation token related keys
1537  public static final String  TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL =
1538      TIMELINE_SERVICE_PREFIX + "delegation.key.update-interval";
1539  public static final long    DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL =
1540      24*60*60*1000; // 1 day
1541  public static final String  TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL =
1542      TIMELINE_SERVICE_PREFIX + "delegation.token.renew-interval";
1543  public static final long    DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL =
1544      24*60*60*1000;  // 1 day
1545  public static final String  TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME =
1546      TIMELINE_SERVICE_PREFIX + "delegation.token.max-lifetime";
1547  public static final long    DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME =
1548      7*24*60*60*1000; // 7 days
1549
1550  // ///////////////////////////////
1551  // Shared Cache Configs
1552  // ///////////////////////////////
1553  public static final String SHARED_CACHE_PREFIX = "yarn.sharedcache.";
1554
1555  // common configs
1556  /** whether the shared cache is enabled/disabled */
1557  public static final String SHARED_CACHE_ENABLED =
1558      SHARED_CACHE_PREFIX + "enabled";
1559  public static final boolean DEFAULT_SHARED_CACHE_ENABLED = false;
1560
1561  /** The config key for the shared cache root directory. */
1562  public static final String SHARED_CACHE_ROOT =
1563      SHARED_CACHE_PREFIX + "root-dir";
1564  public static final String DEFAULT_SHARED_CACHE_ROOT = "/sharedcache";
1565
1566  /** The config key for the level of nested directories before getting to the
1567   * checksum directory. */
1568  public static final String SHARED_CACHE_NESTED_LEVEL =
1569      SHARED_CACHE_PREFIX + "nested-level";
1570  public static final int DEFAULT_SHARED_CACHE_NESTED_LEVEL = 3;
1571  
1572  // Shared Cache Manager Configs
1573
1574  public static final String SCM_STORE_PREFIX = SHARED_CACHE_PREFIX + "store.";
1575
1576  public static final String SCM_STORE_CLASS = SCM_STORE_PREFIX + "class";
1577  public static final String DEFAULT_SCM_STORE_CLASS =
1578      "org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore";
1579
1580  public static final String SCM_APP_CHECKER_CLASS = SHARED_CACHE_PREFIX
1581      + "app-checker.class";
1582  public static final String DEFAULT_SCM_APP_CHECKER_CLASS =
1583      "org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker";
1584
1585  /** The address of the SCM admin interface. */
1586  public static final String SCM_ADMIN_ADDRESS =
1587      SHARED_CACHE_PREFIX + "admin.address";
1588  public static final int DEFAULT_SCM_ADMIN_PORT = 8047;
1589  public static final String DEFAULT_SCM_ADMIN_ADDRESS =
1590      "0.0.0.0:" + DEFAULT_SCM_ADMIN_PORT;
1591
1592  /** Number of threads used to handle SCM admin interface. */
1593  public static final String SCM_ADMIN_CLIENT_THREAD_COUNT =
1594      SHARED_CACHE_PREFIX + "admin.thread-count";
1595  public static final int DEFAULT_SCM_ADMIN_CLIENT_THREAD_COUNT = 1;
1596
1597  /** The address of the SCM web application. */
1598  public static final String SCM_WEBAPP_ADDRESS =
1599      SHARED_CACHE_PREFIX + "webapp.address";
1600  public static final int DEFAULT_SCM_WEBAPP_PORT = 8788;
1601  public static final String DEFAULT_SCM_WEBAPP_ADDRESS =
1602      "0.0.0.0:" + DEFAULT_SCM_WEBAPP_PORT;
1603
1604  // In-memory SCM store configuration
1605  
1606  public static final String IN_MEMORY_STORE_PREFIX =
1607      SCM_STORE_PREFIX + "in-memory.";
1608
1609  /**
1610   * A resource in the InMemorySCMStore is considered stale if the time since
1611   * the last reference exceeds the staleness period. This value is specified in
1612   * minutes.
1613   */
1614  public static final String IN_MEMORY_STALENESS_PERIOD_MINS =
1615      IN_MEMORY_STORE_PREFIX + "staleness-period-mins";
1616  public static final int DEFAULT_IN_MEMORY_STALENESS_PERIOD_MINS =
1617      7 * 24 * 60;
1618
1619  /**
1620   * Initial delay before the in-memory store runs its first check to remove
1621   * dead initial applications. Specified in minutes.
1622   */
1623  public static final String IN_MEMORY_INITIAL_DELAY_MINS =
1624      IN_MEMORY_STORE_PREFIX + "initial-delay-mins";
1625  public static final int DEFAULT_IN_MEMORY_INITIAL_DELAY_MINS = 10;
1626  
1627  /**
1628   * The frequency at which the in-memory store checks to remove dead initial
1629   * applications. Specified in minutes.
1630   */
1631  public static final String IN_MEMORY_CHECK_PERIOD_MINS =
1632      IN_MEMORY_STORE_PREFIX + "check-period-mins";
1633  public static final int DEFAULT_IN_MEMORY_CHECK_PERIOD_MINS = 12 * 60;
1634
1635  // SCM Cleaner service configuration
1636
1637  private static final String SCM_CLEANER_PREFIX = SHARED_CACHE_PREFIX
1638      + "cleaner.";
1639
1640  /**
1641   * The frequency at which a cleaner task runs. Specified in minutes.
1642   */
1643  public static final String SCM_CLEANER_PERIOD_MINS =
1644      SCM_CLEANER_PREFIX + "period-mins";
1645  public static final int DEFAULT_SCM_CLEANER_PERIOD_MINS = 24 * 60;
1646
1647  /**
1648   * Initial delay before the first cleaner task is scheduled. Specified in
1649   * minutes.
1650   */
1651  public static final String SCM_CLEANER_INITIAL_DELAY_MINS =
1652      SCM_CLEANER_PREFIX + "initial-delay-mins";
1653  public static final int DEFAULT_SCM_CLEANER_INITIAL_DELAY_MINS = 10;
1654
1655  /**
1656   * The time to sleep between processing each shared cache resource. Specified
1657   * in milliseconds.
1658   */
1659  public static final String SCM_CLEANER_RESOURCE_SLEEP_MS =
1660      SCM_CLEANER_PREFIX + "resource-sleep-ms";
1661  public static final long DEFAULT_SCM_CLEANER_RESOURCE_SLEEP_MS = 0L;
1662
1663  /** The address of the node manager interface in the SCM. */
1664  public static final String SCM_UPLOADER_SERVER_ADDRESS = SHARED_CACHE_PREFIX
1665      + "uploader.server.address";
1666  public static final int DEFAULT_SCM_UPLOADER_SERVER_PORT = 8046;
1667  public static final String DEFAULT_SCM_UPLOADER_SERVER_ADDRESS = "0.0.0.0:"
1668      + DEFAULT_SCM_UPLOADER_SERVER_PORT;
1669
1670  /**
1671   * The number of SCM threads used to handle notify requests from the node
1672   * manager.
1673   */
1674  public static final String SCM_UPLOADER_SERVER_THREAD_COUNT =
1675      SHARED_CACHE_PREFIX + "uploader.server.thread-count";
1676  public static final int DEFAULT_SCM_UPLOADER_SERVER_THREAD_COUNT = 50;
1677
1678  /** The address of the client interface in the SCM. */
1679  public static final String SCM_CLIENT_SERVER_ADDRESS =
1680      SHARED_CACHE_PREFIX + "client-server.address";
1681  public static final int DEFAULT_SCM_CLIENT_SERVER_PORT = 8045;
1682  public static final String DEFAULT_SCM_CLIENT_SERVER_ADDRESS = "0.0.0.0:"
1683      + DEFAULT_SCM_CLIENT_SERVER_PORT;
1684
1685  /** The number of threads used to handle shared cache manager requests. */
1686  public static final String SCM_CLIENT_SERVER_THREAD_COUNT =
1687      SHARED_CACHE_PREFIX + "client-server.thread-count";
1688  public static final int DEFAULT_SCM_CLIENT_SERVER_THREAD_COUNT = 50;
1689
1690  /** the checksum algorithm implementation **/
1691  public static final String SHARED_CACHE_CHECKSUM_ALGO_IMPL =
1692      SHARED_CACHE_PREFIX + "checksum.algo.impl";
1693  public static final String DEFAULT_SHARED_CACHE_CHECKSUM_ALGO_IMPL =
1694      "org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl";
1695
1696  // node manager (uploader) configs
1697  /**
1698   * The replication factor for the node manager uploader for the shared cache.
1699   */
1700  public static final String SHARED_CACHE_NM_UPLOADER_REPLICATION_FACTOR =
1701      SHARED_CACHE_PREFIX + "nm.uploader.replication.factor";
1702  public static final int DEFAULT_SHARED_CACHE_NM_UPLOADER_REPLICATION_FACTOR =
1703      10;
1704
1705  public static final String SHARED_CACHE_NM_UPLOADER_THREAD_COUNT =
1706      SHARED_CACHE_PREFIX + "nm.uploader.thread-count";
1707  public static final int DEFAULT_SHARED_CACHE_NM_UPLOADER_THREAD_COUNT = 20;
1708
1709  ////////////////////////////////
1710  // Other Configs
1711  ////////////////////////////////
1712
1713  /**
1714   * Use YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS instead.
1715   * The interval of the yarn client's querying application state after
1716   * application submission. The unit is millisecond.
1717   */
1718  @Deprecated
1719  public static final String YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS =
1720      YARN_PREFIX + "client.app-submission.poll-interval";
1721
1722  /**
1723   * The interval that the yarn client library uses to poll the completion
1724   * status of the asynchronous API of application client protocol.
1725   */
1726  public static final String YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS =
1727      YARN_PREFIX + "client.application-client-protocol.poll-interval-ms";
1728  public static final long DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS =
1729      200;
1730
1731  /**
1732   * The duration that the yarn client library waits, cumulatively across polls,
1733   * for an expected state change to occur. Defaults to -1, which indicates no
1734   * limit.
1735   */
1736  public static final String YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_TIMEOUT_MS =
1737      YARN_PREFIX + "client.application-client-protocol.poll-timeout-ms";
1738  public static final long DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_TIMEOUT_MS =
1739      -1;
1740
1741  /**
1742   * Max number of threads in NMClientAsync to process container management
1743   * events
1744   */
1745  public static final String NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE =
1746      YARN_PREFIX + "client.nodemanager-client-async.thread-pool-max-size";
1747  public static final int DEFAULT_NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE = 500;
1748
1749  /**
1750   * Maximum number of proxy connections to cache for node managers. If set
1751   * to a value greater than zero then the cache is enabled and the NMClient
1752   * and MRAppMaster will cache the specified number of node manager proxies.
1753   * There will be at max one proxy per node manager. Ex. configuring it to a
1754   * value of 5 will make sure that client will at max have 5 proxies cached
1755   * with 5 different node managers. These connections for these proxies will
1756   * be timed out if idle for more than the system wide idle timeout period.
1757   * Note that this could cause issues on large clusters as many connections
1758   * could linger simultaneously and lead to a large number of connection
1759   * threads. The token used for authentication will be used only at
1760   * connection creation time. If a new token is received then the earlier
1761   * connection should be closed in order to use the new token. This and
1762   * {@link YarnConfiguration#NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE} are related
1763   * and should be in sync (no need for them to be equal).
1764   * If the value of this property is zero then the connection cache is
1765   * disabled and connections will use a zero idle timeout to prevent too
1766   * many connection threads on large clusters.
1767   */
1768  public static final String NM_CLIENT_MAX_NM_PROXIES =
1769      YARN_PREFIX + "client.max-cached-nodemanagers-proxies";
1770  public static final int DEFAULT_NM_CLIENT_MAX_NM_PROXIES = 0;
1771
1772  /** Max time to wait to establish a connection to NM */
1773  public static final String CLIENT_NM_CONNECT_MAX_WAIT_MS =
1774      YARN_PREFIX + "client.nodemanager-connect.max-wait-ms";
1775  public static final long DEFAULT_CLIENT_NM_CONNECT_MAX_WAIT_MS =
1776      3 * 60 * 1000;
1777
1778  /** Time interval between each attempt to connect to NM */
1779  public static final String CLIENT_NM_CONNECT_RETRY_INTERVAL_MS =
1780      YARN_PREFIX + "client.nodemanager-connect.retry-interval-ms";
1781  public static final long DEFAULT_CLIENT_NM_CONNECT_RETRY_INTERVAL_MS
1782      = 10 * 1000;
1783
1784  public static final String YARN_HTTP_POLICY_KEY = YARN_PREFIX + "http.policy";
1785  public static final String YARN_HTTP_POLICY_DEFAULT = HttpConfig.Policy.HTTP_ONLY
1786      .name();
1787  
1788  /**
1789   * Node-labels configurations
1790   */
1791  public static final String NODE_LABELS_PREFIX = YARN_PREFIX + "node-labels.";
1792  
1793  /** URI for NodeLabelManager */
1794  public static final String FS_NODE_LABELS_STORE_ROOT_DIR = NODE_LABELS_PREFIX
1795      + "fs-store.root-dir";
1796  public static final String FS_NODE_LABELS_STORE_RETRY_POLICY_SPEC =
1797      NODE_LABELS_PREFIX + "fs-store.retry-policy-spec";
1798  public static final String DEFAULT_FS_NODE_LABELS_STORE_RETRY_POLICY_SPEC =
1799      "2000, 500";
1800  
1801  /**
1802   * Flag to indicate if the node labels feature enabled, by default it's
1803   * disabled
1804   */
1805  public static final String NODE_LABELS_ENABLED = NODE_LABELS_PREFIX
1806      + "enabled";
1807  public static final boolean DEFAULT_NODE_LABELS_ENABLED = false;
1808
1809  public YarnConfiguration() {
1810    super();
1811  }
1812  
1813  public YarnConfiguration(Configuration conf) {
1814    super(conf);
1815    if (! (conf instanceof YarnConfiguration)) {
1816      this.reloadConfiguration();
1817    }
1818  }
1819
1820  @Private
1821  public static List<String> getServiceAddressConfKeys(Configuration conf) {
1822    return useHttps(conf) ? RM_SERVICES_ADDRESS_CONF_KEYS_HTTPS
1823        : RM_SERVICES_ADDRESS_CONF_KEYS_HTTP;
1824  }
1825
1826  /**
1827   * Get the socket address for <code>name</code> property as a
1828   * <code>InetSocketAddress</code>. On a HA cluster,
1829   * this fetches the address corresponding to the RM identified by
1830   * {@link #RM_HA_ID}.
1831   * @param name property name.
1832   * @param defaultAddress the default value
1833   * @param defaultPort the default port
1834   * @return InetSocketAddress
1835   */
1836  @Override
1837  public InetSocketAddress getSocketAddr(
1838      String name, String defaultAddress, int defaultPort) {
1839    String address;
1840    if (HAUtil.isHAEnabled(this) && getServiceAddressConfKeys(this).contains(name)) {
1841      address = HAUtil.getConfValueForRMInstance(name, defaultAddress, this);
1842    } else {
1843      address = get(name, defaultAddress);
1844    }
1845    return NetUtils.createSocketAddr(address, defaultPort, name);
1846  }
1847
1848  @Override
1849  public InetSocketAddress updateConnectAddr(String name,
1850                                             InetSocketAddress addr) {
1851    String prefix = name;
1852    if (HAUtil.isHAEnabled(this)) {
1853      prefix = HAUtil.addSuffix(prefix, HAUtil.getRMHAId(this));
1854    }
1855    return super.updateConnectAddr(prefix, addr);
1856  }
1857
1858  @Private
1859  public static int getRMDefaultPortNumber(String addressPrefix,
1860      Configuration conf) {
1861    if (addressPrefix.equals(YarnConfiguration.RM_ADDRESS)) {
1862      return YarnConfiguration.DEFAULT_RM_PORT;
1863    } else if (addressPrefix.equals(YarnConfiguration.RM_SCHEDULER_ADDRESS)) {
1864      return YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT;
1865    } else if (addressPrefix.equals(YarnConfiguration.RM_WEBAPP_ADDRESS)) {
1866      return YarnConfiguration.DEFAULT_RM_WEBAPP_PORT;
1867    } else if (addressPrefix.equals(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS)) {
1868      return YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT;
1869    } else if (addressPrefix
1870        .equals(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS)) {
1871      return YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_PORT;
1872    } else if (addressPrefix.equals(YarnConfiguration.RM_ADMIN_ADDRESS)) {
1873      return YarnConfiguration.DEFAULT_RM_ADMIN_PORT;
1874    } else {
1875      throw new HadoopIllegalArgumentException(
1876          "Invalid RM RPC address Prefix: " + addressPrefix
1877              + ". The valid value should be one of "
1878              + getServiceAddressConfKeys(conf));
1879    }
1880  }
1881
1882  public static boolean useHttps(Configuration conf) {
1883    return HttpConfig.Policy.HTTPS_ONLY == HttpConfig.Policy.fromString(conf
1884        .get(YARN_HTTP_POLICY_KEY,
1885            YARN_HTTP_POLICY_DEFAULT));
1886  }
1887
1888  public static boolean shouldRMFailFast(Configuration conf) {
1889    return conf.getBoolean(YarnConfiguration.RM_FAIL_FAST,
1890        conf.getBoolean(YarnConfiguration.YARN_FAIL_FAST,
1891            YarnConfiguration.DEFAULT_YARN_FAIL_FAST));
1892  }
1893
1894  @Private
1895  public static String getClusterId(Configuration conf) {
1896    String clusterId = conf.get(YarnConfiguration.RM_CLUSTER_ID);
1897    if (clusterId == null) {
1898      throw new HadoopIllegalArgumentException("Configuration doesn't specify " +
1899          YarnConfiguration.RM_CLUSTER_ID);
1900    }
1901    return clusterId;
1902  }
1903}