yarn-default.xml 60 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617
  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>Factory to create server IPC classes.</description>
  28. <name>yarn.ipc.server.factory.class</name>
  29. </property>
  30. <property>
  31. <description>Factory to create serializeable records.</description>
  32. <name>yarn.ipc.record.factory.class</name>
  33. </property>
  34. <property>
  35. <description>RPC class implementation</description>
  36. <name>yarn.ipc.rpc.class</name>
  37. <value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value>
  38. </property>
  39. <!-- Resource Manager Configs -->
  40. <property>
  41. <description>The hostname of the RM.</description>
  42. <name>yarn.resourcemanager.hostname</name>
  43. <value>0.0.0.0</value>
  44. </property>
  45. <property>
  46. <description>The address of the applications manager interface in the RM.</description>
  47. <name>yarn.resourcemanager.address</name>
  48. <value>${yarn.resourcemanager.hostname}:8032</value>
  49. </property>
  50. <property>
  51. <description>
  52. The actual address the server will bind to. If this optional address is
  53. set, the RPC and webapp servers will bind to this address and the port specified in
  54. yarn.resourcemanager.address and yarn.resourcemanager.webapp.address, respectively. This
  55. is most useful for making RM listen to all interfaces by setting to 0.0.0.0.
  56. </description>
  57. <name>yarn.resourcemanager.bind-host</name>
  58. <value></value>
  59. </property>
  60. <property>
  61. <description>The number of threads used to handle applications manager requests.</description>
  62. <name>yarn.resourcemanager.client.thread-count</name>
  63. <value>50</value>
  64. </property>
  65. <property>
  66. <description>The expiry interval for application master reporting.</description>
  67. <name>yarn.am.liveness-monitor.expiry-interval-ms</name>
  68. <value>600000</value>
  69. </property>
  70. <property>
  71. <description>The Kerberos principal for the resource manager.</description>
  72. <name>yarn.resourcemanager.principal</name>
  73. </property>
  74. <property>
  75. <description>The address of the scheduler interface.</description>
  76. <name>yarn.resourcemanager.scheduler.address</name>
  77. <value>${yarn.resourcemanager.hostname}:8030</value>
  78. </property>
  79. <property>
  80. <description>Number of threads to handle scheduler interface.</description>
  81. <name>yarn.resourcemanager.scheduler.client.thread-count</name>
  82. <value>50</value>
  83. </property>
  84. <property>
  85. <description>
  86. This configures the HTTP endpoint for Yarn Daemons.The following
  87. values are supported:
  88. - HTTP_ONLY : Service is provided only on http
  89. - HTTPS_ONLY : Service is provided only on https
  90. </description>
  91. <name>yarn.http.policy</name>
  92. <value>HTTP_ONLY</value>
  93. </property>
  94. <property>
  95. <description>The http address of the RM web application.</description>
  96. <name>yarn.resourcemanager.webapp.address</name>
  97. <value>${yarn.resourcemanager.hostname}:8088</value>
  98. </property>
  99. <property>
  100. <description>The https adddress of the RM web application.</description>
  101. <name>yarn.resourcemanager.webapp.https.address</name>
  102. <value>${yarn.resourcemanager.hostname}:8090</value>
  103. </property>
  104. <property>
  105. <name>yarn.resourcemanager.resource-tracker.address</name>
  106. <value>${yarn.resourcemanager.hostname}:8031</value>
  107. </property>
  108. <property>
  109. <description>Are acls enabled.</description>
  110. <name>yarn.acl.enable</name>
  111. <value>false</value>
  112. </property>
  113. <property>
  114. <description>ACL of who can be admin of the YARN cluster.</description>
  115. <name>yarn.admin.acl</name>
  116. <value>*</value>
  117. </property>
  118. <property>
  119. <description>The address of the RM admin interface.</description>
  120. <name>yarn.resourcemanager.admin.address</name>
  121. <value>${yarn.resourcemanager.hostname}:8033</value>
  122. </property>
  123. <property>
  124. <description>Number of threads used to handle RM admin interface.</description>
  125. <name>yarn.resourcemanager.admin.client.thread-count</name>
  126. <value>1</value>
  127. </property>
  128. <property>
  129. <description>Maximum time to wait to establish connection to
  130. ResourceManager.</description>
  131. <name>yarn.resourcemanager.connect.max-wait.ms</name>
  132. <value>900000</value>
  133. </property>
  134. <property>
  135. <description>How often to try connecting to the
  136. ResourceManager.</description>
  137. <name>yarn.resourcemanager.connect.retry-interval.ms</name>
  138. <value>30000</value>
  139. </property>
  140. <property>
  141. <description>The maximum number of application attempts. It's a global
  142. setting for all application masters. Each application master can specify
  143. its individual maximum number of application attempts via the API, but the
  144. individual number cannot be more than the global upper bound. If it is,
  145. the resourcemanager will override it. The default number is set to 2, to
  146. allow at least one retry for AM.</description>
  147. <name>yarn.resourcemanager.am.max-attempts</name>
  148. <value>2</value>
  149. </property>
  150. <property>
  151. <description>How often to check that containers are still alive. </description>
  152. <name>yarn.resourcemanager.container.liveness-monitor.interval-ms</name>
  153. <value>600000</value>
  154. </property>
  155. <property>
  156. <description>The keytab for the resource manager.</description>
  157. <name>yarn.resourcemanager.keytab</name>
  158. <value>/etc/krb5.keytab</value>
  159. </property>
  160. <property>
  161. <description>Flag to enable override of the default kerberos authentication
  162. filter with the RM authentication filter to allow authentication using
  163. delegation tokens(fallback to kerberos if the tokens are missing). Only
  164. applicable when the http authentication type is kerberos.</description>
  165. <name>yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled</name>
  166. <value>true</value>
  167. </property>
  168. <property>
  169. <description>How long to wait until a node manager is considered dead.</description>
  170. <name>yarn.nm.liveness-monitor.expiry-interval-ms</name>
  171. <value>600000</value>
  172. </property>
  173. <property>
  174. <description>Path to file with nodes to include.</description>
  175. <name>yarn.resourcemanager.nodes.include-path</name>
  176. <value></value>
  177. </property>
  178. <property>
  179. <description>Path to file with nodes to exclude.</description>
  180. <name>yarn.resourcemanager.nodes.exclude-path</name>
  181. <value></value>
  182. </property>
  183. <property>
  184. <description>Number of threads to handle resource tracker calls.</description>
  185. <name>yarn.resourcemanager.resource-tracker.client.thread-count</name>
  186. <value>50</value>
  187. </property>
  188. <property>
  189. <description>The class to use as the resource scheduler.</description>
  190. <name>yarn.resourcemanager.scheduler.class</name>
  191. <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</value>
  192. </property>
  193. <property>
  194. <description>The minimum allocation for every container request at the RM,
  195. in MBs. Memory requests lower than this won't take effect,
  196. and the specified value will get allocated at minimum.</description>
  197. <name>yarn.scheduler.minimum-allocation-mb</name>
  198. <value>1024</value>
  199. </property>
  200. <property>
  201. <description>The maximum allocation for every container request at the RM,
  202. in MBs. Memory requests higher than this won't take effect,
  203. and will get capped to this value.</description>
  204. <name>yarn.scheduler.maximum-allocation-mb</name>
  205. <value>8192</value>
  206. </property>
  207. <property>
  208. <description>The minimum allocation for every container request at the RM,
  209. in terms of virtual CPU cores. Requests lower than this won't take effect,
  210. and the specified value will get allocated the minimum.</description>
  211. <name>yarn.scheduler.minimum-allocation-vcores</name>
  212. <value>1</value>
  213. </property>
  214. <property>
  215. <description>The maximum allocation for every container request at the RM,
  216. in terms of virtual CPU cores. Requests higher than this won't take effect,
  217. and will get capped to this value.</description>
  218. <name>yarn.scheduler.maximum-allocation-vcores</name>
  219. <value>32</value>
  220. </property>
  221. <property>
  222. <description>Enable RM to recover state after starting. If true, then
  223. yarn.resourcemanager.store.class must be specified. </description>
  224. <name>yarn.resourcemanager.recovery.enabled</name>
  225. <value>false</value>
  226. </property>
  227. <property>
  228. <description>Enable RM work preserving recovery. This configuration is private
  229. to YARN for experimenting the feature.
  230. </description>
  231. <name>yarn.resourcemanager.work-preserving-recovery.enabled</name>
  232. <value>false</value>
  233. </property>
  234. <property>
  235. <description>Set the amount of time RM waits before allocating new
  236. containers on work-preserving-recovery. Such wait period gives RM a chance
  237. to settle down resyncing with NMs in the cluster on recovery, before assigning
  238. new containers to applications.
  239. </description>
  240. <name>yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms</name>
  241. <value>10000</value>
  242. </property>
  243. <property>
  244. <description>The class to use as the persistent store.
  245. If org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
  246. is used, the store is implicitly fenced; meaning a single ResourceManager
  247. is able to use the store at any point in time. More details on this
  248. implicit fencing, along with setting up appropriate ACLs is discussed
  249. under yarn.resourcemanager.zk-state-store.root-node.acl.
  250. </description>
  251. <name>yarn.resourcemanager.store.class</name>
  252. <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore</value>
  253. </property>
  254. <property>
  255. <description>The maximum number of completed applications RM state
  256. store keeps, less than or equals to ${yarn.resourcemanager.max-completed-applications}.
  257. By default, it equals to ${yarn.resourcemanager.max-completed-applications}.
  258. This ensures that the applications kept in the state store are consistent with
  259. the applications remembered in RM memory.
  260. Any values larger than ${yarn.resourcemanager.max-completed-applications} will
  261. be reset to ${yarn.resourcemanager.max-completed-applications}.
  262. Note that this value impacts the RM recovery performance.Typically,
  263. a smaller value indicates better performance on RM recovery.
  264. </description>
  265. <name>yarn.resourcemanager.state-store.max-completed-applications</name>
  266. <value>${yarn.resourcemanager.max-completed-applications}</value>
  267. </property>
  268. <property>
  269. <description>Host:Port of the ZooKeeper server to be used by the RM. This
  270. must be supplied when using the ZooKeeper based implementation of the
  271. RM state store and/or embedded automatic failover in a HA setting.
  272. </description>
  273. <name>yarn.resourcemanager.zk-address</name>
  274. <!--value>127.0.0.1:2181</value-->
  275. </property>
  276. <property>
  277. <description>Number of times RM tries to connect to ZooKeeper.</description>
  278. <name>yarn.resourcemanager.zk-num-retries</name>
  279. <value>1000</value>
  280. </property>
  281. <property>
  282. <description>Retry interval in milliseconds when connecting to ZooKeeper.
  283. When HA is enabled, the value here is NOT used. It is generated
  284. automatically from yarn.resourcemanager.zk-timeout-ms and
  285. yarn.resourcemanager.zk-num-retries.
  286. </description>
  287. <name>yarn.resourcemanager.zk-retry-interval-ms</name>
  288. <value>1000</value>
  289. </property>
  290. <property>
  291. <description>Full path of the ZooKeeper znode where RM state will be
  292. stored. This must be supplied when using
  293. org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
  294. as the value for yarn.resourcemanager.store.class</description>
  295. <name>yarn.resourcemanager.zk-state-store.parent-path</name>
  296. <value>/rmstore</value>
  297. </property>
  298. <property>
  299. <description>ZooKeeper session timeout in milliseconds. Session expiration
  300. is managed by the ZooKeeper cluster itself, not by the client. This value is
  301. used by the cluster to determine when the client's session expires.
  302. Expirations happens when the cluster does not hear from the client within
  303. the specified session timeout period (i.e. no heartbeat).</description>
  304. <name>yarn.resourcemanager.zk-timeout-ms</name>
  305. <value>10000</value>
  306. </property>
  307. <property>
  308. <description>ACL's to be used for ZooKeeper znodes.</description>
  309. <name>yarn.resourcemanager.zk-acl</name>
  310. <value>world:anyone:rwcda</value>
  311. </property>
  312. <property>
  313. <description>
  314. ACLs to be used for the root znode when using ZKRMStateStore in a HA
  315. scenario for fencing.
  316. ZKRMStateStore supports implicit fencing to allow a single
  317. ResourceManager write-access to the store. For fencing, the
  318. ResourceManagers in the cluster share read-write-admin privileges on the
  319. root node, but the Active ResourceManager claims exclusive create-delete
  320. permissions.
  321. By default, when this property is not set, we use the ACLs from
  322. yarn.resourcemanager.zk-acl for shared admin access and
  323. rm-address:random-number for username-based exclusive create-delete
  324. access.
  325. This property allows users to set ACLs of their choice instead of using
  326. the default mechanism. For fencing to work, the ACLs should be
  327. carefully set differently on each ResourceManger such that all the
  328. ResourceManagers have shared admin access and the Active ResourceManger
  329. takes over (exclusively) the create-delete access.
  330. </description>
  331. <name>yarn.resourcemanager.zk-state-store.root-node.acl</name>
  332. </property>
  333. <property>
  334. <description>
  335. Specify the auths to be used for the ACL's specified in both the
  336. yarn.resourcemanager.zk-acl and
  337. yarn.resourcemanager.zk-state-store.root-node.acl properties. This
  338. takes a comma-separated list of authentication mechanisms, each of the
  339. form 'scheme:auth' (the same syntax used for the 'addAuth' command in
  340. the ZK CLI).
  341. </description>
  342. <name>yarn.resourcemanager.zk-auth</name>
  343. </property>
  344. <property>
  345. <description>URI pointing to the location of the FileSystem path where
  346. RM state will be stored. This must be supplied when using
  347. org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore
  348. as the value for yarn.resourcemanager.store.class</description>
  349. <name>yarn.resourcemanager.fs.state-store.uri</name>
  350. <value>${hadoop.tmp.dir}/yarn/system/rmstore</value>
  351. <!--value>hdfs://localhost:9000/rmstore</value-->
  352. </property>
  353. <property>
  354. <description>hdfs client retry policy specification. hdfs client retry
  355. is always enabled. Specified in pairs of sleep-time and number-of-retries
  356. and (t0, n0), (t1, n1), ..., the first n0 retries sleep t0 milliseconds on
  357. average, the following n1 retries sleep t1 milliseconds on average, and so on.
  358. </description>
  359. <name>yarn.resourcemanager.fs.state-store.retry-policy-spec</name>
  360. <value>2000, 500</value>
  361. </property>
  362. <property>
  363. <description>Enable RM high-availability. When enabled,
  364. (1) The RM starts in the Standby mode by default, and transitions to
  365. the Active mode when prompted to.
  366. (2) The nodes in the RM ensemble are listed in
  367. yarn.resourcemanager.ha.rm-ids
  368. (3) The id of each RM either comes from yarn.resourcemanager.ha.id
  369. if yarn.resourcemanager.ha.id is explicitly specified or can be
  370. figured out by matching yarn.resourcemanager.address.{id} with local address
  371. (4) The actual physical addresses come from the configs of the pattern
  372. - {rpc-config}.{id}</description>
  373. <name>yarn.resourcemanager.ha.enabled</name>
  374. <value>false</value>
  375. </property>
  376. <property>
  377. <description>Enable automatic failover.
  378. By default, it is enabled only when HA is enabled</description>
  379. <name>yarn.resourcemanager.ha.automatic-failover.enabled</name>
  380. <value>true</value>
  381. </property>
  382. <property>
  383. <description>Enable embedded automatic failover.
  384. By default, it is enabled only when HA is enabled.
  385. The embedded elector relies on the RM state store to handle fencing,
  386. and is primarily intended to be used in conjunction with ZKRMStateStore.
  387. </description>
  388. <name>yarn.resourcemanager.ha.automatic-failover.embedded</name>
  389. <value>true</value>
  390. </property>
  391. <property>
  392. <description>The base znode path to use for storing leader information,
  393. when using ZooKeeper based leader election.</description>
  394. <name>yarn.resourcemanager.ha.automatic-failover.zk-base-path</name>
  395. <value>/yarn-leader-election</value>
  396. </property>
  397. <property>
  398. <description>Name of the cluster. In a HA setting,
  399. this is used to ensure the RM participates in leader
  400. election for this cluster and ensures it does not affect
  401. other clusters</description>
  402. <name>yarn.resourcemanager.cluster-id</name>
  403. <!--value>yarn-cluster</value-->
  404. </property>
  405. <property>
  406. <description>The list of RM nodes in the cluster when HA is
  407. enabled. See description of yarn.resourcemanager.ha
  408. .enabled for full details on how this is used.</description>
  409. <name>yarn.resourcemanager.ha.rm-ids</name>
  410. <!--value>rm1,rm2</value-->
  411. </property>
  412. <property>
  413. <description>The id (string) of the current RM. When HA is enabled, this
  414. is an optional config. The id of current RM can be set by explicitly
  415. specifying yarn.resourcemanager.ha.id or figured out by matching
  416. yarn.resourcemanager.address.{id} with local address
  417. See description of yarn.resourcemanager.ha.enabled
  418. for full details on how this is used.</description>
  419. <name>yarn.resourcemanager.ha.id</name>
  420. <!--value>rm1</value-->
  421. </property>
  422. <property>
  423. <description>When HA is enabled, the class to be used by Clients, AMs and
  424. NMs to failover to the Active RM. It should extend
  425. org.apache.hadoop.yarn.client.RMFailoverProxyProvider</description>
  426. <name>yarn.client.failover-proxy-provider</name>
  427. <value>org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider</value>
  428. </property>
  429. <property>
  430. <description>When HA is enabled, the max number of times
  431. FailoverProxyProvider should attempt failover. When set,
  432. this overrides the yarn.resourcemanager.connect.max-wait.ms. When
  433. not set, this is inferred from
  434. yarn.resourcemanager.connect.max-wait.ms.</description>
  435. <name>yarn.client.failover-max-attempts</name>
  436. <!--value>15</value-->
  437. </property>
  438. <property>
  439. <description>When HA is enabled, the sleep base (in milliseconds) to be
  440. used for calculating the exponential delay between failovers. When set,
  441. this overrides the yarn.resourcemanager.connect.* settings. When
  442. not set, yarn.resourcemanager.connect.retry-interval.ms is used instead.
  443. </description>
  444. <name>yarn.client.failover-sleep-base-ms</name>
  445. <!--value>500</value-->
  446. </property>
  447. <property>
  448. <description>When HA is enabled, the maximum sleep time (in milliseconds)
  449. between failovers. When set, this overrides the
  450. yarn.resourcemanager.connect.* settings. When not set,
  451. yarn.resourcemanager.connect.retry-interval.ms is used instead.</description>
  452. <name>yarn.client.failover-sleep-max-ms</name>
  453. <!--value>15000</value-->
  454. </property>
  455. <property>
  456. <description>When HA is enabled, the number of retries per
  457. attempt to connect to a ResourceManager. In other words,
  458. it is the ipc.client.connect.max.retries to be used during
  459. failover attempts</description>
  460. <name>yarn.client.failover-retries</name>
  461. <value>0</value>
  462. </property>
  463. <property>
  464. <description>When HA is enabled, the number of retries per
  465. attempt to connect to a ResourceManager on socket timeouts. In other
  466. words, it is the ipc.client.connect.max.retries.on.timeouts to be used
  467. during failover attempts</description>
  468. <name>yarn.client.failover-retries-on-socket-timeouts</name>
  469. <value>0</value>
  470. </property>
  471. <property>
  472. <description>The maximum number of completed applications RM keeps. </description>
  473. <name>yarn.resourcemanager.max-completed-applications</name>
  474. <value>10000</value>
  475. </property>
  476. <property>
  477. <description>Interval at which the delayed token removal thread runs</description>
  478. <name>yarn.resourcemanager.delayed.delegation-token.removal-interval-ms</name>
  479. <value>30000</value>
  480. </property>
  481. <property>
  482. <description>If true, ResourceManager will have proxy-user privileges.
  483. Use case: In a secure cluster, YARN requires the user hdfs delegation-tokens to
  484. do localization and log-aggregation on behalf of the user. If this is set to true,
  485. ResourceManager is able to request new hdfs delegation tokens on behalf of
  486. the user. This is needed by long-running-service, because the hdfs tokens
  487. will eventually expire and YARN requires new valid tokens to do localization
  488. and log-aggregation. Note that to enable this use case, the corresponding
  489. HDFS NameNode has to configure ResourceManager as the proxy-user so that
  490. ResourceManager can itself ask for new tokens on behalf of the user when
  491. tokens are past their max-life-time.</description>
  492. <name>yarn.resourcemanager.proxy-user-privileges.enabled</name>
  493. <value>false</value>
  494. </property>
  495. <property>
  496. <description>Interval for the roll over for the master key used to generate
  497. application tokens
  498. </description>
  499. <name>yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs</name>
  500. <value>86400</value>
  501. </property>
  502. <property>
  503. <description>Interval for the roll over for the master key used to generate
  504. container tokens. It is expected to be much greater than
  505. yarn.nm.liveness-monitor.expiry-interval-ms and
  506. yarn.rm.container-allocation.expiry-interval-ms. Otherwise the
  507. behavior is undefined.
  508. </description>
  509. <name>yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs</name>
  510. <value>86400</value>
  511. </property>
  512. <property>
  513. <description>The heart-beat interval in milliseconds for every NodeManager in the cluster.</description>
  514. <name>yarn.resourcemanager.nodemanagers.heartbeat-interval-ms</name>
  515. <value>1000</value>
  516. </property>
  517. <property>
  518. <description>The minimum allowed version of a connecting nodemanager. The valid values are
  519. NONE (no version checking), EqualToRM (the nodemanager's version is equal to
  520. or greater than the RM version), or a Version String.</description>
  521. <name>yarn.resourcemanager.nodemanager.minimum.version</name>
  522. <value>NONE</value>
  523. </property>
  524. <property>
  525. <description>Enable a set of periodic monitors (specified in
  526. yarn.resourcemanager.scheduler.monitor.policies) that affect the
  527. scheduler.</description>
  528. <name>yarn.resourcemanager.scheduler.monitor.enable</name>
  529. <value>false</value>
  530. </property>
  531. <property>
  532. <description>The list of SchedulingEditPolicy classes that interact with
  533. the scheduler. A particular module may be incompatible with the
  534. scheduler, other policies, or a configuration of either.</description>
  535. <name>yarn.resourcemanager.scheduler.monitor.policies</name>
  536. <value>org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy</value>
  537. </property>
  538. <property>
  539. <description>The class to use as the configuration provider.
  540. If org.apache.hadoop.yarn.LocalConfigurationProvider is used,
  541. the local configuration will be loaded.
  542. If org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider is used,
  543. the configuration which will be loaded should be uploaded to remote File system first.
  544. </description>
  545. <name>yarn.resourcemanager.configuration.provider-class</name>
  546. <value>org.apache.hadoop.yarn.LocalConfigurationProvider</value>
  547. <!-- <value>org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider</value> -->
  548. </property>
  549. <property>
  550. <description>The setting that controls whether yarn system metrics is
  551. published on the timeline server or not by RM.</description>
  552. <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
  553. <value>false</value>
  554. </property>
  555. <property>
  556. <description>Number of worker threads that send the yarn system metrics
  557. data.</description>
  558. <name>yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size</name>
  559. <value>10</value>
  560. </property>
  561. <!-- Node Manager Configs -->
  562. <property>
  563. <description>The hostname of the NM.</description>
  564. <name>yarn.nodemanager.hostname</name>
  565. <value>0.0.0.0</value>
  566. </property>
  567. <property>
  568. <description>The address of the container manager in the NM.</description>
  569. <name>yarn.nodemanager.address</name>
  570. <value>${yarn.nodemanager.hostname}:0</value>
  571. </property>
  572. <property>
  573. <description>
  574. The actual address the server will bind to. If this optional address is
  575. set, the RPC and webapp servers will bind to this address and the port specified in
  576. yarn.nodemanager.address and yarn.nodemanager.webapp.address, respectively. This is
  577. most useful for making NM listen to all interfaces by setting to 0.0.0.0.
  578. </description>
  579. <name>yarn.nodemanager.bind-host</name>
  580. <value></value>
  581. </property>
  582. <property>
  583. <description>Environment variables that should be forwarded from the NodeManager's environment to the container's.</description>
  584. <name>yarn.nodemanager.admin-env</name>
  585. <value>MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX</value>
  586. </property>
  587. <property>
  588. <description>Environment variables that containers may override rather than use NodeManager's default.</description>
  589. <name>yarn.nodemanager.env-whitelist</name>
  590. <value>JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,HADOOP_YARN_HOME</value>
  591. </property>
  592. <property>
  593. <description>who will execute(launch) the containers.</description>
  594. <name>yarn.nodemanager.container-executor.class</name>
  595. <value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
  596. <!--<value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>-->
  597. </property>
  598. <property>
  599. <description>Number of threads container manager uses.</description>
  600. <name>yarn.nodemanager.container-manager.thread-count</name>
  601. <value>20</value>
  602. </property>
  603. <property>
  604. <description>Number of threads used in cleanup.</description>
  605. <name>yarn.nodemanager.delete.thread-count</name>
  606. <value>4</value>
  607. </property>
  608. <property>
  609. <description>
  610. Number of seconds after an application finishes before the nodemanager's
  611. DeletionService will delete the application's localized file directory
  612. and log directory.
  613. To diagnose Yarn application problems, set this property's value large
  614. enough (for example, to 600 = 10 minutes) to permit examination of these
  615. directories. After changing the property's value, you must restart the
  616. nodemanager in order for it to have an effect.
  617. The roots of Yarn applications' work directories is configurable with
  618. the yarn.nodemanager.local-dirs property (see below), and the roots
  619. of the Yarn applications' log directories is configurable with the
  620. yarn.nodemanager.log-dirs property (see also below).
  621. </description>
  622. <name>yarn.nodemanager.delete.debug-delay-sec</name>
  623. <value>0</value>
  624. </property>
  625. <property>
  626. <description>Keytab for NM.</description>
  627. <name>yarn.nodemanager.keytab</name>
  628. <value>/etc/krb5.keytab</value>
  629. </property>
  630. <property>
  631. <description>List of directories to store localized files in. An
  632. application's localized file directory will be found in:
  633. ${yarn.nodemanager.local-dirs}/usercache/${user}/appcache/application_${appid}.
  634. Individual containers' work directories, called container_${contid}, will
  635. be subdirectories of this.
  636. </description>
  637. <name>yarn.nodemanager.local-dirs</name>
  638. <value>${hadoop.tmp.dir}/nm-local-dir</value>
  639. </property>
  640. <property>
  641. <description>It limits the maximum number of files which will be localized
  642. in a single local directory. If the limit is reached then sub-directories
  643. will be created and new files will be localized in them. If it is set to
  644. a value less than or equal to 36 [which are sub-directories (0-9 and then
  645. a-z)] then NodeManager will fail to start. For example; [for public
  646. cache] if this is configured with a value of 40 ( 4 files +
  647. 36 sub-directories) and the local-dir is "/tmp/local-dir1" then it will
  648. allow 4 files to be created directly inside "/tmp/local-dir1/filecache".
  649. For files that are localized further it will create a sub-directory "0"
  650. inside "/tmp/local-dir1/filecache" and will localize files inside it
  651. until it becomes full. If a file is removed from a sub-directory that
  652. is marked full, then that sub-directory will be used back again to
  653. localize files.
  654. </description>
  655. <name>yarn.nodemanager.local-cache.max-files-per-directory</name>
  656. <value>8192</value>
  657. </property>
  658. <property>
  659. <description>Address where the localizer IPC is.</description>
  660. <name>yarn.nodemanager.localizer.address</name>
  661. <value>${yarn.nodemanager.hostname}:8040</value>
  662. </property>
  663. <property>
  664. <description>Interval in between cache cleanups.</description>
  665. <name>yarn.nodemanager.localizer.cache.cleanup.interval-ms</name>
  666. <value>600000</value>
  667. </property>
  668. <property>
  669. <description>Target size of localizer cache in MB, per nodemanager. It is
  670. a target retention size that only includes resources with PUBLIC and
  671. PRIVATE visibility and excludes resources with APPLICATION visibility
  672. </description>
  673. <name>yarn.nodemanager.localizer.cache.target-size-mb</name>
  674. <value>10240</value>
  675. </property>
  676. <property>
  677. <description>Number of threads to handle localization requests.</description>
  678. <name>yarn.nodemanager.localizer.client.thread-count</name>
  679. <value>5</value>
  680. </property>
  681. <property>
  682. <description>Number of threads to use for localization fetching.</description>
  683. <name>yarn.nodemanager.localizer.fetch.thread-count</name>
  684. <value>4</value>
  685. </property>
  686. <property>
  687. <description>
  688. Where to store container logs. An application's localized log directory
  689. will be found in ${yarn.nodemanager.log-dirs}/application_${appid}.
  690. Individual containers' log directories will be below this, in directories
  691. named container_{$contid}. Each container directory will contain the files
  692. stderr, stdin, and syslog generated by that container.
  693. </description>
  694. <name>yarn.nodemanager.log-dirs</name>
  695. <value>${yarn.log.dir}/userlogs</value>
  696. </property>
  697. <property>
  698. <description>Whether to enable log aggregation. Log aggregation collects
  699. each container's logs and moves these logs onto a file-system, for e.g.
  700. HDFS, after the application completes. Users can configure the
  701. "yarn.nodemanager.remote-app-log-dir" and
  702. "yarn.nodemanager.remote-app-log-dir-suffix" properties to determine
  703. where these logs are moved to. Users can access the logs via the
  704. Application Timeline Server.
  705. </description>
  706. <name>yarn.log-aggregation-enable</name>
  707. <value>false</value>
  708. </property>
  709. <property>
  710. <description>How long to keep aggregation logs before deleting them. -1 disables.
  711. Be careful set this too small and you will spam the name node.</description>
  712. <name>yarn.log-aggregation.retain-seconds</name>
  713. <value>-1</value>
  714. </property>
  715. <property>
  716. <description>How long to wait between aggregated log retention checks.
  717. If set to 0 or a negative value then the value is computed as one-tenth
  718. of the aggregated log retention time. Be careful set this too small and
  719. you will spam the name node.</description>
  720. <name>yarn.log-aggregation.retain-check-interval-seconds</name>
  721. <value>-1</value>
  722. </property>
  723. <property>
  724. <description>Time in seconds to retain user logs. Only applicable if
  725. log aggregation is disabled
  726. </description>
  727. <name>yarn.nodemanager.log.retain-seconds</name>
  728. <value>10800</value>
  729. </property>
  730. <property>
  731. <description>Where to aggregate logs to.</description>
  732. <name>yarn.nodemanager.remote-app-log-dir</name>
  733. <value>/tmp/logs</value>
  734. </property>
  735. <property>
  736. <description>The remote log dir will be created at
  737. {yarn.nodemanager.remote-app-log-dir}/${user}/{thisParam}
  738. </description>
  739. <name>yarn.nodemanager.remote-app-log-dir-suffix</name>
  740. <value>logs</value>
  741. </property>
  742. <property>
  743. <description>Amount of physical memory, in MB, that can be allocated
  744. for containers.</description>
  745. <name>yarn.nodemanager.resource.memory-mb</name>
  746. <value>8192</value>
  747. </property>
  748. <property>
  749. <description>Whether physical memory limits will be enforced for
  750. containers.</description>
  751. <name>yarn.nodemanager.pmem-check-enabled</name>
  752. <value>true</value>
  753. </property>
  754. <property>
  755. <description>Whether virtual memory limits will be enforced for
  756. containers.</description>
  757. <name>yarn.nodemanager.vmem-check-enabled</name>
  758. <value>true</value>
  759. </property>
  760. <property>
  761. <description>Ratio between virtual memory to physical memory when
  762. setting memory limits for containers. Container allocations are
  763. expressed in terms of physical memory, and virtual memory usage
  764. is allowed to exceed this allocation by this ratio.
  765. </description>
  766. <name>yarn.nodemanager.vmem-pmem-ratio</name>
  767. <value>2.1</value>
  768. </property>
  769. <property>
  770. <description>Number of vcores that can be allocated
  771. for containers. This is used by the RM scheduler when allocating
  772. resources for containers. This is not used to limit the number of
  773. physical cores used by YARN containers.</description>
  774. <name>yarn.nodemanager.resource.cpu-vcores</name>
  775. <value>8</value>
  776. </property>
  777. <property>
  778. <description>Percentage of CPU that can be allocated
  779. for containers. This setting allows users to limit the amount of
  780. CPU that YARN containers use. Currently functional only
  781. on Linux using cgroups. The default is to use 100% of CPU.
  782. </description>
  783. <name>yarn.nodemanager.resource.percentage-physical-cpu-limit</name>
  784. <value>100</value>
  785. </property>
  786. <property>
  787. <description>NM Webapp address.</description>
  788. <name>yarn.nodemanager.webapp.address</name>
  789. <value>${yarn.nodemanager.hostname}:8042</value>
  790. </property>
  791. <property>
  792. <description>How often to monitor containers.</description>
  793. <name>yarn.nodemanager.container-monitor.interval-ms</name>
  794. <value>3000</value>
  795. </property>
  796. <property>
  797. <description>Class that calculates containers current resource utilization.</description>
  798. <name>yarn.nodemanager.container-monitor.resource-calculator.class</name>
  799. </property>
  800. <property>
  801. <description>Frequency of running node health script.</description>
  802. <name>yarn.nodemanager.health-checker.interval-ms</name>
  803. <value>600000</value>
  804. </property>
  805. <property>
  806. <description>Script time out period.</description>
  807. <name>yarn.nodemanager.health-checker.script.timeout-ms</name>
  808. <value>1200000</value>
  809. </property>
  810. <property>
  811. <description>The health check script to run.</description>
  812. <name>yarn.nodemanager.health-checker.script.path</name>
  813. <value></value>
  814. </property>
  815. <property>
  816. <description>The arguments to pass to the health check script.</description>
  817. <name>yarn.nodemanager.health-checker.script.opts</name>
  818. <value></value>
  819. </property>
  820. <property>
  821. <description>Frequency of running disk health checker code.</description>
  822. <name>yarn.nodemanager.disk-health-checker.interval-ms</name>
  823. <value>120000</value>
  824. </property>
  825. <property>
  826. <description>The minimum fraction of number of disks to be healthy for the
  827. nodemanager to launch new containers. This correspond to both
  828. yarn-nodemanager.local-dirs and yarn.nodemanager.log-dirs. i.e. If there
  829. are less number of healthy local-dirs (or log-dirs) available, then
  830. new containers will not be launched on this node.</description>
  831. <name>yarn.nodemanager.disk-health-checker.min-healthy-disks</name>
  832. <value>0.25</value>
  833. </property>
  834. <property>
  835. <description>The maximum percentage of disk space utilization allowed after
  836. which a disk is marked as bad. Values can range from 0.0 to 100.0.
  837. If the value is greater than or equal to 100, the nodemanager will check
  838. for full disk. This applies to yarn-nodemanager.local-dirs and
  839. yarn.nodemanager.log-dirs.</description>
  840. <name>yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage</name>
  841. <value>90.0</value>
  842. </property>
  843. <property>
  844. <description>The minimum space that must be available on a disk for
  845. it to be used. This applies to yarn-nodemanager.local-dirs and
  846. yarn.nodemanager.log-dirs.</description>
  847. <name>yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb</name>
  848. <value>0</value>
  849. </property>
  850. <property>
  851. <description>The path to the Linux container executor.</description>
  852. <name>yarn.nodemanager.linux-container-executor.path</name>
  853. </property>
  854. <property>
  855. <description>The class which should help the LCE handle resources.</description>
  856. <name>yarn.nodemanager.linux-container-executor.resources-handler.class</name>
  857. <value>org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler</value>
  858. <!-- <value>org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler</value> -->
  859. </property>
  860. <property>
  861. <description>The cgroups hierarchy under which to place YARN proccesses (cannot contain commas).
  862. If yarn.nodemanager.linux-container-executor.cgroups.mount is false (that is, if cgroups have
  863. been pre-configured), then this cgroups hierarchy must already exist and be writable by the
  864. NodeManager user, otherwise the NodeManager may fail.
  865. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler.</description>
  866. <name>yarn.nodemanager.linux-container-executor.cgroups.hierarchy</name>
  867. <value>/hadoop-yarn</value>
  868. </property>
  869. <property>
  870. <description>Whether the LCE should attempt to mount cgroups if not found.
  871. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler.</description>
  872. <name>yarn.nodemanager.linux-container-executor.cgroups.mount</name>
  873. <value>false</value>
  874. </property>
  875. <property>
  876. <description>Where the LCE should attempt to mount cgroups if not found. Common locations
  877. include /sys/fs/cgroup and /cgroup; the default location can vary depending on the Linux
  878. distribution in use. This path must exist before the NodeManager is launched.
  879. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler, and
  880. yarn.nodemanager.linux-container-executor.cgroups.mount is true.</description>
  881. <name>yarn.nodemanager.linux-container-executor.cgroups.mount-path</name>
  882. </property>
  883. <property>
  884. <description>This determines which of the two modes that LCE should use on a non-secure
  885. cluster. If this value is set to true, then all containers will be launched as the user
  886. specified in yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user. If
  887. this value is set to false, then containers will run as the user who submitted the
  888. application.
  889. </description>
  890. <name>yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users</name>
  891. <value>true</value>
  892. </property>
  893. <property>
  894. <description>The UNIX user that containers will run as when Linux-container-executor
  895. is used in nonsecure mode (a use case for this is using cgroups) if the
  896. yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users is set
  897. to true.</description>
  898. <name>yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user</name>
  899. <value>nobody</value>
  900. </property>
  901. <property>
  902. <description>The allowed pattern for UNIX user names enforced by
  903. Linux-container-executor when used in nonsecure mode (use case for this
  904. is using cgroups). The default value is taken from /usr/sbin/adduser</description>
  905. <name>yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern</name>
  906. <value>^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$</value>
  907. </property>
  908. <property>
  909. <description>This flag determines whether apps should run with strict resource limits
  910. or be allowed to consume spare resources if they need them. For example, turning the
  911. flag on will restrict apps to use only their share of CPU, even if the node has spare
  912. CPU cycles. The default value is false i.e. use available resources. Please note that
  913. turning this flag on may reduce job throughput on the cluster.</description>
  914. <name>yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage</name>
  915. <value>false</value>
  916. </property>
  917. <property>
  918. <description>T-file compression types used to compress aggregated logs.</description>
  919. <name>yarn.nodemanager.log-aggregation.compression-type</name>
  920. <value>none</value>
  921. </property>
  922. <property>
  923. <description>The kerberos principal for the node manager.</description>
  924. <name>yarn.nodemanager.principal</name>
  925. <value></value>
  926. </property>
  927. <property>
  928. <description>the valid service name should only contain a-zA-Z0-9_ and can not start with numbers</description>
  929. <name>yarn.nodemanager.aux-services</name>
  930. <value></value>
  931. <!--<value>mapreduce_shuffle</value>-->
  932. </property>
  933. <property>
  934. <description>No. of ms to wait between sending a SIGTERM and SIGKILL to a container</description>
  935. <name>yarn.nodemanager.sleep-delay-before-sigkill.ms</name>
  936. <value>250</value>
  937. </property>
  938. <property>
  939. <description>Max time to wait for a process to come up when trying to cleanup a container</description>
  940. <name>yarn.nodemanager.process-kill-wait.ms</name>
  941. <value>2000</value>
  942. </property>
  943. <property>
  944. <description>The minimum allowed version of a resourcemanager that a nodemanager will connect to.
  945. The valid values are NONE (no version checking), EqualToNM (the resourcemanager's version is
  946. equal to or greater than the NM version), or a Version String.</description>
  947. <name>yarn.nodemanager.resourcemanager.minimum.version</name>
  948. <value>NONE</value>
  949. </property>
  950. <property>
  951. <description>Max number of threads in NMClientAsync to process container
  952. management events</description>
  953. <name>yarn.client.nodemanager-client-async.thread-pool-max-size</name>
  954. <value>500</value>
  955. </property>
  956. <property>
  957. <description>Max time to wait to establish a connection to NM</description>
  958. <name>yarn.client.nodemanager-connect.max-wait-ms</name>
  959. <value>900000</value>
  960. </property>
  961. <property>
  962. <description>Time interval between each attempt to connect to NM</description>
  963. <name>yarn.client.nodemanager-connect.retry-interval-ms</name>
  964. <value>10000</value>
  965. </property>
  966. <property>
  967. <description>
  968. Maximum number of proxy connections to cache for node managers. If set
  969. to a value greater than zero then the cache is enabled and the NMClient
  970. and MRAppMaster will cache the specified number of node manager proxies.
  971. There will be at max one proxy per node manager. Ex. configuring it to a
  972. value of 5 will make sure that client will at max have 5 proxies cached
  973. with 5 different node managers. These connections for these proxies will
  974. be timed out if idle for more than the system wide idle timeout period.
  975. Note that this could cause issues on large clusters as many connections
  976. could linger simultaneously and lead to a large number of connection
  977. threads. The token used for authentication will be used only at
  978. connection creation time. If a new token is received then the earlier
  979. connection should be closed in order to use the new token. This and
  980. (yarn.client.nodemanager-client-async.thread-pool-max-size) are related
  981. and should be in sync (no need for them to be equal).
  982. If the value of this property is zero then the connection cache is
  983. disabled and connections will use a zero idle timeout to prevent too
  984. many connection threads on large clusters.
  985. </description>
  986. <name>yarn.client.max-cached-nodemanagers-proxies</name>
  987. <value>0</value>
  988. </property>
  989. <property>
  990. <description>Enable the node manager to recover after starting</description>
  991. <name>yarn.nodemanager.recovery.enabled</name>
  992. <value>false</value>
  993. </property>
  994. <property>
  995. <description>The local filesystem directory in which the node manager will
  996. store state when recovery is enabled.</description>
  997. <name>yarn.nodemanager.recovery.dir</name>
  998. <value>${hadoop.tmp.dir}/yarn-nm-recovery</value>
  999. </property>
  1000. <!--Docker configuration-->
  1001. <property>
  1002. <name>yarn.nodemanager.docker-container-executor.exec-name</name>
  1003. <value>/usr/bin/docker</value>
  1004. <description>
  1005. Name or path to the Docker client.
  1006. </description>
  1007. </property>
  1008. <!--Map Reduce configuration-->
  1009. <property>
  1010. <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
  1011. <value>org.apache.hadoop.mapred.ShuffleHandler</value>
  1012. </property>
  1013. <property>
  1014. <name>mapreduce.job.jar</name>
  1015. <value/>
  1016. </property>
  1017. <property>
  1018. <name>mapreduce.job.hdfs-servers</name>
  1019. <value>${fs.defaultFS}</value>
  1020. </property>
  1021. <!-- WebAppProxy Configuration-->
  1022. <property>
  1023. <description>The kerberos principal for the proxy, if the proxy is not
  1024. running as part of the RM.</description>
  1025. <name>yarn.web-proxy.principal</name>
  1026. <value/>
  1027. </property>
  1028. <property>
  1029. <description>Keytab for WebAppProxy, if the proxy is not running as part of
  1030. the RM.</description>
  1031. <name>yarn.web-proxy.keytab</name>
  1032. </property>
  1033. <property>
  1034. <description>The address for the web proxy as HOST:PORT, if this is not
  1035. given then the proxy will run as part of the RM</description>
  1036. <name>yarn.web-proxy.address</name>
  1037. <value/>
  1038. </property>
  1039. <!-- Applications' Configuration-->
  1040. <property>
  1041. <description>
  1042. CLASSPATH for YARN applications. A comma-separated list
  1043. of CLASSPATH entries. When this value is empty, the following default
  1044. CLASSPATH for YARN applications would be used.
  1045. For Linux:
  1046. $HADOOP_CONF_DIR,
  1047. $HADOOP_COMMON_HOME/share/hadoop/common/*,
  1048. $HADOOP_COMMON_HOME/share/hadoop/common/lib/*,
  1049. $HADOOP_HDFS_HOME/share/hadoop/hdfs/*,
  1050. $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,
  1051. $HADOOP_YARN_HOME/share/hadoop/yarn/*,
  1052. $HADOOP_YARN_HOME/share/hadoop/yarn/lib/*
  1053. For Windows:
  1054. %HADOOP_CONF_DIR%,
  1055. %HADOOP_COMMON_HOME%/share/hadoop/common/*,
  1056. %HADOOP_COMMON_HOME%/share/hadoop/common/lib/*,
  1057. %HADOOP_HDFS_HOME%/share/hadoop/hdfs/*,
  1058. %HADOOP_HDFS_HOME%/share/hadoop/hdfs/lib/*,
  1059. %HADOOP_YARN_HOME%/share/hadoop/yarn/*,
  1060. %HADOOP_YARN_HOME%/share/hadoop/yarn/lib/*
  1061. </description>
  1062. <name>yarn.application.classpath</name>
  1063. <value></value>
  1064. </property>
  1065. <!-- Timeline Service's Configuration-->
  1066. <property>
  1067. <description>Indicate to clients whether timeline service is enabled or not.
  1068. If enabled, clients will put entities and events to the timeline server.
  1069. </description>
  1070. <name>yarn.timeline-service.enabled</name>
  1071. <value>false</value>
  1072. </property>
  1073. <property>
  1074. <description>The hostname of the timeline service web application.</description>
  1075. <name>yarn.timeline-service.hostname</name>
  1076. <value>0.0.0.0</value>
  1077. </property>
  1078. <property>
  1079. <description>This is default address for the timeline server to start the
  1080. RPC server.</description>
  1081. <name>yarn.timeline-service.address</name>
  1082. <value>${yarn.timeline-service.hostname}:10200</value>
  1083. </property>
  1084. <property>
  1085. <description>The http address of the timeline service web application.</description>
  1086. <name>yarn.timeline-service.webapp.address</name>
  1087. <value>${yarn.timeline-service.hostname}:8188</value>
  1088. </property>
  1089. <property>
  1090. <description>The https address of the timeline service web application.</description>
  1091. <name>yarn.timeline-service.webapp.https.address</name>
  1092. <value>${yarn.timeline-service.hostname}:8190</value>
  1093. </property>
  1094. <property>
  1095. <description>
  1096. The actual address the server will bind to. If this optional address is
  1097. set, the RPC and webapp servers will bind to this address and the port specified in
  1098. yarn.timeline-service.address and yarn.timeline-service.webapp.address, respectively.
  1099. This is most useful for making the service listen to all interfaces by setting to
  1100. 0.0.0.0.
  1101. </description>
  1102. <name>yarn.timeline-service.bind-host</name>
  1103. <value></value>
  1104. </property>
  1105. <property>
  1106. <description>Store class name for timeline store.</description>
  1107. <name>yarn.timeline-service.store-class</name>
  1108. <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
  1109. </property>
  1110. <property>
  1111. <description>Enable age off of timeline store data.</description>
  1112. <name>yarn.timeline-service.ttl-enable</name>
  1113. <value>true</value>
  1114. </property>
  1115. <property>
  1116. <description>Time to live for timeline store data in milliseconds.</description>
  1117. <name>yarn.timeline-service.ttl-ms</name>
  1118. <value>604800000</value>
  1119. </property>
  1120. <property>
  1121. <description>Store file name for leveldb timeline store.</description>
  1122. <name>yarn.timeline-service.leveldb-timeline-store.path</name>
  1123. <value>${hadoop.tmp.dir}/yarn/timeline</value>
  1124. </property>
  1125. <property>
  1126. <description>Length of time to wait between deletion cycles of leveldb timeline store in milliseconds.</description>
  1127. <name>yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms</name>
  1128. <value>300000</value>
  1129. </property>
  1130. <property>
  1131. <description>Size of read cache for uncompressed blocks for leveldb timeline store in bytes.</description>
  1132. <name>yarn.timeline-service.leveldb-timeline-store.read-cache-size</name>
  1133. <value>104857600</value>
  1134. </property>
  1135. <property>
  1136. <description>Size of cache for recently read entity start times for leveldb timeline store in number of entities.</description>
  1137. <name>yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size</name>
  1138. <value>10000</value>
  1139. </property>
  1140. <property>
  1141. <description>Size of cache for recently written entity start times for leveldb timeline store in number of entities.</description>
  1142. <name>yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size</name>
  1143. <value>10000</value>
  1144. </property>
  1145. <property>
  1146. <description>Handler thread count to serve the client RPC requests.</description>
  1147. <name>yarn.timeline-service.handler-thread-count</name>
  1148. <value>10</value>
  1149. </property>
  1150. <property>
  1151. <name>yarn.timeline-service.http-authentication.type</name>
  1152. <value>simple</value>
  1153. <description>
  1154. Defines authentication used for the timeline server HTTP endpoint.
  1155. Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME#
  1156. </description>
  1157. </property>
  1158. <property>
  1159. <name>yarn.timeline-service.http-authentication.simple.anonymous.allowed</name>
  1160. <value>true</value>
  1161. <description>
  1162. Indicates if anonymous requests are allowed by the timeline server when using
  1163. 'simple' authentication.
  1164. </description>
  1165. </property>
  1166. <property>
  1167. <description>The Kerberos principal for the timeline server.</description>
  1168. <name>yarn.timeline-service.principal</name>
  1169. <value></value>
  1170. </property>
  1171. <property>
  1172. <description>The Kerberos keytab for the timeline server.</description>
  1173. <name>yarn.timeline-service.keytab</name>
  1174. <value>/etc/krb5.keytab</value>
  1175. </property>
  1176. <property>
  1177. <description>
  1178. Default maximum number of retires for timeline servive client.
  1179. </description>
  1180. <name>yarn.timeline-service.client.max-retries</name>
  1181. <value>30</value>
  1182. </property>
  1183. <property>
  1184. <description>
  1185. Default retry time interval for timeline servive client.
  1186. </description>
  1187. <name>yarn.timeline-service.client.retry-interval-ms</name>
  1188. <value>1000</value>
  1189. </property>
  1190. <!-- Shared Cache Configuration -->
  1191. <property>
  1192. <description>Whether the shared cache is enabled</description>
  1193. <name>yarn.sharedcache.enabled</name>
  1194. <value>false</value>
  1195. </property>
  1196. <property>
  1197. <description>The root directory for the shared cache</description>
  1198. <name>yarn.sharedcache.root-dir</name>
  1199. <value>/sharedcache</value>
  1200. </property>
  1201. <property>
  1202. <description>The level of nested directories before getting to the checksum
  1203. directories. It must be non-negative.</description>
  1204. <name>yarn.sharedcache.nested-level</name>
  1205. <value>3</value>
  1206. </property>
  1207. <property>
  1208. <description>The implementation to be used for the SCM store</description>
  1209. <name>yarn.sharedcache.store.class</name>
  1210. <value>org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore</value>
  1211. </property>
  1212. <property>
  1213. <description>The implementation to be used for the SCM app-checker</description>
  1214. <name>yarn.sharedcache.app-checker.class</name>
  1215. <value>org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker</value>
  1216. </property>
  1217. <property>
  1218. <description>A resource in the in-memory store is considered stale
  1219. if the time since the last reference exceeds the staleness period.
  1220. This value is specified in minutes.</description>
  1221. <name>yarn.sharedcache.store.in-memory.staleness-period-mins</name>
  1222. <value>10080</value>
  1223. </property>
  1224. <property>
  1225. <description>Initial delay before the in-memory store runs its first check
  1226. to remove dead initial applications. Specified in minutes.</description>
  1227. <name>yarn.sharedcache.store.in-memory.initial-delay-mins</name>
  1228. <value>10</value>
  1229. </property>
  1230. <property>
  1231. <description>The frequency at which the in-memory store checks to remove
  1232. dead initial applications. Specified in minutes.</description>
  1233. <name>yarn.sharedcache.store.in-memory.check-period-mins</name>
  1234. <value>720</value>
  1235. </property>
  1236. <property>
  1237. <description>The frequency at which a cleaner task runs.
  1238. Specified in minutes.</description>
  1239. <name>yarn.sharedcache.cleaner.period-mins</name>
  1240. <value>1440</value>
  1241. </property>
  1242. <property>
  1243. <description>Initial delay before the first cleaner task is scheduled.
  1244. Specified in minutes.</description>
  1245. <name>yarn.sharedcache.cleaner.initial-delay-mins</name>
  1246. <value>10</value>
  1247. </property>
  1248. <property>
  1249. <description>The time to sleep between processing each shared cache
  1250. resource. Specified in milliseconds.</description>
  1251. <name>yarn.sharedcache.cleaner.resource-sleep-ms</name>
  1252. <value>0</value>
  1253. </property>
  1254. <property>
  1255. <description>The address of the node manager interface in the SCM
  1256. (shared cache manager)</description>
  1257. <name>yarn.sharedcache.uploader.server.address</name>
  1258. <value>0.0.0.0:8046</value>
  1259. </property>
  1260. <property>
  1261. <description>The number of threads used to handle shared cache manager
  1262. requests from the node manager (50 by default)</description>
  1263. <name>yarn.sharedcache.uploader.server.thread-count</name>
  1264. <value>50</value>
  1265. </property>
  1266. <!-- Other configuration -->
  1267. <property>
  1268. <description>The interval that the yarn client library uses to poll the
  1269. completion status of the asynchronous API of application client protocol.
  1270. </description>
  1271. <name>yarn.client.application-client-protocol.poll-interval-ms</name>
  1272. <value>200</value>
  1273. </property>
  1274. <property>
  1275. <description>RSS usage of a process computed via
  1276. /proc/pid/stat is not very accurate as it includes shared pages of a
  1277. process. /proc/pid/smaps provides useful information like
  1278. Private_Dirty, Private_Clean, Shared_Dirty, Shared_Clean which can be used
  1279. for computing more accurate RSS. When this flag is enabled, RSS is computed
  1280. as Min(Shared_Dirty, Pss) + Private_Clean + Private_Dirty. It excludes
  1281. read-only shared mappings in RSS computation.
  1282. </description>
  1283. <name>yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled</name>
  1284. <value>false</value>
  1285. </property>
  1286. <!-- YARN registry -->
  1287. <property>
  1288. <description>
  1289. Is the registry enabled: does the RM start it up,
  1290. create the user and system paths, and purge
  1291. service records when containers, application attempts
  1292. and applications complete
  1293. </description>
  1294. <name>hadoop.registry.rm.enabled</name>
  1295. <value>false</value>
  1296. </property>
  1297. <property>
  1298. <description>
  1299. </description>
  1300. <name>hadoop.registry.zk.root</name>
  1301. <value>/registry</value>
  1302. </property>
  1303. <property>
  1304. <description>
  1305. Zookeeper session timeout in milliseconds
  1306. </description>
  1307. <name>hadoop.registry.zk.session.timeout.ms</name>
  1308. <value>60000</value>
  1309. </property>
  1310. <property>
  1311. <description>
  1312. Zookeeper session timeout in milliseconds
  1313. </description>
  1314. <name>hadoop.registry.zk.connection.timeout.ms</name>
  1315. <value>15000</value>
  1316. </property>
  1317. <property>
  1318. <description>
  1319. Zookeeper connection retry count before failing
  1320. </description>
  1321. <name>hadoop.registry.zk.retry.times</name>
  1322. <value>5</value>
  1323. </property>
  1324. <property>
  1325. <description>
  1326. </description>
  1327. <name>hadoop.registry.zk.retry.interval.ms</name>
  1328. <value>1000</value>
  1329. </property>
  1330. <property>
  1331. <description>
  1332. Zookeeper retry limit in milliseconds, during
  1333. exponential backoff: {@value}
  1334. This places a limit even
  1335. if the retry times and interval limit, combined
  1336. with the backoff policy, result in a long retry
  1337. period
  1338. </description>
  1339. <name>hadoop.registry.zk.retry.ceiling.ms</name>
  1340. <value>60000</value>
  1341. </property>
  1342. <property>
  1343. <description>
  1344. List of hostname:port pairs defining the
  1345. zookeeper quorum binding for the registry
  1346. </description>
  1347. <name>hadoop.registry.zk.quorum</name>
  1348. <value>localhost:2181</value>
  1349. </property>
  1350. <property>
  1351. <description>
  1352. Key to set if the registry is secure. Turning it on
  1353. changes the permissions policy from "open access"
  1354. to restrictions on kerberos with the option of
  1355. a user adding one or more auth key pairs down their
  1356. own tree.
  1357. </description>
  1358. <name>hadoop.registry.secure</name>
  1359. <value>false</value>
  1360. </property>
  1361. <property>
  1362. <description>
  1363. A comma separated list of Zookeeper ACL identifiers with
  1364. system access to the registry in a secure cluster.
  1365. These are given full access to all entries.
  1366. If there is an "@" at the end of a SASL entry it
  1367. instructs the registry client to append the default kerberos domain.
  1368. </description>
  1369. <name>hadoop.registry.system.acls</name>
  1370. <value>sasl:yarn@, sasl:mapred@, sasl:mapred@hdfs@</value>
  1371. </property>
  1372. <property>
  1373. <description>
  1374. The kerberos realm: used to set the realm of
  1375. system principals which do not declare their realm,
  1376. and any other accounts that need the value.
  1377. If empty, the default realm of the running process
  1378. is used.
  1379. If neither are known and the realm is needed, then the registry
  1380. service/client will fail.
  1381. </description>
  1382. <name>hadoop.registry.kerberos.realm</name>
  1383. <value></value>
  1384. </property>
  1385. <property>
  1386. <description>
  1387. Key to define the JAAS context. Used in secure
  1388. mode
  1389. </description>
  1390. <name>hadoop.registry.jaas.context</name>
  1391. <value>Client</value>
  1392. </property>
  1393. <property>
  1394. <description>Defines how often NMs wake up to upload log files.
  1395. The default value is -1. By default, the logs will be uploaded when
  1396. the application is finished. By setting this configure, logs can be uploaded
  1397. periodically when the application is running. The minimum rolling-interval-seconds
  1398. can be set is 3600.
  1399. </description>
  1400. <name>yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds</name>
  1401. <value>-1</value>
  1402. </property>
  1403. </configuration>