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