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