123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679 |
- <?xml version="1.0"?>
- <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
- <!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
- http://www.apache.org/licenses/LICENSE-2.0
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
- -->
- <!-- Do not modify this file directly. Instead, copy entries that you -->
- <!-- wish to modify from this file into yarn-site.xml and change them -->
- <!-- there. If yarn-site.xml does not already exist, create it. -->
- <configuration>
-
- <!-- IPC Configs -->
- <property>
- <description>Factory to create client IPC classes.</description>
- <name>yarn.ipc.client.factory.class</name>
- </property>
- <property>
- <description>Type of serialization to use.</description>
- <name>yarn.ipc.serializer.type</name>
- <value>protocolbuffers</value>
- </property>
- <property>
- <description>Factory to create server IPC classes.</description>
- <name>yarn.ipc.server.factory.class</name>
- </property>
- <property>
- <description>Factory to create IPC exceptions.</description>
- <name>yarn.ipc.exception.factory.class</name>
- </property>
- <property>
- <description>Factory to create serializeable records.</description>
- <name>yarn.ipc.record.factory.class</name>
- </property>
- <property>
- <description>RPC class implementation</description>
- <name>yarn.ipc.rpc.class</name>
- <value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value>
- </property>
-
- <!-- Resource Manager Configs -->
- <property>
- <description>The address of the applications manager interface in the RM.</description>
- <name>yarn.resourcemanager.address</name>
- <value>0.0.0.0:8032</value>
- </property>
- <property>
- <description>The number of threads used to handle applications manager requests.</description>
- <name>yarn.resourcemanager.client.thread-count</name>
- <value>50</value>
- </property>
- <property>
- <description>The expiry interval for application master reporting.</description>
- <name>yarn.am.liveness-monitor.expiry-interval-ms</name>
- <value>600000</value>
- </property>
- <property>
- <description>The Kerberos principal for the resource manager.</description>
- <name>yarn.resourcemanager.principal</name>
- </property>
- <property>
- <description>The address of the scheduler interface.</description>
- <name>yarn.resourcemanager.scheduler.address</name>
- <value>0.0.0.0:8030</value>
- </property>
- <property>
- <description>Number of threads to handle scheduler interface.</description>
- <name>yarn.resourcemanager.scheduler.client.thread-count</name>
- <value>50</value>
- </property>
- <property>
- <description>The address of the RM web application.</description>
- <name>yarn.resourcemanager.webapp.address</name>
- <value>0.0.0.0:8088</value>
- </property>
- <property>
- <name>yarn.resourcemanager.resource-tracker.address</name>
- <value>0.0.0.0:8031</value>
- </property>
- <property>
- <description>Are acls enabled.</description>
- <name>yarn.acl.enable</name>
- <value>true</value>
- </property>
- <property>
- <description>ACL of who can be admin of the YARN cluster.</description>
- <name>yarn.admin.acl</name>
- <value>*</value>
- </property>
- <property>
- <description>The address of the RM admin interface.</description>
- <name>yarn.resourcemanager.admin.address</name>
- <value>0.0.0.0:8033</value>
- </property>
- <property>
- <description>Number of threads used to handle RM admin interface.</description>
- <name>yarn.resourcemanager.admin.client.thread-count</name>
- <value>1</value>
- </property>
- <property>
- <description>How often should the RM check that the AM is still alive.</description>
- <name>yarn.resourcemanager.amliveliness-monitor.interval-ms</name>
- <value>1000</value>
- </property>
- <property>
- <description>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.</description>
- <name>yarn.resourcemanager.am.max-attempts</name>
- <value>1</value>
- </property>
- <property>
- <description>How often to check that containers are still alive. </description>
- <name>yarn.resourcemanager.container.liveness-monitor.interval-ms</name>
- <value>600000</value>
- </property>
- <property>
- <description>The keytab for the resource manager.</description>
- <name>yarn.resourcemanager.keytab</name>
- <value>/etc/krb5.keytab</value>
- </property>
- <property>
- <description>How long to wait until a node manager is considered dead.</description>
- <name>yarn.nm.liveness-monitor.expiry-interval-ms</name>
- <value>600000</value>
- </property>
- <property>
- <description>How often to check that node managers are still alive.</description>
- <name>yarn.resourcemanager.nm.liveness-monitor.interval-ms</name>
- <value>1000</value>
- </property>
- <property>
- <description>Path to file with nodes to include.</description>
- <name>yarn.resourcemanager.nodes.include-path</name>
- <value></value>
- </property>
- <property>
- <description>Path to file with nodes to exclude.</description>
- <name>yarn.resourcemanager.nodes.exclude-path</name>
- <value></value>
- </property>
- <property>
- <description>Number of threads to handle resource tracker calls.</description>
- <name>yarn.resourcemanager.resource-tracker.client.thread-count</name>
- <value>50</value>
- </property>
- <property>
- <description>The class to use as the resource scheduler.</description>
- <name>yarn.resourcemanager.scheduler.class</name>
- <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</value>
- </property>
- <property>
- <description>The minimum allocation for every container request at the RM,
- in MBs. Memory requests lower than this won't take effect,
- and the specified value will get allocated at minimum.</description>
- <name>yarn.scheduler.minimum-allocation-mb</name>
- <value>1024</value>
- </property>
- <property>
- <description>The maximum allocation for every container request at the RM,
- in MBs. Memory requests higher than this won't take effect,
- and will get capped to this value.</description>
- <name>yarn.scheduler.maximum-allocation-mb</name>
- <value>8192</value>
- </property>
- <property>
- <description>The minimum allocation for every container request at the RM,
- in terms of virtual CPU cores. Requests lower than this won't take effect,
- and the specified value will get allocated the minimum.</description>
- <name>yarn.scheduler.minimum-allocation-vcores</name>
- <value>1</value>
- </property>
- <property>
- <description>The maximum allocation for every container request at the RM,
- in terms of virtual CPU cores. Requests higher than this won't take effect,
- and will get capped to this value.</description>
- <name>yarn.scheduler.maximum-allocation-vcores</name>
- <value>32</value>
- </property>
- <property>
- <description>Enable RM to recover state after starting. If true, then
- yarn.resourcemanager.store.class must be specified</description>
- <name>yarn.resourcemanager.recovery.enabled</name>
- <value>false</value>
- </property>
- <property>
- <description>The class to use as the persistent store.</description>
- <name>yarn.resourcemanager.store.class</name>
- <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore</value>
- </property>
- <property>
- <description>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</description>
- <name>yarn.resourcemanager.fs.rm-state-store.uri</name>
- <value>${hadoop.tmp.dir}/yarn/system/rmstore</value>
- <!--value>hdfs://localhost:9000/rmstore</value-->
- </property>
- <property>
- <description>The maximum number of completed applications RM keeps. </description>
- <name>yarn.resourcemanager.max-completed-applications</name>
- <value>10000</value>
- </property>
- <property>
- <description>Interval at which the delayed token removal thread runs</description>
- <name>yarn.resourcemanager.delayed.delegation-token.removal-interval-ms</name>
- <value>30000</value>
- </property>
- <property>
- <description>Interval for the roll over for the master key used to generate
- application tokens
- </description>
- <name>yarn.resourcemanager.application-tokens.master-key-rolling-interval-secs</name>
- <value>86400</value>
- </property>
- <property>
- <description>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.rm.container-allocation.expiry-interval-ms. Otherwise the
- behavior is undefined.
- </description>
- <name>yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs</name>
- <value>86400</value>
- </property>
- <property>
- <description>The heart-beat interval in milliseconds for every NodeManager in the cluster.</description>
- <name>yarn.resourcemanager.nodemanagers.heartbeat-interval-ms</name>
- <value>1000</value>
- </property>
- <!-- Node Manager Configs -->
- <property>
- <description>The address of the container manager in the NM.</description>
- <name>yarn.nodemanager.address</name>
- <value>0.0.0.0:0</value>
- </property>
- <property>
- <description>Environment variables that should be forwarded from the NodeManager's environment to the container's.</description>
- <name>yarn.nodemanager.admin-env</name>
- <value>MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX</value>
- </property>
- <property>
- <description>Environment variables that containers may override rather than use NodeManager's default.</description>
- <name>yarn.nodemanager.env-whitelist</name>
- <value>JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,HADOOP_YARN_HOME</value>
- </property>
- <property>
- <description>who will execute(launch) the containers.</description>
- <name>yarn.nodemanager.container-executor.class</name>
- <value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
- <!--<value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>-->
- </property>
- <property>
- <description>Number of threads container manager uses.</description>
- <name>yarn.nodemanager.container-manager.thread-count</name>
- <value>20</value>
- </property>
- <property>
- <description>Number of threads used in cleanup.</description>
- <name>yarn.nodemanager.delete.thread-count</name>
- <value>4</value>
- </property>
- <property>
- <description>
- 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).
- </description>
- <name>yarn.nodemanager.delete.debug-delay-sec</name>
- <value>0</value>
- </property>
- <property>
- <description>Keytab for NM.</description>
- <name>yarn.nodemanager.keytab</name>
- <value>/etc/krb5.keytab</value>
- </property>
- <property>
- <description>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.
- </description>
- <name>yarn.nodemanager.local-dirs</name>
- <value>${hadoop.tmp.dir}/nm-local-dir</value>
- </property>
- <property>
- <description>Address where the localizer IPC is.</description>
- <name>yarn.nodemanager.localizer.address</name>
- <value>0.0.0.0:8040</value>
- </property>
- <property>
- <description>Interval in between cache cleanups.</description>
- <name>yarn.nodemanager.localizer.cache.cleanup.interval-ms</name>
- <value>600000</value>
- </property>
- <property>
- <description>Target size of localizer cache in MB, per local directory.</description>
- <name>yarn.nodemanager.localizer.cache.target-size-mb</name>
- <value>10240</value>
- </property>
- <property>
- <description>Number of threads to handle localization requests.</description>
- <name>yarn.nodemanager.localizer.client.thread-count</name>
- <value>5</value>
- </property>
- <property>
- <description>Number of threads to use for localization fetching.</description>
- <name>yarn.nodemanager.localizer.fetch.thread-count</name>
- <value>4</value>
- </property>
- <property>
- <description>
- 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.
- </description>
- <name>yarn.nodemanager.log-dirs</name>
- <value>${yarn.log.dir}/userlogs</value>
- </property>
- <property>
- <description>Whether to enable log aggregation</description>
- <name>yarn.log-aggregation-enable</name>
- <value>false</value>
- </property>
- <property>
- <description>How long to keep aggregation logs before deleting them. -1 disables.
- Be careful set this too small and you will spam the name node.</description>
- <name>yarn.log-aggregation.retain-seconds</name>
- <value>-1</value>
- </property>
-
- <property>
- <description>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.</description>
- <name>yarn.log-aggregation.retain-check-interval-seconds</name>
- <value>-1</value>
- </property>
- <property>
- <description>Time in seconds to retain user logs. Only applicable if
- log aggregation is disabled
- </description>
- <name>yarn.nodemanager.log.retain-seconds</name>
- <value>10800</value>
- </property>
- <property>
- <description>Where to aggregate logs to.</description>
- <name>yarn.nodemanager.remote-app-log-dir</name>
- <value>/tmp/logs</value>
- </property>
- <property>
- <description>The remote log dir will be created at
- {yarn.nodemanager.remote-app-log-dir}/${user}/{thisParam}
- </description>
- <name>yarn.nodemanager.remote-app-log-dir-suffix</name>
- <value>logs</value>
- </property>
- <property>
- <description>Amount of physical memory, in MB, that can be allocated
- for containers.</description>
- <name>yarn.nodemanager.resource.memory-mb</name>
- <value>8192</value>
- </property>
- <property>
- <description>Whether physical memory limits will be enforced for
- containers.</description>
- <name>yarn.nodemanager.pmem-check-enabled</name>
- <value>true</value>
- </property>
- <property>
- <description>Whether virtual memory limits will be enforced for
- containers.</description>
- <name>yarn.nodemanager.vmem-check-enabled</name>
- <value>true</value>
- </property>
- <property>
- <description>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.
- </description>
- <name>yarn.nodemanager.vmem-pmem-ratio</name>
- <value>2.1</value>
- </property>
- <property>
- <description>Number of CPU cores that can be allocated
- for containers.</description>
- <name>yarn.nodemanager.resource.cpu-cores</name>
- <value>8</value>
- </property>
- <property>
- <description>Ratio between virtual cores to physical cores when
- allocating CPU resources to containers.
- </description>
- <name>yarn.nodemanager.vcores-pcores-ratio</name>
- <value>2</value>
- </property>
- <property>
- <description>NM Webapp address.</description>
- <name>yarn.nodemanager.webapp.address</name>
- <value>0.0.0.0:8042</value>
- </property>
- <property>
- <description>How often to monitor containers.</description>
- <name>yarn.nodemanager.container-monitor.interval-ms</name>
- <value>3000</value>
- </property>
- <property>
- <description>Class that calculates containers current resource utilization.</description>
- <name>yarn.nodemanager.container-monitor.resource-calculator.class</name>
- </property>
- <property>
- <description>Frequency of running node health script.</description>
- <name>yarn.nodemanager.health-checker.interval-ms</name>
- <value>600000</value>
- </property>
- <property>
- <description>Script time out period.</description>
- <name>yarn.nodemanager.health-checker.script.timeout-ms</name>
- <value>1200000</value>
- </property>
- <property>
- <description>The health check script to run.</description>
- <name>yarn.nodemanager.health-checker.script.path</name>
- <value></value>
- </property>
- <property>
- <description>The arguments to pass to the health check script.</description>
- <name>yarn.nodemanager.health-checker.script.opts</name>
- <value></value>
- </property>
- <property>
- <description>Frequency of running disk health checker code.</description>
- <name>yarn.nodemanager.disk-health-checker.interval-ms</name>
- <value>120000</value>
- </property>
- <property>
- <description>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.</description>
- <name>yarn.nodemanager.disk-health-checker.min-healthy-disks</name>
- <value>0.25</value>
- </property>
- <property>
- <description>The path to the Linux container executor.</description>
- <name>yarn.nodemanager.linux-container-executor.path</name>
- </property>
- <property>
- <description>The class which should help the LCE handle resources.</description>
- <name>yarn.nodemanager.linux-container-executor.resources-handler.class</name>
- <value>org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler</value>
- <!-- <value>org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler</value> -->
- </property>
- <property>
- <description>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.</description>
- <name>yarn.nodemanager.linux-container-executor.cgroups.hierarchy</name>
- <value>/hadoop-yarn</value>
- </property>
- <property>
- <description>Whether the LCE should attempt to mount cgroups if not found.
- Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler.</description>
- <name>yarn.nodemanager.linux-container-executor.cgroups.mount</name>
- <value>false</value>
- </property>
- <property>
- <description>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.</description>
- <name>yarn.nodemanager.linux-container-executor.cgroups.mount-path</name>
- </property>
- <property>
- <description>T-file compression types used to compress aggregated logs.</description>
- <name>yarn.nodemanager.log-aggregation.compression-type</name>
- <value>none</value>
- </property>
- <property>
- <description>The kerberos principal for the node manager.</description>
- <name>yarn.nodemanager.principal</name>
- <value></value>
- </property>
- <property>
- <name>yarn.nodemanager.aux-services</name>
- <value></value>
- <!-- <value>mapreduce.shuffle</value> -->
- </property>
- <property>
- <description>No. of ms to wait between sending a SIGTERM and SIGKILL to a container</description>
- <name>yarn.nodemanager.sleep-delay-before-sigkill.ms</name>
- <value>250</value>
- </property>
- <property>
- <description>Max time to wait for a process to come up when trying to cleanup a container</description>
- <name>yarn.nodemanager.process-kill-wait.ms</name>
- <value>2000</value>
- </property>
- <property>
- <description>Max time, in seconds, to wait to establish a connection to RM when NM starts.
- The NM will shutdown if it cannot connect to RM within the specified max time period.
- If the value is set as -1, then NM will retry forever.</description>
- <name>yarn.nodemanager.resourcemanager.connect.wait.secs</name>
- <value>900</value>
- </property>
- <property>
- <description>Time interval, in seconds, between each NM attempt to connect to RM.</description>
- <name>yarn.nodemanager.resourcemanager.connect.retry_interval.secs</name>
- <value>30</value>
- </property>
- <!--Map Reduce configuration-->
- <property>
- <name>yarn.nodemanager.aux-services.mapreduce.shuffle.class</name>
- <value>org.apache.hadoop.mapred.ShuffleHandler</value>
- </property>
- <property>
- <name>mapreduce.job.jar</name>
- <value/>
- </property>
- <property>
- <name>mapreduce.job.hdfs-servers</name>
- <value>${fs.defaultFS}</value>
- </property>
- <!-- WebAppProxy Configuration-->
-
- <property>
- <description>The kerberos principal for the proxy, if the proxy is not
- running as part of the RM.</description>
- <name>yarn.web-proxy.principal</name>
- <value/>
- </property>
-
- <property>
- <description>Keytab for WebAppProxy, if the proxy is not running as part of
- the RM.</description>
- <name>yarn.web-proxy.keytab</name>
- </property>
-
- <property>
- <description>The address for the web proxy as HOST:PORT, if this is not
- given then the proxy will run as part of the RM</description>
- <name>yarn.web-proxy.address</name>
- <value/>
- </property>
- <!-- Applications' Configuration-->
-
- <property>
- <description>CLASSPATH for YARN applications. A comma-separated list
- of CLASSPATH entries</description>
- <name>yarn.application.classpath</name>
- <value>$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/*</value>
- </property>
- </configuration>
|