yarn-default.xml 35 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947
  1. <?xml version="1.0"?>
  2. <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
  3. <!--
  4. Licensed to the Apache Software Foundation (ASF) under one or more
  5. contributor license agreements. See the NOTICE file distributed with
  6. this work for additional information regarding copyright ownership.
  7. The ASF licenses this file to You under the Apache License, Version 2.0
  8. (the "License"); you may not use this file except in compliance with
  9. the License. You may obtain a copy of the License at
  10. http://www.apache.org/licenses/LICENSE-2.0
  11. Unless required by applicable law or agreed to in writing, software
  12. distributed under the License is distributed on an "AS IS" BASIS,
  13. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  14. See the License for the specific language governing permissions and
  15. limitations under the License.
  16. -->
  17. <!-- Do not modify this file directly. Instead, copy entries that you -->
  18. <!-- wish to modify from this file into yarn-site.xml and change them -->
  19. <!-- there. If yarn-site.xml does not already exist, create it. -->
  20. <configuration>
  21. <!-- IPC Configs -->
  22. <property>
  23. <description>Factory to create client IPC classes.</description>
  24. <name>yarn.ipc.client.factory.class</name>
  25. </property>
  26. <property>
  27. <description>Type of serialization to use.</description>
  28. <name>yarn.ipc.serializer.type</name>
  29. <value>protocolbuffers</value>
  30. </property>
  31. <property>
  32. <description>Factory to create server IPC classes.</description>
  33. <name>yarn.ipc.server.factory.class</name>
  34. </property>
  35. <property>
  36. <description>Factory to create IPC exceptions.</description>
  37. <name>yarn.ipc.exception.factory.class</name>
  38. </property>
  39. <property>
  40. <description>Factory to create serializeable records.</description>
  41. <name>yarn.ipc.record.factory.class</name>
  42. </property>
  43. <property>
  44. <description>RPC class implementation</description>
  45. <name>yarn.ipc.rpc.class</name>
  46. <value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value>
  47. </property>
  48. <!-- Resource Manager Configs -->
  49. <property>
  50. <description>The hostname of the RM.</description>
  51. <name>yarn.resourcemanager.hostname</name>
  52. <value>0.0.0.0</value>
  53. </property>
  54. <property>
  55. <description>The address of the applications manager interface in the RM.</description>
  56. <name>yarn.resourcemanager.address</name>
  57. <value>${yarn.resourcemanager.hostname}:8032</value>
  58. </property>
  59. <property>
  60. <description>The number of threads used to handle applications manager requests.</description>
  61. <name>yarn.resourcemanager.client.thread-count</name>
  62. <value>50</value>
  63. </property>
  64. <property>
  65. <description>The expiry interval for application master reporting.</description>
  66. <name>yarn.am.liveness-monitor.expiry-interval-ms</name>
  67. <value>600000</value>
  68. </property>
  69. <property>
  70. <description>The Kerberos principal for the resource manager.</description>
  71. <name>yarn.resourcemanager.principal</name>
  72. </property>
  73. <property>
  74. <description>The address of the scheduler interface.</description>
  75. <name>yarn.resourcemanager.scheduler.address</name>
  76. <value>${yarn.resourcemanager.hostname}:8030</value>
  77. </property>
  78. <property>
  79. <description>Number of threads to handle scheduler interface.</description>
  80. <name>yarn.resourcemanager.scheduler.client.thread-count</name>
  81. <value>50</value>
  82. </property>
  83. <property>
  84. <description>
  85. This configures the HTTP endpoint for Yarn Daemons.The following
  86. values are supported:
  87. - HTTP_ONLY : Service is provided only on http
  88. - HTTPS_ONLY : Service is provided only on https
  89. </description>
  90. <name>yarn.http.policy</name>
  91. <value>HTTP_ONLY</value>
  92. </property>
  93. <property>
  94. <description>The http address of the RM web application.</description>
  95. <name>yarn.resourcemanager.webapp.address</name>
  96. <value>${yarn.resourcemanager.hostname}:8088</value>
  97. </property>
  98. <property>
  99. <description>The https adddress of the RM web application.</description>
  100. <name>yarn.resourcemanager.webapp.https.address</name>
  101. <value>${yarn.resourcemanager.hostname}:8090</value>
  102. </property>
  103. <property>
  104. <name>yarn.resourcemanager.resource-tracker.address</name>
  105. <value>${yarn.resourcemanager.hostname}:8031</value>
  106. </property>
  107. <property>
  108. <description>Are acls enabled.</description>
  109. <name>yarn.acl.enable</name>
  110. <value>true</value>
  111. </property>
  112. <property>
  113. <description>ACL of who can be admin of the YARN cluster.</description>
  114. <name>yarn.admin.acl</name>
  115. <value>*</value>
  116. </property>
  117. <property>
  118. <description>The address of the RM admin interface.</description>
  119. <name>yarn.resourcemanager.admin.address</name>
  120. <value>${yarn.resourcemanager.hostname}:8033</value>
  121. </property>
  122. <property>
  123. <description>Number of threads used to handle RM admin interface.</description>
  124. <name>yarn.resourcemanager.admin.client.thread-count</name>
  125. <value>1</value>
  126. </property>
  127. <property>
  128. <description>How often should the RM check that the AM is still alive.</description>
  129. <name>yarn.resourcemanager.amliveliness-monitor.interval-ms</name>
  130. <value>1000</value>
  131. </property>
  132. <property>
  133. <description>Maximum time to wait to establish connection to
  134. ResourceManager.</description>
  135. <name>yarn.resourcemanager.connect.max-wait.ms</name>
  136. <value>900000</value>
  137. </property>
  138. <property>
  139. <description>How often to try connecting to the
  140. ResourceManager.</description>
  141. <name>yarn.resourcemanager.connect.retry-interval.ms</name>
  142. <value>30000</value>
  143. </property>
  144. <property>
  145. <description>The maximum number of application attempts. It's a global
  146. setting for all application masters. Each application master can specify
  147. its individual maximum number of application attempts via the API, but the
  148. individual number cannot be more than the global upper bound. If it is,
  149. the resourcemanager will override it. The default number is set to 2, to
  150. allow at least one retry for AM.</description>
  151. <name>yarn.resourcemanager.am.max-attempts</name>
  152. <value>2</value>
  153. </property>
  154. <property>
  155. <description>How often to check that containers are still alive. </description>
  156. <name>yarn.resourcemanager.container.liveness-monitor.interval-ms</name>
  157. <value>600000</value>
  158. </property>
  159. <property>
  160. <description>The keytab for the resource manager.</description>
  161. <name>yarn.resourcemanager.keytab</name>
  162. <value>/etc/krb5.keytab</value>
  163. </property>
  164. <property>
  165. <description>How long to wait until a node manager is considered dead.</description>
  166. <name>yarn.nm.liveness-monitor.expiry-interval-ms</name>
  167. <value>600000</value>
  168. </property>
  169. <property>
  170. <description>How often to check that node managers are still alive.</description>
  171. <name>yarn.resourcemanager.nm.liveness-monitor.interval-ms</name>
  172. <value>1000</value>
  173. </property>
  174. <property>
  175. <description>Path to file with nodes to include.</description>
  176. <name>yarn.resourcemanager.nodes.include-path</name>
  177. <value></value>
  178. </property>
  179. <property>
  180. <description>Path to file with nodes to exclude.</description>
  181. <name>yarn.resourcemanager.nodes.exclude-path</name>
  182. <value></value>
  183. </property>
  184. <property>
  185. <description>Number of threads to handle resource tracker calls.</description>
  186. <name>yarn.resourcemanager.resource-tracker.client.thread-count</name>
  187. <value>50</value>
  188. </property>
  189. <property>
  190. <description>The class to use as the resource scheduler.</description>
  191. <name>yarn.resourcemanager.scheduler.class</name>
  192. <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</value>
  193. </property>
  194. <property>
  195. <description>The minimum allocation for every container request at the RM,
  196. in MBs. Memory requests lower than this won't take effect,
  197. and the specified value will get allocated at minimum.</description>
  198. <name>yarn.scheduler.minimum-allocation-mb</name>
  199. <value>1024</value>
  200. </property>
  201. <property>
  202. <description>The maximum allocation for every container request at the RM,
  203. in MBs. Memory requests higher than this won't take effect,
  204. and will get capped to this value.</description>
  205. <name>yarn.scheduler.maximum-allocation-mb</name>
  206. <value>8192</value>
  207. </property>
  208. <property>
  209. <description>The minimum allocation for every container request at the RM,
  210. in terms of virtual CPU cores. Requests lower than this won't take effect,
  211. and the specified value will get allocated the minimum.</description>
  212. <name>yarn.scheduler.minimum-allocation-vcores</name>
  213. <value>1</value>
  214. </property>
  215. <property>
  216. <description>The maximum allocation for every container request at the RM,
  217. in terms of virtual CPU cores. Requests higher than this won't take effect,
  218. and will get capped to this value.</description>
  219. <name>yarn.scheduler.maximum-allocation-vcores</name>
  220. <value>32</value>
  221. </property>
  222. <property>
  223. <description>Enable RM to recover state after starting. If true, then
  224. yarn.resourcemanager.store.class must be specified</description>
  225. <name>yarn.resourcemanager.recovery.enabled</name>
  226. <value>false</value>
  227. </property>
  228. <property>
  229. <description>The class to use as the persistent store.</description>
  230. <name>yarn.resourcemanager.store.class</name>
  231. <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore</value>
  232. </property>
  233. <property>
  234. <description>Host:Port of the ZooKeeper server where RM state will
  235. be stored. This must be supplied when using
  236. org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
  237. as the value for yarn.resourcemanager.store.class</description>
  238. <name>yarn.resourcemanager.zk.state-store.address</name>
  239. <!--value>127.0.0.1:2181</value-->
  240. </property>
  241. <property>
  242. <description>Number of times ZKRMStateStore tries to connect to
  243. ZooKeeper. This may be supplied when using
  244. org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
  245. as the value for yarn.resourcemanager.store.class</description>
  246. <name>yarn.resourcemanager.zk.state-store.num-retries</name>
  247. <value>3</value>
  248. </property>
  249. <property>
  250. <description>Full path of the ZooKeeper znode where RM state will be
  251. stored. This must be supplied when using
  252. org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
  253. as the value for yarn.resourcemanager.store.class</description>
  254. <name>yarn.resourcemanager.zk.state-store.parent-path</name>
  255. <value>/rmstore</value>
  256. </property>
  257. <property>
  258. <description>Timeout when connecting to ZooKeeper.
  259. This may be supplied when using
  260. org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
  261. as the value for yarn.resourcemanager.store.class</description>
  262. <name>yarn.resourcemanager.zk.state-store.timeout.ms</name>
  263. <value>60000</value>
  264. </property>
  265. <property>
  266. <description>ACL's to be used for ZooKeeper znodes.
  267. This may be supplied when using
  268. org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
  269. as the value for yarn.resourcemanager.store.class</description>
  270. <name>yarn.resourcemanager.zk.state-store.acl</name>
  271. <value>world:anyone:rwcda</value>
  272. </property>
  273. <property>
  274. <description>URI pointing to the location of the FileSystem path where
  275. RM state will be stored. This must be supplied when using
  276. org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore
  277. as the value for yarn.resourcemanager.store.class</description>
  278. <name>yarn.resourcemanager.fs.state-store.uri</name>
  279. <value>${hadoop.tmp.dir}/yarn/system/rmstore</value>
  280. <!--value>hdfs://localhost:9000/rmstore</value-->
  281. </property>
  282. <property>
  283. <description>Enable RM high-availability. When enabled,
  284. (1) The RM starts in the Standby mode by default, and transitions to
  285. the Active mode when prompted to.
  286. (2) The nodes in the RM ensemble are listed in
  287. yarn.resourcemanager.ha.rm-ids
  288. (3) The id of each RM comes from yarn.resourcemanager.ha.id
  289. (4) The actual physical addresses come from the configs of the pattern
  290. - {rpc-config}.{id}</description>
  291. <name>yarn.resourcemanager.ha.enabled</name>
  292. <value>false</value>
  293. </property>
  294. <property>
  295. <description>The list of RM nodes in the cluster when HA is
  296. enabled. See description of yarn.resourcemanager.ha
  297. .enabled for full details on how this is used.</description>
  298. <name>yarn.resourcemanager.ha.rm-ids</name>
  299. <!--value>rm1,rm2</value-->
  300. </property>
  301. <property>
  302. <description>The id (string) of the current RM. When HA is enabled, this
  303. is a required config. See description of yarn.resourcemanager.ha.enabled
  304. for full details on how this is used.</description>
  305. <name>yarn.resourcemanager.ha.id</name>
  306. <!--value>rm1</value-->
  307. </property>
  308. <property>
  309. <description>The maximum number of completed applications RM keeps. </description>
  310. <name>yarn.resourcemanager.max-completed-applications</name>
  311. <value>10000</value>
  312. </property>
  313. <property>
  314. <description>Interval at which the delayed token removal thread runs</description>
  315. <name>yarn.resourcemanager.delayed.delegation-token.removal-interval-ms</name>
  316. <value>30000</value>
  317. </property>
  318. <property>
  319. <description>Interval for the roll over for the master key used to generate
  320. application tokens
  321. </description>
  322. <name>yarn.resourcemanager.application-tokens.master-key-rolling-interval-secs</name>
  323. <value>86400</value>
  324. </property>
  325. <property>
  326. <description>Interval for the roll over for the master key used to generate
  327. container tokens. It is expected to be much greater than
  328. yarn.nm.liveness-monitor.expiry-interval-ms and
  329. yarn.rm.container-allocation.expiry-interval-ms. Otherwise the
  330. behavior is undefined.
  331. </description>
  332. <name>yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs</name>
  333. <value>86400</value>
  334. </property>
  335. <property>
  336. <description>The heart-beat interval in milliseconds for every NodeManager in the cluster.</description>
  337. <name>yarn.resourcemanager.nodemanagers.heartbeat-interval-ms</name>
  338. <value>1000</value>
  339. </property>
  340. <property>
  341. <description>The minimum allowed version of a connecting nodemanager. The valid values are
  342. NONE (no version checking), EqualToRM (the nodemanager's version is equal to
  343. or greater than the RM version), or a Version String.</description>
  344. <name>yarn.resourcemanager.nodemanager.minimum.version</name>
  345. <value>NONE</value>
  346. </property>
  347. <property>
  348. <description>Enable a set of periodic monitors (specified in
  349. yarn.resourcemanager.scheduler.monitor.policies) that affect the
  350. scheduler.</description>
  351. <name>yarn.resourcemanager.scheduler.monitor.enable</name>
  352. <value>false</value>
  353. </property>
  354. <property>
  355. <description>The list of SchedulingEditPolicy classes that interact with
  356. the scheduler. A particular module may be incompatible with the
  357. scheduler, other policies, or a configuration of either.</description>
  358. <name>yarn.resourcemanager.scheduler.monitor.policies</name>
  359. <value>org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy</value>
  360. </property>
  361. <property>
  362. <description>Enable RM to write history data. If true, then
  363. yarn.resourcemanager.history-writer.class must be specified</description>
  364. <name>yarn.resourcemanager.history-writer.enabled</name>
  365. <value>false</value>
  366. </property>
  367. <property>
  368. <description>Number of worker threads that write the history data.</description>
  369. <name>yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size</name>
  370. <value>10</value>
  371. </property>
  372. <property>
  373. <description>The implementation class of ApplicationHistoryStore, which is
  374. to be used by RMApplicationHistoryWriter.
  375. </description>
  376. <name>yarn.resourcemanager.history-writer.class</name>
  377. <value>org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore</value>
  378. </property>
  379. <!-- Node Manager Configs -->
  380. <property>
  381. <description>The hostname of the NM.</description>
  382. <name>yarn.nodemanager.hostname</name>
  383. <value>0.0.0.0</value>
  384. </property>
  385. <property>
  386. <description>The address of the container manager in the NM.</description>
  387. <name>yarn.nodemanager.address</name>
  388. <value>${yarn.nodemanager.hostname}:0</value>
  389. </property>
  390. <property>
  391. <description>Environment variables that should be forwarded from the NodeManager's environment to the container's.</description>
  392. <name>yarn.nodemanager.admin-env</name>
  393. <value>MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX</value>
  394. </property>
  395. <property>
  396. <description>Environment variables that containers may override rather than use NodeManager's default.</description>
  397. <name>yarn.nodemanager.env-whitelist</name>
  398. <value>JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,HADOOP_YARN_HOME</value>
  399. </property>
  400. <property>
  401. <description>who will execute(launch) the containers.</description>
  402. <name>yarn.nodemanager.container-executor.class</name>
  403. <value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
  404. <!--<value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>-->
  405. </property>
  406. <property>
  407. <description>Number of threads container manager uses.</description>
  408. <name>yarn.nodemanager.container-manager.thread-count</name>
  409. <value>20</value>
  410. </property>
  411. <property>
  412. <description>Number of threads used in cleanup.</description>
  413. <name>yarn.nodemanager.delete.thread-count</name>
  414. <value>4</value>
  415. </property>
  416. <property>
  417. <description>
  418. Number of seconds after an application finishes before the nodemanager's
  419. DeletionService will delete the application's localized file directory
  420. and log directory.
  421. To diagnose Yarn application problems, set this property's value large
  422. enough (for example, to 600 = 10 minutes) to permit examination of these
  423. directories. After changing the property's value, you must restart the
  424. nodemanager in order for it to have an effect.
  425. The roots of Yarn applications' work directories is configurable with
  426. the yarn.nodemanager.local-dirs property (see below), and the roots
  427. of the Yarn applications' log directories is configurable with the
  428. yarn.nodemanager.log-dirs property (see also below).
  429. </description>
  430. <name>yarn.nodemanager.delete.debug-delay-sec</name>
  431. <value>0</value>
  432. </property>
  433. <property>
  434. <description>Keytab for NM.</description>
  435. <name>yarn.nodemanager.keytab</name>
  436. <value>/etc/krb5.keytab</value>
  437. </property>
  438. <property>
  439. <description>List of directories to store localized files in. An
  440. application's localized file directory will be found in:
  441. ${yarn.nodemanager.local-dirs}/usercache/${user}/appcache/application_${appid}.
  442. Individual containers' work directories, called container_${contid}, will
  443. be subdirectories of this.
  444. </description>
  445. <name>yarn.nodemanager.local-dirs</name>
  446. <value>${hadoop.tmp.dir}/nm-local-dir</value>
  447. </property>
  448. <property>
  449. <description>It limits the maximum number of files which will be localized
  450. in a single local directory. If the limit is reached then sub-directories
  451. will be created and new files will be localized in them. If it is set to
  452. a value less than or equal to 36 [which are sub-directories (0-9 and then
  453. a-z)] then NodeManager will fail to start. For example; [for public
  454. cache] if this is configured with a value of 40 ( 4 files +
  455. 36 sub-directories) and the local-dir is "/tmp/local-dir1" then it will
  456. allow 4 files to be created directly inside "/tmp/local-dir1/filecache".
  457. For files that are localized further it will create a sub-directory "0"
  458. inside "/tmp/local-dir1/filecache" and will localize files inside it
  459. until it becomes full. If a file is removed from a sub-directory that
  460. is marked full, then that sub-directory will be used back again to
  461. localize files.
  462. </description>
  463. <name>yarn.nodemanager.local-cache.max-files-per-directory</name>
  464. <value>8192</value>
  465. </property>
  466. <property>
  467. <description>Address where the localizer IPC is.</description>
  468. <name>yarn.nodemanager.localizer.address</name>
  469. <value>${yarn.nodemanager.hostname}:8040</value>
  470. </property>
  471. <property>
  472. <description>Interval in between cache cleanups.</description>
  473. <name>yarn.nodemanager.localizer.cache.cleanup.interval-ms</name>
  474. <value>600000</value>
  475. </property>
  476. <property>
  477. <description>Target size of localizer cache in MB, per local directory.</description>
  478. <name>yarn.nodemanager.localizer.cache.target-size-mb</name>
  479. <value>10240</value>
  480. </property>
  481. <property>
  482. <description>Number of threads to handle localization requests.</description>
  483. <name>yarn.nodemanager.localizer.client.thread-count</name>
  484. <value>5</value>
  485. </property>
  486. <property>
  487. <description>Number of threads to use for localization fetching.</description>
  488. <name>yarn.nodemanager.localizer.fetch.thread-count</name>
  489. <value>4</value>
  490. </property>
  491. <property>
  492. <description>
  493. Where to store container logs. An application's localized log directory
  494. will be found in ${yarn.nodemanager.log-dirs}/application_${appid}.
  495. Individual containers' log directories will be below this, in directories
  496. named container_{$contid}. Each container directory will contain the files
  497. stderr, stdin, and syslog generated by that container.
  498. </description>
  499. <name>yarn.nodemanager.log-dirs</name>
  500. <value>${yarn.log.dir}/userlogs</value>
  501. </property>
  502. <property>
  503. <description>Whether to enable log aggregation</description>
  504. <name>yarn.log-aggregation-enable</name>
  505. <value>false</value>
  506. </property>
  507. <property>
  508. <description>How long to keep aggregation logs before deleting them. -1 disables.
  509. Be careful set this too small and you will spam the name node.</description>
  510. <name>yarn.log-aggregation.retain-seconds</name>
  511. <value>-1</value>
  512. </property>
  513. <property>
  514. <description>How long to wait between aggregated log retention checks.
  515. If set to 0 or a negative value then the value is computed as one-tenth
  516. of the aggregated log retention time. Be careful set this too small and
  517. you will spam the name node.</description>
  518. <name>yarn.log-aggregation.retain-check-interval-seconds</name>
  519. <value>-1</value>
  520. </property>
  521. <property>
  522. <description>Time in seconds to retain user logs. Only applicable if
  523. log aggregation is disabled
  524. </description>
  525. <name>yarn.nodemanager.log.retain-seconds</name>
  526. <value>10800</value>
  527. </property>
  528. <property>
  529. <description>Where to aggregate logs to.</description>
  530. <name>yarn.nodemanager.remote-app-log-dir</name>
  531. <value>/tmp/logs</value>
  532. </property>
  533. <property>
  534. <description>The remote log dir will be created at
  535. {yarn.nodemanager.remote-app-log-dir}/${user}/{thisParam}
  536. </description>
  537. <name>yarn.nodemanager.remote-app-log-dir-suffix</name>
  538. <value>logs</value>
  539. </property>
  540. <property>
  541. <description>Amount of physical memory, in MB, that can be allocated
  542. for containers.</description>
  543. <name>yarn.nodemanager.resource.memory-mb</name>
  544. <value>8192</value>
  545. </property>
  546. <property>
  547. <description>Whether physical memory limits will be enforced for
  548. containers.</description>
  549. <name>yarn.nodemanager.pmem-check-enabled</name>
  550. <value>true</value>
  551. </property>
  552. <property>
  553. <description>Whether virtual memory limits will be enforced for
  554. containers.</description>
  555. <name>yarn.nodemanager.vmem-check-enabled</name>
  556. <value>true</value>
  557. </property>
  558. <property>
  559. <description>Ratio between virtual memory to physical memory when
  560. setting memory limits for containers. Container allocations are
  561. expressed in terms of physical memory, and virtual memory usage
  562. is allowed to exceed this allocation by this ratio.
  563. </description>
  564. <name>yarn.nodemanager.vmem-pmem-ratio</name>
  565. <value>2.1</value>
  566. </property>
  567. <property>
  568. <description>Number of CPU cores that can be allocated
  569. for containers.</description>
  570. <name>yarn.nodemanager.resource.cpu-vcores</name>
  571. <value>8</value>
  572. </property>
  573. <property>
  574. <description>NM Webapp address.</description>
  575. <name>yarn.nodemanager.webapp.address</name>
  576. <value>${yarn.nodemanager.hostname}:8042</value>
  577. </property>
  578. <property>
  579. <description>How often to monitor containers.</description>
  580. <name>yarn.nodemanager.container-monitor.interval-ms</name>
  581. <value>3000</value>
  582. </property>
  583. <property>
  584. <description>Class that calculates containers current resource utilization.</description>
  585. <name>yarn.nodemanager.container-monitor.resource-calculator.class</name>
  586. </property>
  587. <property>
  588. <description>Frequency of running node health script.</description>
  589. <name>yarn.nodemanager.health-checker.interval-ms</name>
  590. <value>600000</value>
  591. </property>
  592. <property>
  593. <description>Script time out period.</description>
  594. <name>yarn.nodemanager.health-checker.script.timeout-ms</name>
  595. <value>1200000</value>
  596. </property>
  597. <property>
  598. <description>The health check script to run.</description>
  599. <name>yarn.nodemanager.health-checker.script.path</name>
  600. <value></value>
  601. </property>
  602. <property>
  603. <description>The arguments to pass to the health check script.</description>
  604. <name>yarn.nodemanager.health-checker.script.opts</name>
  605. <value></value>
  606. </property>
  607. <property>
  608. <description>Frequency of running disk health checker code.</description>
  609. <name>yarn.nodemanager.disk-health-checker.interval-ms</name>
  610. <value>120000</value>
  611. </property>
  612. <property>
  613. <description>The minimum fraction of number of disks to be healthy for the
  614. nodemanager to launch new containers. This correspond to both
  615. yarn-nodemanager.local-dirs and yarn.nodemanager.log-dirs. i.e. If there
  616. are less number of healthy local-dirs (or log-dirs) available, then
  617. new containers will not be launched on this node.</description>
  618. <name>yarn.nodemanager.disk-health-checker.min-healthy-disks</name>
  619. <value>0.25</value>
  620. </property>
  621. <property>
  622. <description>The path to the Linux container executor.</description>
  623. <name>yarn.nodemanager.linux-container-executor.path</name>
  624. </property>
  625. <property>
  626. <description>The class which should help the LCE handle resources.</description>
  627. <name>yarn.nodemanager.linux-container-executor.resources-handler.class</name>
  628. <value>org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler</value>
  629. <!-- <value>org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler</value> -->
  630. </property>
  631. <property>
  632. <description>The cgroups hierarchy under which to place YARN proccesses (cannot contain commas).
  633. If yarn.nodemanager.linux-container-executor.cgroups.mount is false (that is, if cgroups have
  634. been pre-configured), then this cgroups hierarchy must already exist and be writable by the
  635. NodeManager user, otherwise the NodeManager may fail.
  636. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler.</description>
  637. <name>yarn.nodemanager.linux-container-executor.cgroups.hierarchy</name>
  638. <value>/hadoop-yarn</value>
  639. </property>
  640. <property>
  641. <description>Whether the LCE should attempt to mount cgroups if not found.
  642. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler.</description>
  643. <name>yarn.nodemanager.linux-container-executor.cgroups.mount</name>
  644. <value>false</value>
  645. </property>
  646. <property>
  647. <description>Where the LCE should attempt to mount cgroups if not found. Common locations
  648. include /sys/fs/cgroup and /cgroup; the default location can vary depending on the Linux
  649. distribution in use. This path must exist before the NodeManager is launched.
  650. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler, and
  651. yarn.nodemanager.linux-container-executor.cgroups.mount is true.</description>
  652. <name>yarn.nodemanager.linux-container-executor.cgroups.mount-path</name>
  653. </property>
  654. <property>
  655. <description>The UNIX user that containers will run as when Linux-container-executor
  656. is used in nonsecure mode (a use case for this is using cgroups).</description>
  657. <name>yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user</name>
  658. <value>nobody</value>
  659. </property>
  660. <property>
  661. <description>The allowed pattern for UNIX user names enforced by
  662. Linux-container-executor when used in nonsecure mode (use case for this
  663. is using cgroups). The default value is taken from /usr/sbin/adduser</description>
  664. <name>yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern</name>
  665. <value>^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$</value>
  666. </property>
  667. <property>
  668. <description>T-file compression types used to compress aggregated logs.</description>
  669. <name>yarn.nodemanager.log-aggregation.compression-type</name>
  670. <value>none</value>
  671. </property>
  672. <property>
  673. <description>The kerberos principal for the node manager.</description>
  674. <name>yarn.nodemanager.principal</name>
  675. <value></value>
  676. </property>
  677. <property>
  678. <description>the valid service name should only contain a-zA-Z0-9_ and can not start with numbers</description>
  679. <name>yarn.nodemanager.aux-services</name>
  680. <value></value>
  681. <!--<value>mapreduce_shuffle</value>-->
  682. </property>
  683. <property>
  684. <description>No. of ms to wait between sending a SIGTERM and SIGKILL to a container</description>
  685. <name>yarn.nodemanager.sleep-delay-before-sigkill.ms</name>
  686. <value>250</value>
  687. </property>
  688. <property>
  689. <description>Max time to wait for a process to come up when trying to cleanup a container</description>
  690. <name>yarn.nodemanager.process-kill-wait.ms</name>
  691. <value>2000</value>
  692. </property>
  693. <property>
  694. <description>Max time, in seconds, to wait to establish a connection to RM when NM starts.
  695. The NM will shutdown if it cannot connect to RM within the specified max time period.
  696. If the value is set as -1, then NM will retry forever.</description>
  697. <name>yarn.nodemanager.resourcemanager.connect.wait.secs</name>
  698. <value>900</value>
  699. </property>
  700. <property>
  701. <description>Time interval, in seconds, between each NM attempt to connect to RM.</description>
  702. <name>yarn.nodemanager.resourcemanager.connect.retry_interval.secs</name>
  703. <value>30</value>
  704. </property>
  705. <property>
  706. <description>The minimum allowed version of a resourcemanager that a nodemanager will connect to.
  707. The valid values are NONE (no version checking), EqualToNM (the resourcemanager's version is
  708. equal to or greater than the NM version), or a Version String.</description>
  709. <name>yarn.nodemanager.resourcemanager.minimum.version</name>
  710. <value>NONE</value>
  711. </property>
  712. <property>
  713. <description>Max number of threads in NMClientAsync to process container
  714. management events</description>
  715. <name>yarn.client.nodemanager-client-async.thread-pool-max-size</name>
  716. <value>500</value>
  717. </property>
  718. <property>
  719. <description>
  720. Maximum number of proxy connections for node manager. It should always be
  721. more than 1. NMClient and MRAppMaster will use this to cache connection
  722. with node manager. There will be at max one connection per node manager.
  723. Ex. configuring it to a value of 5 will make sure that client will at
  724. max have 5 connections cached with 5 different node managers. These
  725. connections will be timed out if idle for more than system wide idle
  726. timeout period. The token if used for authentication then it will be used
  727. only at connection creation time. If new token is received then earlier
  728. connection should be closed in order to use newer token. This and
  729. (yarn.client.nodemanager-client-async.thread-pool-max-size) are related
  730. and should be sync (no need for them to be equal).
  731. </description>
  732. <name>yarn.client.max-nodemanagers-proxies</name>
  733. <value>500</value>
  734. </property>
  735. <!--Map Reduce configuration-->
  736. <property>
  737. <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
  738. <value>org.apache.hadoop.mapred.ShuffleHandler</value>
  739. </property>
  740. <property>
  741. <name>mapreduce.job.jar</name>
  742. <value/>
  743. </property>
  744. <property>
  745. <name>mapreduce.job.hdfs-servers</name>
  746. <value>${fs.defaultFS}</value>
  747. </property>
  748. <!-- WebAppProxy Configuration-->
  749. <property>
  750. <description>The kerberos principal for the proxy, if the proxy is not
  751. running as part of the RM.</description>
  752. <name>yarn.web-proxy.principal</name>
  753. <value/>
  754. </property>
  755. <property>
  756. <description>Keytab for WebAppProxy, if the proxy is not running as part of
  757. the RM.</description>
  758. <name>yarn.web-proxy.keytab</name>
  759. </property>
  760. <property>
  761. <description>The address for the web proxy as HOST:PORT, if this is not
  762. given then the proxy will run as part of the RM</description>
  763. <name>yarn.web-proxy.address</name>
  764. <value/>
  765. </property>
  766. <!-- Applications' Configuration-->
  767. <property>
  768. <description>CLASSPATH for YARN applications. A comma-separated list
  769. of CLASSPATH entries</description>
  770. <name>yarn.application.classpath</name>
  771. <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>
  772. </property>
  773. <!-- Application History Service's Configuration-->
  774. <property>
  775. <description>URI pointing to the location of the FileSystem path where
  776. the history will be persisted. This must be supplied when using
  777. org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore
  778. as the value for yarn.resourcemanager.history-writer.store.class</description>
  779. <name>yarn.ahs.fs-history-store.uri</name>
  780. <value>${hadoop.log.dir}/yarn/system/ahstore</value>
  781. </property>
  782. <property>
  783. <description>This is default address for the Application History server
  784. to start the RPC server.</description>
  785. <name>yarn.ahs.address</name>
  786. <value>0.0.0.0:10200</value>
  787. </property>
  788. <property>
  789. <description>CLient thread count to serve the client requests.</description>
  790. <name>yarn.ahs.client.thread-count</name>
  791. <value>10</value>
  792. </property>
  793. <property>
  794. <description>T-file compression types used to compress history data.</description>
  795. <name>yarn.ahs.fs-history-store.compression-type</name>
  796. <value>none</value>
  797. </property>
  798. <property>
  799. <description> Store class name for history store, defaulting to file
  800. system store </description>
  801. <name>yarn.ahs.store.class</name>
  802. <value>org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore</value>
  803. </property>
  804. <!-- Other configuration -->
  805. <property>
  806. <description>The interval of the yarn client's querying application state
  807. after application submission. The unit is millisecond.</description>
  808. <name>yarn.client.app-submission.poll-interval</name>
  809. <value>1000</value>
  810. </property>
  811. </configuration>