Factory to create client IPC classes. yarn.ipc.client.factory.class Factory to create server IPC classes. yarn.ipc.server.factory.class Factory to create serializeable records. yarn.ipc.record.factory.class RPC class implementation yarn.ipc.rpc.class org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC The hostname of the RM. yarn.resourcemanager.hostname 0.0.0.0 The address of the applications manager interface in the RM. yarn.resourcemanager.address ${yarn.resourcemanager.hostname}:8032 The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in yarn.resourcemanager.address and yarn.resourcemanager.webapp.address, respectively. This is most useful for making RM listen to all interfaces by setting to 0.0.0.0. yarn.resourcemanager.bind-host The number of threads used to handle applications manager requests. yarn.resourcemanager.client.thread-count 50 Number of threads used to launch/cleanup AM. yarn.resourcemanager.amlauncher.thread-count 50 Retry times to connect with NM. yarn.resourcemanager.nodemanager-connect-retries 10 Timeout in milliseconds when YARN dispatcher tries to drain the events. Typically, this happens when service is stopping. e.g. RM drains the ATS events dispatcher when stopping. yarn.dispatcher.drain-events.timeout 300000 The expiry interval for application master reporting. yarn.am.liveness-monitor.expiry-interval-ms 600000 The Kerberos principal for the resource manager. yarn.resourcemanager.principal The address of the scheduler interface. yarn.resourcemanager.scheduler.address ${yarn.resourcemanager.hostname}:8030 Number of threads to handle scheduler interface. yarn.resourcemanager.scheduler.client.thread-count 50 This configures the HTTP endpoint for Yarn Daemons.The following values are supported: - HTTP_ONLY : Service is provided only on http - HTTPS_ONLY : Service is provided only on https yarn.http.policy HTTP_ONLY The http address of the RM web application. If only a host is provided as the value, the webapp will be served on a random port. yarn.resourcemanager.webapp.address ${yarn.resourcemanager.hostname}:8088 The https address of the RM web application. If only a host is provided as the value, the webapp will be served on a random port. yarn.resourcemanager.webapp.https.address ${yarn.resourcemanager.hostname}:8090 The Kerberos keytab file to be used for spnego filter for the RM web interface. yarn.resourcemanager.webapp.spnego-keytab-file The Kerberos principal to be used for spnego filter for the RM web interface. yarn.resourcemanager.webapp.spnego-principal Add button to kill application in the RM Application view. yarn.resourcemanager.webapp.ui-actions.enabled true yarn.resourcemanager.resource-tracker.address ${yarn.resourcemanager.hostname}:8031 Are acls enabled. yarn.acl.enable false Are reservation acls enabled. yarn.acl.reservation-enable false ACL of who can be admin of the YARN cluster. yarn.admin.acl * The address of the RM admin interface. yarn.resourcemanager.admin.address ${yarn.resourcemanager.hostname}:8033 Number of threads used to handle RM admin interface. yarn.resourcemanager.admin.client.thread-count 1 Maximum time to wait to establish connection to ResourceManager. yarn.resourcemanager.connect.max-wait.ms 900000 How often to try connecting to the ResourceManager. yarn.resourcemanager.connect.retry-interval.ms 30000 The maximum number of application attempts. It's a global setting for all application masters. Each application master can specify its individual maximum number of application attempts via the API, but the individual number cannot be more than the global upper bound. If it is, the resourcemanager will override it. The default number is set to 2, to allow at least one retry for AM. yarn.resourcemanager.am.max-attempts 2 How often to check that containers are still alive. yarn.resourcemanager.container.liveness-monitor.interval-ms 600000 The keytab for the resource manager. yarn.resourcemanager.keytab /etc/krb5.keytab Flag to enable override of the default kerberos authentication filter with the RM authentication filter to allow authentication using delegation tokens(fallback to kerberos if the tokens are missing). Only applicable when the http authentication type is kerberos. yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled true Flag to enable cross-origin (CORS) support in the RM. This flag requires the CORS filter initializer to be added to the filter initializers list in core-site.xml. yarn.resourcemanager.webapp.cross-origin.enabled false How long to wait until a node manager is considered dead. yarn.nm.liveness-monitor.expiry-interval-ms 600000 Path to file with nodes to include. yarn.resourcemanager.nodes.include-path Path to file with nodes to exclude. yarn.resourcemanager.nodes.exclude-path The expiry interval for node IP caching. -1 disables the caching yarn.resourcemanager.node-ip-cache.expiry-interval-secs -1 Number of threads to handle resource tracker calls. yarn.resourcemanager.resource-tracker.client.thread-count 50 The class to use as the resource scheduler. yarn.resourcemanager.scheduler.class org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler The minimum allocation for every container request at the RM, in MBs. Memory requests lower than this will throw a InvalidResourceRequestException. yarn.scheduler.minimum-allocation-mb 1024 The maximum allocation for every container request at the RM, in MBs. Memory requests higher than this will throw a InvalidResourceRequestException. yarn.scheduler.maximum-allocation-mb 8192 The minimum allocation for every container request at the RM, in terms of virtual CPU cores. Requests lower than this will throw a InvalidResourceRequestException. yarn.scheduler.minimum-allocation-vcores 1 The maximum allocation for every container request at the RM, in terms of virtual CPU cores. Requests higher than this will throw a InvalidResourceRequestException. yarn.scheduler.maximum-allocation-vcores 4 Used by node labels. If set to true, the port should be included in the node name. Only usable if your scheduler supports node labels. yarn.scheduler.include-port-in-node-name false Enable RM to recover state after starting. If true, then yarn.resourcemanager.store.class must be specified. yarn.resourcemanager.recovery.enabled false Should RM fail fast if it encounters any errors. By defalt, it points to ${yarn.fail-fast}. Errors include: 1) exceptions when state-store write/read operations fails. yarn.resourcemanager.fail-fast ${yarn.fail-fast} Should YARN fail fast if it encounters any errors. This is a global config for all other components including RM,NM etc. If no value is set for component-specific config (e.g yarn.resourcemanager.fail-fast), this value will be the default. yarn.fail-fast false Enable RM work preserving recovery. This configuration is private to YARN for experimenting the feature. yarn.resourcemanager.work-preserving-recovery.enabled true Set the amount of time RM waits before allocating new containers on work-preserving-recovery. Such wait period gives RM a chance to settle down resyncing with NMs in the cluster on recovery, before assigning new containers to applications. yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms 10000 The class to use as the persistent store. If org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore is used, the store is implicitly fenced; meaning a single ResourceManager is able to use the store at any point in time. More details on this implicit fencing, along with setting up appropriate ACLs is discussed under yarn.resourcemanager.zk-state-store.root-node.acl. yarn.resourcemanager.store.class org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore When automatic failover is enabled, number of zookeeper operation retry times in ActiveStandbyElector yarn.resourcemanager.ha.failover-controller.active-standby-elector.zk.retries The maximum number of completed applications RM state store keeps, less than or equals to ${yarn.resourcemanager.max-completed-applications}. By default, it equals to ${yarn.resourcemanager.max-completed-applications}. This ensures that the applications kept in the state store are consistent with the applications remembered in RM memory. Any values larger than ${yarn.resourcemanager.max-completed-applications} will be reset to ${yarn.resourcemanager.max-completed-applications}. Note that this value impacts the RM recovery performance.Typically, a smaller value indicates better performance on RM recovery. yarn.resourcemanager.state-store.max-completed-applications ${yarn.resourcemanager.max-completed-applications} Host:Port of the ZooKeeper server to be used by the RM. This must be supplied when using the ZooKeeper based implementation of the RM state store and/or embedded automatic failover in a HA setting. yarn.resourcemanager.zk-address Number of times RM tries to connect to ZooKeeper. yarn.resourcemanager.zk-num-retries 1000 Retry interval in milliseconds when connecting to ZooKeeper. When HA is enabled, the value here is NOT used. It is generated automatically from yarn.resourcemanager.zk-timeout-ms and yarn.resourcemanager.zk-num-retries. yarn.resourcemanager.zk-retry-interval-ms 1000 Full path of the ZooKeeper znode where RM state will be stored. This must be supplied when using org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore as the value for yarn.resourcemanager.store.class yarn.resourcemanager.zk-state-store.parent-path /rmstore ZooKeeper session timeout in milliseconds. Session expiration is managed by the ZooKeeper cluster itself, not by the client. This value is used by the cluster to determine when the client's session expires. Expirations happens when the cluster does not hear from the client within the specified session timeout period (i.e. no heartbeat). yarn.resourcemanager.zk-timeout-ms 10000 ACL's to be used for ZooKeeper znodes. yarn.resourcemanager.zk-acl world:anyone:rwcda ACLs to be used for the root znode when using ZKRMStateStore in a HA scenario for fencing. ZKRMStateStore supports implicit fencing to allow a single ResourceManager write-access to the store. For fencing, the ResourceManagers in the cluster share read-write-admin privileges on the root node, but the Active ResourceManager claims exclusive create-delete permissions. By default, when this property is not set, we use the ACLs from yarn.resourcemanager.zk-acl for shared admin access and rm-address:random-number for username-based exclusive create-delete access. This property allows users to set ACLs of their choice instead of using the default mechanism. For fencing to work, the ACLs should be carefully set differently on each ResourceManger such that all the ResourceManagers have shared admin access and the Active ResourceManger takes over (exclusively) the create-delete access. yarn.resourcemanager.zk-state-store.root-node.acl Specify the auths to be used for the ACL's specified in both the yarn.resourcemanager.zk-acl and yarn.resourcemanager.zk-state-store.root-node.acl properties. This takes a comma-separated list of authentication mechanisms, each of the form 'scheme:auth' (the same syntax used for the 'addAuth' command in the ZK CLI). yarn.resourcemanager.zk-auth URI pointing to the location of the FileSystem path where RM state will be stored. This must be supplied when using org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore as the value for yarn.resourcemanager.store.class yarn.resourcemanager.fs.state-store.uri ${hadoop.tmp.dir}/yarn/system/rmstore hdfs client retry policy specification. hdfs client retry is always enabled. Specified in pairs of sleep-time and number-of-retries and (t0, n0), (t1, n1), ..., the first n0 retries sleep t0 milliseconds on average, the following n1 retries sleep t1 milliseconds on average, and so on. yarn.resourcemanager.fs.state-store.retry-policy-spec 2000, 500 the number of retries to recover from IOException in FileSystemRMStateStore. yarn.resourcemanager.fs.state-store.num-retries 0 Retry interval in milliseconds in FileSystemRMStateStore. yarn.resourcemanager.fs.state-store.retry-interval-ms 1000 Local path where the RM state will be stored when using org.apache.hadoop.yarn.server.resourcemanager.recovery.LeveldbRMStateStore as the value for yarn.resourcemanager.store.class yarn.resourcemanager.leveldb-state-store.path ${hadoop.tmp.dir}/yarn/system/rmstore The time in seconds between full compactions of the leveldb database. Setting the interval to zero disables the full compaction cycles. yarn.resourcemanager.leveldb-state-store.compaction-interval-secs 3600 Enable RM high-availability. When enabled, (1) The RM starts in the Standby mode by default, and transitions to the Active mode when prompted to. (2) The nodes in the RM ensemble are listed in yarn.resourcemanager.ha.rm-ids (3) The id of each RM either comes from yarn.resourcemanager.ha.id if yarn.resourcemanager.ha.id is explicitly specified or can be figured out by matching yarn.resourcemanager.address.{id} with local address (4) The actual physical addresses come from the configs of the pattern - {rpc-config}.{id} yarn.resourcemanager.ha.enabled false Enable automatic failover. By default, it is enabled only when HA is enabled yarn.resourcemanager.ha.automatic-failover.enabled true Enable embedded automatic failover. By default, it is enabled only when HA is enabled. The embedded elector relies on the RM state store to handle fencing, and is primarily intended to be used in conjunction with ZKRMStateStore. yarn.resourcemanager.ha.automatic-failover.embedded true The base znode path to use for storing leader information, when using ZooKeeper based leader election. yarn.resourcemanager.ha.automatic-failover.zk-base-path /yarn-leader-election Name of the cluster. In a HA setting, this is used to ensure the RM participates in leader election for this cluster and ensures it does not affect other clusters yarn.resourcemanager.cluster-id The list of RM nodes in the cluster when HA is enabled. See description of yarn.resourcemanager.ha .enabled for full details on how this is used. yarn.resourcemanager.ha.rm-ids The id (string) of the current RM. When HA is enabled, this is an optional config. The id of current RM can be set by explicitly specifying yarn.resourcemanager.ha.id or figured out by matching yarn.resourcemanager.address.{id} with local address See description of yarn.resourcemanager.ha.enabled for full details on how this is used. yarn.resourcemanager.ha.id When HA is enabled, the class to be used by Clients, AMs and NMs to failover to the Active RM. It should extend org.apache.hadoop.yarn.client.RMFailoverProxyProvider yarn.client.failover-proxy-provider org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider When HA is enabled, the max number of times FailoverProxyProvider should attempt failover. When set, this overrides the yarn.resourcemanager.connect.max-wait.ms. When not set, this is inferred from yarn.resourcemanager.connect.max-wait.ms. yarn.client.failover-max-attempts When HA is enabled, the sleep base (in milliseconds) to be used for calculating the exponential delay between failovers. When set, this overrides the yarn.resourcemanager.connect.* settings. When not set, yarn.resourcemanager.connect.retry-interval.ms is used instead. yarn.client.failover-sleep-base-ms When HA is enabled, the maximum sleep time (in milliseconds) between failovers. When set, this overrides the yarn.resourcemanager.connect.* settings. When not set, yarn.resourcemanager.connect.retry-interval.ms is used instead. yarn.client.failover-sleep-max-ms When HA is enabled, the number of retries per attempt to connect to a ResourceManager. In other words, it is the ipc.client.connect.max.retries to be used during failover attempts yarn.client.failover-retries 0 When HA is enabled, the number of retries per attempt to connect to a ResourceManager on socket timeouts. In other words, it is the ipc.client.connect.max.retries.on.timeouts to be used during failover attempts yarn.client.failover-retries-on-socket-timeouts 0 The maximum number of completed applications RM keeps. yarn.resourcemanager.max-completed-applications 10000 Interval at which the delayed token removal thread runs yarn.resourcemanager.delayed.delegation-token.removal-interval-ms 30000 If true, ResourceManager will have proxy-user privileges. Use case: In a secure cluster, YARN requires the user hdfs delegation-tokens to do localization and log-aggregation on behalf of the user. If this is set to true, ResourceManager is able to request new hdfs delegation tokens on behalf of the user. This is needed by long-running-service, because the hdfs tokens will eventually expire and YARN requires new valid tokens to do localization and log-aggregation. Note that to enable this use case, the corresponding HDFS NameNode has to configure ResourceManager as the proxy-user so that ResourceManager can itself ask for new tokens on behalf of the user when tokens are past their max-life-time. yarn.resourcemanager.proxy-user-privileges.enabled false Interval for the roll over for the master key used to generate application tokens yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs 86400 Interval for the roll over for the master key used to generate container tokens. It is expected to be much greater than yarn.nm.liveness-monitor.expiry-interval-ms and yarn.resourcemanager.rm.container-allocation.expiry-interval-ms. Otherwise the behavior is undefined. yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs 86400 The heart-beat interval in milliseconds for every NodeManager in the cluster. yarn.resourcemanager.nodemanagers.heartbeat-interval-ms 1000 The minimum allowed version of a connecting nodemanager. The valid values are NONE (no version checking), EqualToRM (the nodemanager's version is equal to or greater than the RM version), or a Version String. yarn.resourcemanager.nodemanager.minimum.version NONE Enable a set of periodic monitors (specified in yarn.resourcemanager.scheduler.monitor.policies) that affect the scheduler. yarn.resourcemanager.scheduler.monitor.enable false The list of SchedulingEditPolicy classes that interact with the scheduler. A particular module may be incompatible with the scheduler, other policies, or a configuration of either. yarn.resourcemanager.scheduler.monitor.policies org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy The class to use as the configuration provider. If org.apache.hadoop.yarn.LocalConfigurationProvider is used, the local configuration will be loaded. If org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider is used, the configuration which will be loaded should be uploaded to remote File system first. yarn.resourcemanager.configuration.provider-class org.apache.hadoop.yarn.LocalConfigurationProvider The value specifies the file system (e.g. HDFS) path where ResourceManager loads configuration if yarn.resourcemanager.configuration.provider-class is set to org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider. yarn.resourcemanager.configuration.file-system-based-store /yarn/conf The setting that controls whether yarn system metrics is published on the timeline server or not by RM. yarn.resourcemanager.system-metrics-publisher.enabled false Number of worker threads that send the yarn system metrics data. yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size 10 Number of diagnostics/failure messages can be saved in RM for log aggregation. It also defines the number of diagnostics/failure messages can be shown in log aggregation web ui. yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory 10 RM DelegationTokenRenewer thread count yarn.resourcemanager.delegation-token-renewer.thread-count 50 RM secret key update interval in ms yarn.resourcemanager.delegation.key.update-interval 86400000 RM delegation token maximum lifetime in ms yarn.resourcemanager.delegation.token.max-lifetime 604800000 RM delegation token update interval in ms yarn.resourcemanager.delegation.token.renew-interval 86400000 Thread pool size for RMApplicationHistoryWriter. yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size 10 Comma-separated list of values (in minutes) for schedule queue related metrics. yarn.resourcemanager.metrics.runtime.buckets 60,300,1440 Interval for the roll over for the master key used to generate NodeManager tokens. It is expected to be set to a value much larger than yarn.nm.liveness-monitor.expiry-interval-ms. yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs 86400 Flag to enable the ResourceManager reservation system. yarn.resourcemanager.reservation-system.enable false The Java class to use as the ResourceManager reservation system. By default, is set to org.apache.hadoop.yarn.server.resourcemanager.reservation.CapacityReservationSystem when using CapacityScheduler and is set to org.apache.hadoop.yarn.server.resourcemanager.reservation.FairReservationSystem when using FairScheduler. yarn.resourcemanager.reservation-system.class The plan follower policy class name to use for the ResourceManager reservation system. By default, is set to org.apache.hadoop.yarn.server.resourcemanager.reservation.CapacitySchedulerPlanFollower is used when using CapacityScheduler, and is set to org.apache.hadoop.yarn.server.resourcemanager.reservation.FairSchedulerPlanFollower when using FairScheduler. yarn.resourcemanager.reservation-system.plan.follower Step size of the reservation system in ms yarn.resourcemanager.reservation-system.planfollower.time-step 1000 The expiry interval for a container yarn.resourcemanager.rm.container-allocation.expiry-interval-ms 600000 The hostname of the NM. yarn.nodemanager.hostname 0.0.0.0 The address of the container manager in the NM. yarn.nodemanager.address ${yarn.nodemanager.hostname}:0 The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in yarn.nodemanager.address and yarn.nodemanager.webapp.address, respectively. This is most useful for making NM listen to all interfaces by setting to 0.0.0.0. yarn.nodemanager.bind-host Environment variables that should be forwarded from the NodeManager's environment to the container's. yarn.nodemanager.admin-env MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX Environment variables that containers may override rather than use NodeManager's default. yarn.nodemanager.env-whitelist JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME who will execute(launch) the containers. yarn.nodemanager.container-executor.class org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor Number of threads container manager uses. yarn.nodemanager.container-manager.thread-count 20 Number of threads used in cleanup. yarn.nodemanager.delete.thread-count 4 Number of seconds after an application finishes before the nodemanager's DeletionService will delete the application's localized file directory and log directory. To diagnose Yarn application problems, set this property's value large enough (for example, to 600 = 10 minutes) to permit examination of these directories. After changing the property's value, you must restart the nodemanager in order for it to have an effect. The roots of Yarn applications' work directories is configurable with the yarn.nodemanager.local-dirs property (see below), and the roots of the Yarn applications' log directories is configurable with the yarn.nodemanager.log-dirs property (see also below). yarn.nodemanager.delete.debug-delay-sec 0 Keytab for NM. yarn.nodemanager.keytab /etc/krb5.keytab List of directories to store localized files in. An application's localized file directory will be found in: ${yarn.nodemanager.local-dirs}/usercache/${user}/appcache/application_${appid}. Individual containers' work directories, called container_${contid}, will be subdirectories of this. yarn.nodemanager.local-dirs ${hadoop.tmp.dir}/nm-local-dir It limits the maximum number of files which will be localized in a single local directory. If the limit is reached then sub-directories will be created and new files will be localized in them. If it is set to a value less than or equal to 36 [which are sub-directories (0-9 and then a-z)] then NodeManager will fail to start. For example; [for public cache] if this is configured with a value of 40 ( 4 files + 36 sub-directories) and the local-dir is "/tmp/local-dir1" then it will allow 4 files to be created directly inside "/tmp/local-dir1/filecache". For files that are localized further it will create a sub-directory "0" inside "/tmp/local-dir1/filecache" and will localize files inside it until it becomes full. If a file is removed from a sub-directory that is marked full, then that sub-directory will be used back again to localize files. yarn.nodemanager.local-cache.max-files-per-directory 8192 Address where the localizer IPC is. yarn.nodemanager.localizer.address ${yarn.nodemanager.hostname}:8040 Interval in between cache cleanups. yarn.nodemanager.localizer.cache.cleanup.interval-ms 600000 Target size of localizer cache in MB, per nodemanager. It is a target retention size that only includes resources with PUBLIC and PRIVATE visibility and excludes resources with APPLICATION visibility yarn.nodemanager.localizer.cache.target-size-mb 10240 Number of threads to handle localization requests. yarn.nodemanager.localizer.client.thread-count 5 Number of threads to use for localization fetching. yarn.nodemanager.localizer.fetch.thread-count 4 yarn.nodemanager.container-localizer.java.opts -Xmx256m Where to store container logs. An application's localized log directory will be found in ${yarn.nodemanager.log-dirs}/application_${appid}. Individual containers' log directories will be below this, in directories named container_{$contid}. Each container directory will contain the files stderr, stdin, and syslog generated by that container. yarn.nodemanager.log-dirs ${yarn.log.dir}/userlogs Whether to enable log aggregation. Log aggregation collects each container's logs and moves these logs onto a file-system, for e.g. HDFS, after the application completes. Users can configure the "yarn.nodemanager.remote-app-log-dir" and "yarn.nodemanager.remote-app-log-dir-suffix" properties to determine where these logs are moved to. Users can access the logs via the Application Timeline Server. yarn.log-aggregation-enable false How long to keep aggregation logs before deleting them. -1 disables. Be careful set this too small and you will spam the name node. yarn.log-aggregation.retain-seconds -1 How long to wait between aggregated log retention checks. If set to 0 or a negative value then the value is computed as one-tenth of the aggregated log retention time. Be careful set this too small and you will spam the name node. yarn.log-aggregation.retain-check-interval-seconds -1 How long for ResourceManager to wait for NodeManager to report its log aggregation status. If waiting time of which the log aggregation status is reported from NodeManager exceeds the configured value, RM will report log aggregation status for this NodeManager as TIME_OUT yarn.log-aggregation-status.time-out.ms 600000 Time in seconds to retain user logs. Only applicable if log aggregation is disabled yarn.nodemanager.log.retain-seconds 10800 Where to aggregate logs to. yarn.nodemanager.remote-app-log-dir /tmp/logs The remote log dir will be created at {yarn.nodemanager.remote-app-log-dir}/${user}/{thisParam} yarn.nodemanager.remote-app-log-dir-suffix logs Generate additional logs about container launches. Currently, this creates a copy of the launch script and lists the directory contents of the container work dir. When listing directory contents, we follow symlinks to a max-depth of 5(including symlinks which point to outside the container work dir) which may lead to a slowness in launching containers. yarn.nodemanager.log-container-debug-info.enabled false Amount of physical memory, in MB, that can be allocated for containers. If set to -1 and yarn.nodemanager.resource.detect-hardware-capabilities is true, it is automatically calculated(in case of Windows and Linux). In other cases, the default is 8192MB. yarn.nodemanager.resource.memory-mb -1 Amount of physical memory, in MB, that is reserved for non-YARN processes. This configuration is only used if yarn.nodemanager.resource.detect-hardware-capabilities is set to true and yarn.nodemanager.resource.memory-mb is -1. If set to -1, this amount is calculated as 20% of (system memory - 2*HADOOP_HEAPSIZE) yarn.nodemanager.resource.system-reserved-memory-mb -1 Whether physical memory limits will be enforced for containers. yarn.nodemanager.pmem-check-enabled true Whether virtual memory limits will be enforced for containers. yarn.nodemanager.vmem-check-enabled true Ratio between virtual memory to physical memory when setting memory limits for containers. Container allocations are expressed in terms of physical memory, and virtual memory usage is allowed to exceed this allocation by this ratio. yarn.nodemanager.vmem-pmem-ratio 2.1 Number of vcores that can be allocated for containers. This is used by the RM scheduler when allocating resources for containers. This is not used to limit the number of CPUs used by YARN containers. If it is set to -1 and yarn.nodemanager.resource.detect-hardware-capabilities is true, it is automatically determined from the hardware in case of Windows and Linux. In other cases, number of vcores is 8 by default. yarn.nodemanager.resource.cpu-vcores -1 Flag to determine if logical processors(such as hyperthreads) should be counted as cores. Only applicable on Linux when yarn.nodemanager.resource.cpu-vcores is set to -1 and yarn.nodemanager.resource.detect-hardware-capabilities is true. yarn.nodemanager.resource.count-logical-processors-as-cores false Multiplier to determine how to convert phyiscal cores to vcores. This value is used if yarn.nodemanager.resource.cpu-vcores is set to -1(which implies auto-calculate vcores) and yarn.nodemanager.resource.detect-hardware-capabilities is set to true. The number of vcores will be calculated as number of CPUs * multiplier. yarn.nodemanager.resource.pcores-vcores-multiplier 1.0 Percentage of CPU that can be allocated for containers. This setting allows users to limit the amount of CPU that YARN containers use. Currently functional only on Linux using cgroups. The default is to use 100% of CPU. yarn.nodemanager.resource.percentage-physical-cpu-limit 100 Enable auto-detection of node capabilities such as memory and CPU. yarn.nodemanager.resource.detect-hardware-capabilities false NM Webapp address. yarn.nodemanager.webapp.address ${yarn.nodemanager.hostname}:8042 The https adddress of the NM web application. yarn.nodemanager.webapp.https.address 0.0.0.0:8044 The Kerberos keytab file to be used for spnego filter for the NM web interface. yarn.nodemanager.webapp.spnego-keytab-file The Kerberos principal to be used for spnego filter for the NM web interface. yarn.nodemanager.webapp.spnego-principal How often to monitor the node and the containers. yarn.nodemanager.resource-monitor.interval-ms 3000 Class that calculates current resource utilization. yarn.nodemanager.resource-calculator.class How often to monitor containers. If not set, the value for yarn.nodemanager.resource-monitor.interval-ms will be used. yarn.nodemanager.container-monitor.interval-ms Class that calculates containers current resource utilization. If not set, the value for yarn.nodemanager.resource-calculator.class will be used. yarn.nodemanager.container-monitor.resource-calculator.class Frequency of running node health script. yarn.nodemanager.health-checker.interval-ms 600000 Script time out period. yarn.nodemanager.health-checker.script.timeout-ms 1200000 The health check script to run. yarn.nodemanager.health-checker.script.path The arguments to pass to the health check script. yarn.nodemanager.health-checker.script.opts Frequency of running disk health checker code. yarn.nodemanager.disk-health-checker.interval-ms 120000 The minimum fraction of number of disks to be healthy for the nodemanager to launch new containers. This correspond to both yarn.nodemanager.local-dirs and yarn.nodemanager.log-dirs. i.e. If there are less number of healthy local-dirs (or log-dirs) available, then new containers will not be launched on this node. yarn.nodemanager.disk-health-checker.min-healthy-disks 0.25 The maximum percentage of disk space utilization allowed after which a disk is marked as bad. Values can range from 0.0 to 100.0. If the value is greater than or equal to 100, the nodemanager will check for full disk. This applies to yarn.nodemanager.local-dirs and yarn.nodemanager.log-dirs. yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage 90.0 The low threshold percentage of disk space used when a bad disk is marked as good. Values can range from 0.0 to 100.0. This applies to yarn.nodemanager.local-dirs and yarn.nodemanager.log-dirs. Note that if its value is more than yarn.nodemanager.disk-health-checker. max-disk-utilization-per-disk-percentage or not set, it will be set to the same value as yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage. yarn.nodemanager.disk-health-checker.disk-utilization-watermark-low-per-disk-percentage The minimum space that must be available on a disk for it to be used. This applies to yarn.nodemanager.local-dirs and yarn.nodemanager.log-dirs. yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb 0 The path to the Linux container executor. yarn.nodemanager.linux-container-executor.path The class which should help the LCE handle resources. yarn.nodemanager.linux-container-executor.resources-handler.class org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler The cgroups hierarchy under which to place YARN proccesses (cannot contain commas). If yarn.nodemanager.linux-container-executor.cgroups.mount is false (that is, if cgroups have been pre-configured), then this cgroups hierarchy must already exist and be writable by the NodeManager user, otherwise the NodeManager may fail. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler. yarn.nodemanager.linux-container-executor.cgroups.hierarchy /hadoop-yarn Whether the LCE should attempt to mount cgroups if not found. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler. yarn.nodemanager.linux-container-executor.cgroups.mount false Where the LCE should attempt to mount cgroups if not found. Common locations include /sys/fs/cgroup and /cgroup; the default location can vary depending on the Linux distribution in use. This path must exist before the NodeManager is launched. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler, and yarn.nodemanager.linux-container-executor.cgroups.mount is true. yarn.nodemanager.linux-container-executor.cgroups.mount-path Delay in ms between attempts to remove linux cgroup yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms 20 This determines which of the two modes that LCE should use on a non-secure cluster. If this value is set to true, then all containers will be launched as the user specified in yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user. If this value is set to false, then containers will run as the user who submitted the application. yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users true The UNIX user that containers will run as when Linux-container-executor is used in nonsecure mode (a use case for this is using cgroups) if the yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users is set to true. yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user nobody The allowed pattern for UNIX user names enforced by Linux-container-executor when used in nonsecure mode (use case for this is using cgroups). The default value is taken from /usr/sbin/adduser yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern ^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$ This flag determines whether apps should run with strict resource limits or be allowed to consume spare resources if they need them. For example, turning the flag on will restrict apps to use only their share of CPU, even if the node has spare CPU cycles. The default value is false i.e. use available resources. Please note that turning this flag on may reduce job throughput on the cluster. yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage false Comma separated list of runtimes that are allowed when using LinuxContainerExecutor. The allowed values are default and docker. yarn.nodemanager.runtime.linux.allowed-runtimes default This configuration setting determines the capabilities assigned to docker containers when they are launched. While these may not be case-sensitive from a docker perspective, it is best to keep these uppercase. yarn.nodemanager.runtime.linux.docker.capabilities CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE This configuration setting determines if privileged docker containers are allowed on this cluster. Use with extreme care. yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed false This configuration setting determines who is allowed to run privileged docker containers on this cluster. Use with extreme care. yarn.nodemanager.runtime.linux.docker.privileged-containers.acl This flag determines whether memory limit will be set for the Windows Job Object of the containers launched by the default container executor. yarn.nodemanager.windows-container.memory-limit.enabled false This flag determines whether CPU limit will be set for the Windows Job Object of the containers launched by the default container executor. yarn.nodemanager.windows-container.cpu-limit.enabled false Interval of time the linux container executor should try cleaning up cgroups entry when cleaning up a container. yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms 1000 The UNIX group that the linux-container-executor should run as. yarn.nodemanager.linux-container-executor.group T-file compression types used to compress aggregated logs. yarn.nodemanager.log-aggregation.compression-type none The kerberos principal for the node manager. yarn.nodemanager.principal A comma separated list of services where service name should only contain a-zA-Z0-9_ and can not start with numbers yarn.nodemanager.aux-services No. of ms to wait between sending a SIGTERM and SIGKILL to a container yarn.nodemanager.sleep-delay-before-sigkill.ms 250 Max time to wait for a process to come up when trying to cleanup a container yarn.nodemanager.process-kill-wait.ms 2000 The minimum allowed version of a resourcemanager that a nodemanager will connect to. The valid values are NONE (no version checking), EqualToNM (the resourcemanager's version is equal to or greater than the NM version), or a Version String. yarn.nodemanager.resourcemanager.minimum.version NONE Max number of threads in NMClientAsync to process container management events yarn.client.nodemanager-client-async.thread-pool-max-size 500 Max time to wait to establish a connection to NM yarn.client.nodemanager-connect.max-wait-ms 180000 Time interval between each attempt to connect to NM yarn.client.nodemanager-connect.retry-interval-ms 10000 Max time to wait for NM to connect to RM. When not set, proxy will fall back to use value of yarn.resourcemanager.connect.max-wait.ms. yarn.nodemanager.resourcemanager.connect.max-wait.ms Time interval between each NM attempt to connect to RM. When not set, proxy will fall back to use value of yarn.resourcemanager.connect.retry-interval.ms. yarn.nodemanager.resourcemanager.connect.retry-interval.ms Maximum number of proxy connections to cache for node managers. If set to a value greater than zero then the cache is enabled and the NMClient and MRAppMaster will cache the specified number of node manager proxies. There will be at max one proxy per node manager. Ex. configuring it to a value of 5 will make sure that client will at max have 5 proxies cached with 5 different node managers. These connections for these proxies will be timed out if idle for more than the system wide idle timeout period. Note that this could cause issues on large clusters as many connections could linger simultaneously and lead to a large number of connection threads. The token used for authentication will be used only at connection creation time. If a new token is received then the earlier connection should be closed in order to use the new token. This and (yarn.client.nodemanager-client-async.thread-pool-max-size) are related and should be in sync (no need for them to be equal). If the value of this property is zero then the connection cache is disabled and connections will use a zero idle timeout to prevent too many connection threads on large clusters. yarn.client.max-cached-nodemanagers-proxies 0 Enable the node manager to recover after starting yarn.nodemanager.recovery.enabled false The local filesystem directory in which the node manager will store state when recovery is enabled. yarn.nodemanager.recovery.dir ${hadoop.tmp.dir}/yarn-nm-recovery The time in seconds between full compactions of the NM state database. Setting the interval to zero disables the full compaction cycles. yarn.nodemanager.recovery.compaction-interval-secs 3600 Whether the nodemanager is running under supervision. A nodemanager that supports recovery and is running under supervision will not try to cleanup containers as it exits with the assumption it will be immediately be restarted and recover containers. yarn.nodemanager.recovery.supervised false Adjustment to the container OS scheduling priority. In Linux, passed directly to the nice command. yarn.nodemanager.container-executor.os.sched.priority.adjustment 0 Flag to enable container metrics yarn.nodemanager.container-metrics.enable true Container metrics flush period in ms. Set to -1 for flush on completion. yarn.nodemanager.container-metrics.period-ms -1 The delay time ms to unregister container metrics after completion. yarn.nodemanager.container-metrics.unregister-delay-ms 10000 Class used to calculate current container resource utilization. yarn.nodemanager.container-monitor.process-tree.class Flag to enable NodeManager disk health checker yarn.nodemanager.disk-health-checker.enable true Number of threads to use in NM log cleanup. Used when log aggregation is disabled. yarn.nodemanager.log.deletion-threads-count 4 The Windows group that the windows-container-executor should run as. yarn.nodemanager.windows-secure-container-executor.group yarn.nodemanager.docker-container-executor.exec-name /usr/bin/docker Name or path to the Docker client. The Docker image name to use for DockerContainerExecutor yarn.nodemanager.docker-container-executor.image-name mapreduce.job.hdfs-servers ${fs.defaultFS} yarn.nodemanager.aux-services.mapreduce_shuffle.class org.apache.hadoop.mapred.ShuffleHandler The kerberos principal for the proxy, if the proxy is not running as part of the RM. yarn.web-proxy.principal Keytab for WebAppProxy, if the proxy is not running as part of the RM. yarn.web-proxy.keytab The address for the web proxy as HOST:PORT, if this is not given then the proxy will run as part of the RM yarn.web-proxy.address CLASSPATH for YARN applications. A comma-separated list of CLASSPATH entries. When this value is empty, the following default CLASSPATH for YARN applications would be used. For Linux: $HADOOP_CONF_DIR, $HADOOP_COMMON_HOME/share/hadoop/common/*, $HADOOP_COMMON_HOME/share/hadoop/common/lib/*, $HADOOP_HDFS_HOME/share/hadoop/hdfs/*, $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*, $HADOOP_YARN_HOME/share/hadoop/yarn/*, $HADOOP_YARN_HOME/share/hadoop/yarn/lib/* For Windows: %HADOOP_CONF_DIR%, %HADOOP_COMMON_HOME%/share/hadoop/common/*, %HADOOP_COMMON_HOME%/share/hadoop/common/lib/*, %HADOOP_HDFS_HOME%/share/hadoop/hdfs/*, %HADOOP_HDFS_HOME%/share/hadoop/hdfs/lib/*, %HADOOP_YARN_HOME%/share/hadoop/yarn/*, %HADOOP_YARN_HOME%/share/hadoop/yarn/lib/* yarn.application.classpath Indicate what is the current version of the running timeline service. For example, if "yarn.timeline-service.version" is 1.5, and "yarn.timeline-service.enabled" is true, it means the cluster will and should bring up the timeline service v.1.5. On the client side, if the client uses the same version of timeline service, it should succeed. If the client chooses to use a smaller version in spite of this, then depending on how robust the compatibility story is between versions, the results may vary. yarn.timeline-service.version 1.0f In the server side it indicates whether timeline service is enabled or not. And in the client side, users can enable it to indicate whether client wants to use timeline service. If it's enabled in the client side along with security, then yarn client tries to fetch the delegation tokens for the timeline server. yarn.timeline-service.enabled false The hostname of the timeline service web application. yarn.timeline-service.hostname 0.0.0.0 This is default address for the timeline server to start the RPC server. yarn.timeline-service.address ${yarn.timeline-service.hostname}:10200 The http address of the timeline service web application. yarn.timeline-service.webapp.address ${yarn.timeline-service.hostname}:8188 The https address of the timeline service web application. yarn.timeline-service.webapp.https.address ${yarn.timeline-service.hostname}:8190 The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in yarn.timeline-service.address and yarn.timeline-service.webapp.address, respectively. This is most useful for making the service listen to all interfaces by setting to 0.0.0.0. yarn.timeline-service.bind-host Defines the max number of applications could be fetched using REST API or application history protocol and shown in timeline server web ui. yarn.timeline-service.generic-application-history.max-applications 10000 Store class name for timeline store. yarn.timeline-service.store-class org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore Enable age off of timeline store data. yarn.timeline-service.ttl-enable true Time to live for timeline store data in milliseconds. yarn.timeline-service.ttl-ms 604800000 Store file name for leveldb timeline store. yarn.timeline-service.leveldb-timeline-store.path ${hadoop.tmp.dir}/yarn/timeline Length of time to wait between deletion cycles of leveldb timeline store in milliseconds. yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms 300000 Size of read cache for uncompressed blocks for leveldb timeline store in bytes. yarn.timeline-service.leveldb-timeline-store.read-cache-size 104857600 Size of cache for recently read entity start times for leveldb timeline store in number of entities. yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size 10000 Size of cache for recently written entity start times for leveldb timeline store in number of entities. yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size 10000 Handler thread count to serve the client RPC requests. yarn.timeline-service.handler-thread-count 10 yarn.timeline-service.http-authentication.type simple Defines authentication used for the timeline server HTTP endpoint. Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME# yarn.timeline-service.http-authentication.simple.anonymous.allowed true Indicates if anonymous requests are allowed by the timeline server when using 'simple' authentication. The Kerberos principal for the timeline server. yarn.timeline-service.principal The Kerberos keytab for the timeline server. yarn.timeline-service.keytab /etc/krb5.keytab Comma separated list of UIs that will be hosted yarn.timeline-service.ui-names Default maximum number of retries for timeline service client and value -1 means no limit. yarn.timeline-service.client.max-retries 30 Client policy for whether timeline operations are non-fatal. Should the failure to obtain a delegation token be considered an application failure (option = false), or should the client attempt to continue to publish information without it (option=true) yarn.timeline-service.client.best-effort false Default retry time interval for timeline servive client. yarn.timeline-service.client.retry-interval-ms 1000 Enable timeline server to recover state after starting. If true, then yarn.timeline-service.state-store-class must be specified. yarn.timeline-service.recovery.enabled false Store class name for timeline state store. yarn.timeline-service.state-store-class org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore Store file name for leveldb state store. yarn.timeline-service.leveldb-state-store.path ${hadoop.tmp.dir}/yarn/timeline yarn.timeline-service.entity-group-fs-store.active-dir /tmp/entity-file-history/active HDFS path to store active application’s timeline data yarn.timeline-service.entity-group-fs-store.done-dir /tmp/entity-file-history/done/ HDFS path to store done application’s timeline data yarn.timeline-service.entity-group-fs-store.group-id-plugin-classes Plugins that can translate a timeline entity read request into a list of timeline entity group ids, separated by commas. yarn.timeline-service.entity-group-fs-store.summary-store Summary storage for ATS v1.5 org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore yarn.timeline-service.entity-group-fs-store.scan-interval-seconds Scan interval for ATS v1.5 entity group file system storage reader.This value controls how frequent the reader will scan the HDFS active directory for application status. 60 yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds Scan interval for ATS v1.5 entity group file system storage cleaner.This value controls how frequent the reader will scan the HDFS done directory for stale application data. 3600 yarn.timeline-service.entity-group-fs-store.retain-seconds How long the ATS v1.5 entity group file system storage will keep an application's data in the done directory. 604800 yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size Read cache size for the leveldb cache storage in ATS v1.5 plugin storage. 10485760 yarn.timeline-service.entity-group-fs-store.app-cache-size Size of the reader cache for ATS v1.5 reader. This value controls how many entity groups the ATS v1.5 server should cache. If the number of active read entity groups is greater than the number of caches items, some reads may return empty data. This value must be greater than 0. 10 yarn.timeline-service.client.fd-flush-interval-secs Flush interval for ATS v1.5 writer. This value controls how frequent the writer will flush the HDFS FSStream for the entity/domain. 10 yarn.timeline-service.client.fd-clean-interval-secs Scan interval for ATS v1.5 writer. This value controls how frequent the writer will scan the HDFS FSStream for the entity/domain. If the FSStream is stale for a long time, this FSStream will be close. 60 yarn.timeline-service.client.fd-retain-secs How long the ATS v1.5 writer will keep a FSStream open. If this fsstream does not write anything for this configured time, it will be close. 300 yarn.timeline-service.client.internal-timers-ttl-secs How long the internal Timer Tasks can be alive in writer. If there is no write operation for this configured time, the internal timer tasks will be close. 420 Whether the shared cache is enabled yarn.sharedcache.enabled false The root directory for the shared cache yarn.sharedcache.root-dir /sharedcache The level of nested directories before getting to the checksum directories. It must be non-negative. yarn.sharedcache.nested-level 3 The implementation to be used for the SCM store yarn.sharedcache.store.class org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore The implementation to be used for the SCM app-checker yarn.sharedcache.app-checker.class org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker A resource in the in-memory store is considered stale if the time since the last reference exceeds the staleness period. This value is specified in minutes. yarn.sharedcache.store.in-memory.staleness-period-mins 10080 Initial delay before the in-memory store runs its first check to remove dead initial applications. Specified in minutes. yarn.sharedcache.store.in-memory.initial-delay-mins 10 The frequency at which the in-memory store checks to remove dead initial applications. Specified in minutes. yarn.sharedcache.store.in-memory.check-period-mins 720 The address of the admin interface in the SCM (shared cache manager) yarn.sharedcache.admin.address 0.0.0.0:8047 The number of threads used to handle SCM admin interface (1 by default) yarn.sharedcache.admin.thread-count 1 The address of the web application in the SCM (shared cache manager) yarn.sharedcache.webapp.address 0.0.0.0:8788 The frequency at which a cleaner task runs. Specified in minutes. yarn.sharedcache.cleaner.period-mins 1440 Initial delay before the first cleaner task is scheduled. Specified in minutes. yarn.sharedcache.cleaner.initial-delay-mins 10 The time to sleep between processing each shared cache resource. Specified in milliseconds. yarn.sharedcache.cleaner.resource-sleep-ms 0 The address of the node manager interface in the SCM (shared cache manager) yarn.sharedcache.uploader.server.address 0.0.0.0:8046 The number of threads used to handle shared cache manager requests from the node manager (50 by default) yarn.sharedcache.uploader.server.thread-count 50 The address of the client interface in the SCM (shared cache manager) yarn.sharedcache.client-server.address 0.0.0.0:8045 The number of threads used to handle shared cache manager requests from clients (50 by default) yarn.sharedcache.client-server.thread-count 50 The algorithm used to compute checksums of files (SHA-256 by default) yarn.sharedcache.checksum.algo.impl org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl The replication factor for the node manager uploader for the shared cache (10 by default) yarn.sharedcache.nm.uploader.replication.factor 10 The number of threads used to upload files from a node manager instance (20 by default) yarn.sharedcache.nm.uploader.thread-count 20 ACL protocol for use in the Timeline server. security.applicationhistory.protocol.acl Set to true for MiniYARNCluster unit tests yarn.is.minicluster false Set for MiniYARNCluster unit tests to control resource monitoring yarn.minicluster.control-resource-monitoring false Set to false in order to allow MiniYARNCluster to run tests without port conflicts. yarn.minicluster.fixed.ports false Set to false in order to allow the NodeManager in MiniYARNCluster to use RPC to talk to the RM. yarn.minicluster.use-rpc false As yarn.nodemanager.resource.memory-mb property but for the NodeManager in a MiniYARNCluster. yarn.minicluster.yarn.nodemanager.resource.memory-mb 4096 Enable node labels feature yarn.node-labels.enabled false Retry policy used for FileSystem node label store. The policy is specified by N pairs of sleep-time in milliseconds and number-of-retries "s1,n1,s2,n2,...". yarn.node-labels.fs-store.retry-policy-spec 2000, 500 URI for NodeLabelManager. The default value is /tmp/hadoop-yarn-${user}/node-labels/ in the local filesystem. yarn.node-labels.fs-store.root-dir Set configuration type for node labels. Administrators can specify "centralized", "delegated-centralized" or "distributed". yarn.node-labels.configuration-type centralized When "yarn.node-labels.configuration-type" is configured with "distributed" in RM, Administrators can configure in NM the provider for the node labels by configuring this parameter. Administrators can configure "config", "script" or the class name of the provider. Configured class needs to extend org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider. If "config" is configured, then "ConfigurationNodeLabelsProvider" and if "script" is configured, then "ScriptNodeLabelsProvider" will be used. yarn.nodemanager.node-labels.provider When "yarn.nodemanager.node-labels.provider" is configured with "config", "Script" or the configured class extends AbstractNodeLabelsProvider, then periodically node labels are retrieved from the node labels provider. This configuration is to define the interval period. If -1 is configured then node labels are retrieved from provider only during initialization. Defaults to 10 mins. yarn.nodemanager.node-labels.provider.fetch-interval-ms 600000 Interval at which NM syncs its node labels with RM. NM will send its loaded labels every x intervals configured, along with heartbeat to RM. yarn.nodemanager.node-labels.resync-interval-ms 120000 When "yarn.nodemanager.node-labels.provider" is configured with "config" then ConfigurationNodeLabelsProvider fetches the partition label from this parameter. yarn.nodemanager.node-labels.provider.configured-node-partition When "yarn.nodemanager.node-labels.provider" is configured with "Script" then this configuration provides the timeout period after which it will interrupt the script which queries the Node labels. Defaults to 20 mins. yarn.nodemanager.node-labels.provider.fetch-timeout-ms 1200000 When node labels "yarn.node-labels.configuration-type" is of type "delegated-centralized", administrators should configure the class for fetching node labels by ResourceManager. Configured class needs to extend org.apache.hadoop.yarn.server.resourcemanager.nodelabels. RMNodeLabelsMappingProvider. yarn.resourcemanager.node-labels.provider When "yarn.node-labels.configuration-type" is configured with "delegated-centralized", then periodically node labels are retrieved from the node labels provider. This configuration is to define the interval. If -1 is configured then node labels are retrieved from provider only once for each node after it registers. Defaults to 30 mins. yarn.resourcemanager.node-labels.provider.fetch-interval-ms 1800000 The Node Label script to run. Script output Line starting with "NODE_PARTITION:" will be considered as Node Label Partition. In case of multiple lines have this pattern, then last one will be considered yarn.nodemanager.node-labels.provider.script.path The arguments to pass to the Node label script. yarn.nodemanager.node-labels.provider.script.opts The interval that the yarn client library uses to poll the completion status of the asynchronous API of application client protocol. yarn.client.application-client-protocol.poll-interval-ms 200 The duration (in ms) the YARN client waits for an expected state change to occur. -1 means unlimited wait time. yarn.client.application-client-protocol.poll-timeout-ms -1 RSS usage of a process computed via /proc/pid/stat is not very accurate as it includes shared pages of a process. /proc/pid/smaps provides useful information like Private_Dirty, Private_Clean, Shared_Dirty, Shared_Clean which can be used for computing more accurate RSS. When this flag is enabled, RSS is computed as Min(Shared_Dirty, Pss) + Private_Clean + Private_Dirty. It excludes read-only shared mappings in RSS computation. yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled false URL for log aggregation server yarn.log.server.url RM Application Tracking URL yarn.tracking.url.generator Class to be used for YarnAuthorizationProvider yarn.authorization-provider Defines how often NMs wake up to upload log files. The default value is -1. By default, the logs will be uploaded when the application is finished. By setting this configure, logs can be uploaded periodically when the application is running. The minimum rolling-interval-seconds can be set is 3600. yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds -1 Enable/disable intermediate-data encryption at YARN level. For now, this only is used by the FileSystemRMStateStore to setup right file-system security attributes. yarn.intermediate-data-encryption.enable false Flag to enable cross-origin (CORS) support in the NM. This flag requires the CORS filter initializer to be added to the filter initializers list in core-site.xml. yarn.nodemanager.webapp.cross-origin.enabled false Defines maximum application priority in a cluster. If an application is submitted with a priority higher than this value, it will be reset to this maximum value. yarn.cluster.max-application-priority 0 The default log aggregation policy class. Applications can override it via LogAggregationContext. This configuration can provide some cluster-side default behavior so that if the application doesn't specify any policy via LogAggregationContext administrators of the cluster can adjust the policy globally. yarn.nodemanager.log-aggregation.policy.class org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy The default parameters for the log aggregation policy. Applications can override it via LogAggregationContext. This configuration can provide some cluster-side default behavior so that if the application doesn't specify any policy via LogAggregationContext administrators of the cluster can adjust the policy globally. yarn.nodemanager.log-aggregation.policy.parameters Enable/Disable AMRMProxyService in the node manager. This service is used to intercept calls from the application masters to the resource manager. yarn.nodemanager.amrmproxy.enable false The address of the AMRMProxyService listener. yarn.nodemanager.amrmproxy.address 0.0.0.0:8048 The number of threads used to handle requests by the AMRMProxyService. yarn.nodemanager.amrmproxy.client.thread-count 25 The comma separated list of class names that implement the RequestInterceptor interface. This is used by the AMRMProxyService to create the request processing pipeline for applications. yarn.nodemanager.amrmproxy.interceptor-class.pipeline org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor Choose different implementation of node label's storage yarn.node-labels.fs-store.impl.class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore The least amount of time(msec.) an inactive (decommissioned or shutdown) node can stay in the nodes list of the resourcemanager after being declared untracked. A node is marked untracked if and only if it is absent from both include and exclude nodemanager lists on the RM. All inactive nodes are checked twice per timeout interval or every 10 minutes, whichever is lesser, and marked appropriately. The same is done when refreshNodes command (graceful or otherwise) is invoked. yarn.resourcemanager.node-removal-untracked.timeout-ms 60000