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