hdfs-default.xml 89 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583
  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 hdfs-site.xml and change them -->
  19. <!-- there. If hdfs-site.xml does not already exist, create it. -->
  20. <configuration>
  21. <property>
  22. <name>hadoop.hdfs.configuration.version</name>
  23. <value>1</value>
  24. <description>version of this configuration file</description>
  25. </property>
  26. <property>
  27. <name>dfs.namenode.rpc-address</name>
  28. <value></value>
  29. <description>
  30. RPC address that handles all clients requests. In the case of HA/Federation where multiple namenodes exist,
  31. the name service id is added to the name e.g. dfs.namenode.rpc-address.ns1
  32. dfs.namenode.rpc-address.EXAMPLENAMESERVICE
  33. The value of this property will take the form of nn-host1:rpc-port.
  34. </description>
  35. </property>
  36. <property>
  37. <name>dfs.namenode.rpc-bind-host</name>
  38. <value></value>
  39. <description>
  40. The actual address the RPC server will bind to. If this optional address is
  41. set, it overrides only the hostname portion of dfs.namenode.rpc-address.
  42. It can also be specified per name node or name service for HA/Federation.
  43. This is useful for making the name node listen on all interfaces by
  44. setting it to 0.0.0.0.
  45. </description>
  46. </property>
  47. <property>
  48. <name>dfs.namenode.servicerpc-address</name>
  49. <value></value>
  50. <description>
  51. RPC address for HDFS Services communication. BackupNode, Datanodes and all other services should be
  52. connecting to this address if it is configured. In the case of HA/Federation where multiple namenodes exist,
  53. the name service id is added to the name e.g. dfs.namenode.servicerpc-address.ns1
  54. dfs.namenode.rpc-address.EXAMPLENAMESERVICE
  55. The value of this property will take the form of nn-host1:rpc-port.
  56. If the value of this property is unset the value of dfs.namenode.rpc-address will be used as the default.
  57. </description>
  58. </property>
  59. <property>
  60. <name>dfs.namenode.servicerpc-bind-host</name>
  61. <value></value>
  62. <description>
  63. The actual address the service RPC server will bind to. If this optional address is
  64. set, it overrides only the hostname portion of dfs.namenode.servicerpc-address.
  65. It can also be specified per name node or name service for HA/Federation.
  66. This is useful for making the name node listen on all interfaces by
  67. setting it to 0.0.0.0.
  68. </description>
  69. </property>
  70. <property>
  71. <name>dfs.namenode.lifeline.rpc-address</name>
  72. <value></value>
  73. <description>
  74. NameNode RPC lifeline address. This is an optional separate RPC address
  75. that can be used to isolate health checks and liveness to protect against
  76. resource exhaustion in the main RPC handler pool. In the case of
  77. HA/Federation where multiple NameNodes exist, the name service ID is added
  78. to the name e.g. dfs.namenode.lifeline.rpc-address.ns1. The value of this
  79. property will take the form of nn-host1:rpc-port. If this property is not
  80. defined, then the NameNode will not start a lifeline RPC server. By
  81. default, the property is not defined.
  82. </description>
  83. </property>
  84. <property>
  85. <name>dfs.namenode.lifeline.rpc-bind-host</name>
  86. <value></value>
  87. <description>
  88. The actual address the lifeline RPC server will bind to. If this optional
  89. address is set, it overrides only the hostname portion of
  90. dfs.namenode.lifeline.rpc-address. It can also be specified per name node
  91. or name service for HA/Federation. This is useful for making the name node
  92. listen on all interfaces by setting it to 0.0.0.0.
  93. </description>
  94. </property>
  95. <property>
  96. <name>dfs.namenode.secondary.http-address</name>
  97. <value>0.0.0.0:50090</value>
  98. <description>
  99. The secondary namenode http server address and port.
  100. </description>
  101. </property>
  102. <property>
  103. <name>dfs.namenode.secondary.https-address</name>
  104. <value>0.0.0.0:50091</value>
  105. <description>
  106. The secondary namenode HTTPS server address and port.
  107. </description>
  108. </property>
  109. <property>
  110. <name>dfs.datanode.address</name>
  111. <value>0.0.0.0:50010</value>
  112. <description>
  113. The datanode server address and port for data transfer.
  114. </description>
  115. </property>
  116. <property>
  117. <name>dfs.datanode.http.address</name>
  118. <value>0.0.0.0:50075</value>
  119. <description>
  120. The datanode http server address and port.
  121. </description>
  122. </property>
  123. <property>
  124. <name>dfs.datanode.ipc.address</name>
  125. <value>0.0.0.0:50020</value>
  126. <description>
  127. The datanode ipc server address and port.
  128. </description>
  129. </property>
  130. <property>
  131. <name>dfs.datanode.handler.count</name>
  132. <value>10</value>
  133. <description>The number of server threads for the datanode.</description>
  134. </property>
  135. <property>
  136. <name>dfs.namenode.http-address</name>
  137. <value>0.0.0.0:50070</value>
  138. <description>
  139. The address and the base port where the dfs namenode web ui will listen on.
  140. </description>
  141. </property>
  142. <property>
  143. <name>dfs.namenode.http-bind-host</name>
  144. <value></value>
  145. <description>
  146. The actual adress the HTTP server will bind to. If this optional address
  147. is set, it overrides only the hostname portion of dfs.namenode.http-address.
  148. It can also be specified per name node or name service for HA/Federation.
  149. This is useful for making the name node HTTP server listen on all
  150. interfaces by setting it to 0.0.0.0.
  151. </description>
  152. </property>
  153. <property>
  154. <name>dfs.namenode.heartbeat.recheck-interval</name>
  155. <value>300000</value>
  156. <description>
  157. This time decides the interval to check for expired datanodes.
  158. With this value and dfs.heartbeat.interval, the interval of
  159. deciding the datanode is stale or not is also calculated.
  160. The unit of this configuration is millisecond.
  161. </description>
  162. </property>
  163. <property>
  164. <name>dfs.http.policy</name>
  165. <value>HTTP_ONLY</value>
  166. <description>Decide if HTTPS(SSL) is supported on HDFS
  167. This configures the HTTP endpoint for HDFS daemons:
  168. The following values are supported:
  169. - HTTP_ONLY : Service is provided only on http
  170. - HTTPS_ONLY : Service is provided only on https
  171. - HTTP_AND_HTTPS : Service is provided both on http and https
  172. </description>
  173. </property>
  174. <property>
  175. <name>dfs.client.https.need-auth</name>
  176. <value>false</value>
  177. <description>Whether SSL client certificate authentication is required
  178. </description>
  179. </property>
  180. <property>
  181. <name>dfs.client.cached.conn.retry</name>
  182. <value>3</value>
  183. <description>The number of times the HDFS client will pull a socket from the
  184. cache. Once this number is exceeded, the client will try to create a new
  185. socket.
  186. </description>
  187. </property>
  188. <property>
  189. <name>dfs.https.server.keystore.resource</name>
  190. <value>ssl-server.xml</value>
  191. <description>Resource file from which ssl server keystore
  192. information will be extracted
  193. </description>
  194. </property>
  195. <property>
  196. <name>dfs.client.https.keystore.resource</name>
  197. <value>ssl-client.xml</value>
  198. <description>Resource file from which ssl client keystore
  199. information will be extracted
  200. </description>
  201. </property>
  202. <property>
  203. <name>dfs.datanode.https.address</name>
  204. <value>0.0.0.0:50475</value>
  205. <description>The datanode secure http server address and port.</description>
  206. </property>
  207. <property>
  208. <name>dfs.namenode.https-address</name>
  209. <value>0.0.0.0:50470</value>
  210. <description>The namenode secure http server address and port.</description>
  211. </property>
  212. <property>
  213. <name>dfs.namenode.https-bind-host</name>
  214. <value></value>
  215. <description>
  216. The actual adress the HTTPS server will bind to. If this optional address
  217. is set, it overrides only the hostname portion of dfs.namenode.https-address.
  218. It can also be specified per name node or name service for HA/Federation.
  219. This is useful for making the name node HTTPS server listen on all
  220. interfaces by setting it to 0.0.0.0.
  221. </description>
  222. </property>
  223. <property>
  224. <name>dfs.datanode.dns.interface</name>
  225. <value>default</value>
  226. <description>
  227. The name of the Network Interface from which a data node should
  228. report its IP address. e.g. eth2. This setting may be required for some
  229. multi-homed nodes where the DataNodes are assigned multiple hostnames
  230. and it is desirable for the DataNodes to use a non-default hostname.
  231. Prefer using hadoop.security.dns.interface over
  232. dfs.datanode.dns.interface.
  233. </description>
  234. </property>
  235. <property>
  236. <name>dfs.datanode.dns.nameserver</name>
  237. <value>default</value>
  238. <description>
  239. The host name or IP address of the name server (DNS) which a DataNode
  240. should use to determine its own host name.
  241. Prefer using hadoop.security.dns.nameserver over
  242. dfs.datanode.dns.nameserver.
  243. </description>
  244. </property>
  245. <property>
  246. <name>dfs.namenode.backup.address</name>
  247. <value>0.0.0.0:50100</value>
  248. <description>
  249. The backup node server address and port.
  250. If the port is 0 then the server will start on a free port.
  251. </description>
  252. </property>
  253. <property>
  254. <name>dfs.namenode.backup.http-address</name>
  255. <value>0.0.0.0:50105</value>
  256. <description>
  257. The backup node http server address and port.
  258. If the port is 0 then the server will start on a free port.
  259. </description>
  260. </property>
  261. <property>
  262. <name>dfs.namenode.replication.considerLoad</name>
  263. <value>true</value>
  264. <description>Decide if chooseTarget considers the target's load or not
  265. </description>
  266. </property>
  267. <property>
  268. <name>dfs.namenode.replication.considerLoad.factor</name>
  269. <value>2.0</value>
  270. <description>The factor by which a node's load can exceed the average
  271. before being rejected for writes, only if considerLoad is true.
  272. </description>
  273. </property>
  274. <property>
  275. <name>dfs.default.chunk.view.size</name>
  276. <value>32768</value>
  277. <description>The number of bytes to view for a file on the browser.
  278. </description>
  279. </property>
  280. <property>
  281. <name>dfs.datanode.du.reserved</name>
  282. <value>0</value>
  283. <description>Reserved space in bytes per volume. Always leave this much space free for non dfs use.
  284. </description>
  285. </property>
  286. <property>
  287. <name>dfs.namenode.name.dir</name>
  288. <value>file://${hadoop.tmp.dir}/dfs/name</value>
  289. <description>Determines where on the local filesystem the DFS name node
  290. should store the name table(fsimage). If this is a comma-delimited list
  291. of directories then the name table is replicated in all of the
  292. directories, for redundancy. </description>
  293. </property>
  294. <property>
  295. <name>dfs.namenode.name.dir.restore</name>
  296. <value>false</value>
  297. <description>Set to true to enable NameNode to attempt recovering a
  298. previously failed dfs.namenode.name.dir. When enabled, a recovery of any
  299. failed directory is attempted during checkpoint.</description>
  300. </property>
  301. <property>
  302. <name>dfs.namenode.fs-limits.max-component-length</name>
  303. <value>255</value>
  304. <description>Defines the maximum number of bytes in UTF-8 encoding in each
  305. component of a path. A value of 0 will disable the check.</description>
  306. </property>
  307. <property>
  308. <name>dfs.namenode.fs-limits.max-directory-items</name>
  309. <value>1048576</value>
  310. <description>Defines the maximum number of items that a directory may
  311. contain. Cannot set the property to a value less than 1 or more than
  312. 6400000.</description>
  313. </property>
  314. <property>
  315. <name>dfs.namenode.fs-limits.min-block-size</name>
  316. <value>1048576</value>
  317. <description>Minimum block size in bytes, enforced by the Namenode at create
  318. time. This prevents the accidental creation of files with tiny block
  319. sizes (and thus many blocks), which can degrade
  320. performance.</description>
  321. </property>
  322. <property>
  323. <name>dfs.namenode.fs-limits.max-blocks-per-file</name>
  324. <value>1048576</value>
  325. <description>Maximum number of blocks per file, enforced by the Namenode on
  326. write. This prevents the creation of extremely large files which can
  327. degrade performance.</description>
  328. </property>
  329. <property>
  330. <name>dfs.namenode.edits.dir</name>
  331. <value>${dfs.namenode.name.dir}</value>
  332. <description>Determines where on the local filesystem the DFS name node
  333. should store the transaction (edits) file. If this is a comma-delimited list
  334. of directories then the transaction file is replicated in all of the
  335. directories, for redundancy. Default value is same as dfs.namenode.name.dir
  336. </description>
  337. </property>
  338. <property>
  339. <name>dfs.namenode.shared.edits.dir</name>
  340. <value></value>
  341. <description>A directory on shared storage between the multiple namenodes
  342. in an HA cluster. This directory will be written by the active and read
  343. by the standby in order to keep the namespaces synchronized. This directory
  344. does not need to be listed in dfs.namenode.edits.dir above. It should be
  345. left empty in a non-HA cluster.
  346. </description>
  347. </property>
  348. <property>
  349. <name>dfs.namenode.edits.journal-plugin.qjournal</name>
  350. <value>org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager</value>
  351. </property>
  352. <property>
  353. <name>dfs.permissions.enabled</name>
  354. <value>true</value>
  355. <description>
  356. If "true", enable permission checking in HDFS.
  357. If "false", permission checking is turned off,
  358. but all other behavior is unchanged.
  359. Switching from one parameter value to the other does not change the mode,
  360. owner or group of files or directories.
  361. </description>
  362. </property>
  363. <property>
  364. <name>dfs.permissions.superusergroup</name>
  365. <value>supergroup</value>
  366. <description>The name of the group of super-users.</description>
  367. </property>
  368. <!--
  369. <property>
  370. <name>dfs.cluster.administrators</name>
  371. <value>ACL for the admins</value>
  372. <description>This configuration is used to control who can access the
  373. default servlets in the namenode, etc.
  374. </description>
  375. </property>
  376. -->
  377. <property>
  378. <name>dfs.namenode.acls.enabled</name>
  379. <value>false</value>
  380. <description>
  381. Set to true to enable support for HDFS ACLs (Access Control Lists). By
  382. default, ACLs are disabled. When ACLs are disabled, the NameNode rejects
  383. all RPCs related to setting or getting ACLs.
  384. </description>
  385. </property>
  386. <property>
  387. <name>dfs.namenode.lazypersist.file.scrub.interval.sec</name>
  388. <value>300</value>
  389. <description>
  390. The NameNode periodically scans the namespace for LazyPersist files with
  391. missing blocks and unlinks them from the namespace. This configuration key
  392. controls the interval between successive scans. Set it to a negative value
  393. to disable this behavior.
  394. </description>
  395. </property>
  396. <property>
  397. <name>dfs.block.access.token.enable</name>
  398. <value>false</value>
  399. <description>
  400. If "true", access tokens are used as capabilities for accessing datanodes.
  401. If "false", no access tokens are checked on accessing datanodes.
  402. </description>
  403. </property>
  404. <property>
  405. <name>dfs.block.access.key.update.interval</name>
  406. <value>600</value>
  407. <description>
  408. Interval in minutes at which namenode updates its access keys.
  409. </description>
  410. </property>
  411. <property>
  412. <name>dfs.block.access.token.lifetime</name>
  413. <value>600</value>
  414. <description>The lifetime of access tokens in minutes.</description>
  415. </property>
  416. <property>
  417. <name>dfs.datanode.data.dir</name>
  418. <value>file://${hadoop.tmp.dir}/dfs/data</value>
  419. <description>Determines where on the local filesystem an DFS data node
  420. should store its blocks. If this is a comma-delimited
  421. list of directories, then data will be stored in all named
  422. directories, typically on different devices. The directories should be tagged
  423. with corresponding storage types ([SSD]/[DISK]/[ARCHIVE]/[RAM_DISK]) for HDFS
  424. storage policies. The default storage type will be DISK if the directory does
  425. not have a storage type tagged explicitly. Directories that do not exist will
  426. be created if local filesystem permission allows.
  427. </description>
  428. </property>
  429. <property>
  430. <name>dfs.datanode.data.dir.perm</name>
  431. <value>700</value>
  432. <description>Permissions for the directories on on the local filesystem where
  433. the DFS data node store its blocks. The permissions can either be octal or
  434. symbolic.</description>
  435. </property>
  436. <property>
  437. <name>dfs.replication</name>
  438. <value>3</value>
  439. <description>Default block replication.
  440. The actual number of replications can be specified when the file is created.
  441. The default is used if replication is not specified in create time.
  442. </description>
  443. </property>
  444. <property>
  445. <name>dfs.replication.max</name>
  446. <value>512</value>
  447. <description>Maximal block replication.
  448. </description>
  449. </property>
  450. <property>
  451. <name>dfs.namenode.replication.min</name>
  452. <value>1</value>
  453. <description>Minimal block replication.
  454. </description>
  455. </property>
  456. <property>
  457. <name>dfs.namenode.safemode.replication.min</name>
  458. <value></value>
  459. <description>
  460. a separate minimum replication factor for calculating safe block count.
  461. This is an expert level setting.
  462. Setting this lower than the dfs.namenode.replication.min
  463. is not recommend and/or dangerous for production setups.
  464. When it's not set it takes value from dfs.namenode.replication.min
  465. </description>
  466. </property>
  467. <property>
  468. <name>dfs.blocksize</name>
  469. <value>134217728</value>
  470. <description>
  471. The default block size for new files, in bytes.
  472. You can use the following suffix (case insensitive):
  473. k(kilo), m(mega), g(giga), t(tera), p(peta), e(exa) to specify the size (such as 128k, 512m, 1g, etc.),
  474. Or provide complete size in bytes (such as 134217728 for 128 MB).
  475. </description>
  476. </property>
  477. <property>
  478. <name>dfs.client.block.write.retries</name>
  479. <value>3</value>
  480. <description>The number of retries for writing blocks to the data nodes,
  481. before we signal failure to the application.
  482. </description>
  483. </property>
  484. <property>
  485. <name>dfs.client.block.write.replace-datanode-on-failure.enable</name>
  486. <value>true</value>
  487. <description>
  488. If there is a datanode/network failure in the write pipeline,
  489. DFSClient will try to remove the failed datanode from the pipeline
  490. and then continue writing with the remaining datanodes. As a result,
  491. the number of datanodes in the pipeline is decreased. The feature is
  492. to add new datanodes to the pipeline.
  493. This is a site-wide property to enable/disable the feature.
  494. When the cluster size is extremely small, e.g. 3 nodes or less, cluster
  495. administrators may want to set the policy to NEVER in the default
  496. configuration file or disable this feature. Otherwise, users may
  497. experience an unusually high rate of pipeline failures since it is
  498. impossible to find new datanodes for replacement.
  499. See also dfs.client.block.write.replace-datanode-on-failure.policy
  500. </description>
  501. </property>
  502. <property>
  503. <name>dfs.client.block.write.replace-datanode-on-failure.policy</name>
  504. <value>DEFAULT</value>
  505. <description>
  506. This property is used only if the value of
  507. dfs.client.block.write.replace-datanode-on-failure.enable is true.
  508. ALWAYS: always add a new datanode when an existing datanode is removed.
  509. NEVER: never add a new datanode.
  510. DEFAULT:
  511. Let r be the replication number.
  512. Let n be the number of existing datanodes.
  513. Add a new datanode only if r is greater than or equal to 3 and either
  514. (1) floor(r/2) is greater than or equal to n; or
  515. (2) r is greater than n and the block is hflushed/appended.
  516. </description>
  517. </property>
  518. <property>
  519. <name>dfs.client.block.write.replace-datanode-on-failure.best-effort</name>
  520. <value>false</value>
  521. <description>
  522. This property is used only if the value of
  523. dfs.client.block.write.replace-datanode-on-failure.enable is true.
  524. Best effort means that the client will try to replace a failed datanode
  525. in write pipeline (provided that the policy is satisfied), however, it
  526. continues the write operation in case that the datanode replacement also
  527. fails.
  528. Suppose the datanode replacement fails.
  529. false: An exception should be thrown so that the write will fail.
  530. true : The write should be resumed with the remaining datandoes.
  531. Note that setting this property to true allows writing to a pipeline
  532. with a smaller number of datanodes. As a result, it increases the
  533. probability of data loss.
  534. </description>
  535. </property>
  536. <property>
  537. <name>dfs.blockreport.intervalMsec</name>
  538. <value>21600000</value>
  539. <description>Determines block reporting interval in milliseconds.</description>
  540. </property>
  541. <property>
  542. <name>dfs.blockreport.initialDelay</name> <value>0</value>
  543. <description>Delay for first block report in seconds.</description>
  544. </property>
  545. <property>
  546. <name>dfs.blockreport.split.threshold</name>
  547. <value>1000000</value>
  548. <description>If the number of blocks on the DataNode is below this
  549. threshold then it will send block reports for all Storage Directories
  550. in a single message.
  551. If the number of blocks exceeds this threshold then the DataNode will
  552. send block reports for each Storage Directory in separate messages.
  553. Set to zero to always split.
  554. </description>
  555. </property>
  556. <property>
  557. <name>dfs.namenode.max.full.block.report.leases</name>
  558. <value>6</value>
  559. <description>The maximum number of leases for full block reports that the
  560. NameNode will issue at any given time. This prevents the NameNode from
  561. being flooded with full block reports that use up all the RPC handler
  562. threads. This number should never be more than the number of RPC handler
  563. threads or less than 1.
  564. </description>
  565. </property>
  566. <property>
  567. <name>dfs.namenode.full.block.report.lease.length.ms</name>
  568. <value>300000</value>
  569. <description>
  570. The number of milliseconds that the NameNode will wait before invalidating
  571. a full block report lease. This prevents a crashed DataNode from
  572. permanently using up a full block report lease.
  573. </description>
  574. </property>
  575. <property>
  576. <name>dfs.datanode.directoryscan.interval</name>
  577. <value>21600</value>
  578. <description>Interval in seconds for Datanode to scan data directories and
  579. reconcile the difference between blocks in memory and on the disk.
  580. </description>
  581. </property>
  582. <property>
  583. <name>dfs.datanode.directoryscan.threads</name>
  584. <value>1</value>
  585. <description>How many threads should the threadpool used to compile reports
  586. for volumes in parallel have.
  587. </description>
  588. </property>
  589. <property>
  590. <name>dfs.datanode.directoryscan.throttle.limit.ms.per.sec</name>
  591. <value>1000</value>
  592. <description>The report compilation threads are limited to only running for
  593. a given number of milliseconds per second, as configured by the
  594. property. The limit is taken per thread, not in aggregate, e.g. setting
  595. a limit of 100ms for 4 compiler threads will result in each thread being
  596. limited to 100ms, not 25ms.
  597. Note that the throttle does not interrupt the report compiler threads, so the
  598. actual running time of the threads per second will typically be somewhat
  599. higher than the throttle limit, usually by no more than 20%.
  600. Setting this limit to 1000 disables compiler thread throttling. Only
  601. values between 1 and 1000 are valid. Setting an invalid value will result
  602. in the throttle being disbled and an error message being logged. 1000 is
  603. the default setting.
  604. </description>
  605. </property>
  606. <property>
  607. <name>dfs.heartbeat.interval</name>
  608. <value>3</value>
  609. <description>Determines datanode heartbeat interval in seconds.</description>
  610. </property>
  611. <property>
  612. <name>dfs.namenode.handler.count</name>
  613. <value>10</value>
  614. <description>The number of Namenode RPC server threads that listen to
  615. requests from clients.
  616. If dfs.namenode.servicerpc-address is not configured then
  617. Namenode RPC server threads listen to requests from all nodes.
  618. </description>
  619. </property>
  620. <property>
  621. <name>dfs.namenode.service.handler.count</name>
  622. <value>10</value>
  623. <description>The number of Namenode RPC server threads that listen to
  624. requests from DataNodes and from all other non-client nodes.
  625. dfs.namenode.service.handler.count will be valid only if
  626. dfs.namenode.servicerpc-address is configured.
  627. </description>
  628. </property>
  629. <property>
  630. <name>dfs.namenode.lifeline.handler.count</name>
  631. <value>1</value>
  632. <description>
  633. Sets number of RPC server threads the NameNode runs for handling the
  634. lifeline RPC server. The default value is 1, because this RPC server
  635. handles only HA health check requests from ZKFC. These are lightweight
  636. requests that run single-threaded from the ZKFC client side. This property
  637. has no effect if dfs.namenode.lifeline.rpc-address is not defined.
  638. </description>
  639. </property>
  640. <property>
  641. <name>dfs.namenode.safemode.threshold-pct</name>
  642. <value>0.999f</value>
  643. <description>
  644. Specifies the percentage of blocks that should satisfy
  645. the minimal replication requirement defined by dfs.namenode.replication.min.
  646. Values less than or equal to 0 mean not to wait for any particular
  647. percentage of blocks before exiting safemode.
  648. Values greater than 1 will make safe mode permanent.
  649. </description>
  650. </property>
  651. <property>
  652. <name>dfs.namenode.safemode.min.datanodes</name>
  653. <value>0</value>
  654. <description>
  655. Specifies the number of datanodes that must be considered alive
  656. before the name node exits safemode.
  657. Values less than or equal to 0 mean not to take the number of live
  658. datanodes into account when deciding whether to remain in safe mode
  659. during startup.
  660. Values greater than the number of datanodes in the cluster
  661. will make safe mode permanent.
  662. </description>
  663. </property>
  664. <property>
  665. <name>dfs.namenode.safemode.extension</name>
  666. <value>30000</value>
  667. <description>
  668. Determines extension of safe mode in milliseconds
  669. after the threshold level is reached.
  670. </description>
  671. </property>
  672. <property>
  673. <name>dfs.namenode.resource.check.interval</name>
  674. <value>5000</value>
  675. <description>
  676. The interval in milliseconds at which the NameNode resource checker runs.
  677. The checker calculates the number of the NameNode storage volumes whose
  678. available spaces are more than dfs.namenode.resource.du.reserved, and
  679. enters safemode if the number becomes lower than the minimum value
  680. specified by dfs.namenode.resource.checked.volumes.minimum.
  681. </description>
  682. </property>
  683. <property>
  684. <name>dfs.namenode.resource.du.reserved</name>
  685. <value>104857600</value>
  686. <description>
  687. The amount of space to reserve/require for a NameNode storage directory
  688. in bytes. The default is 100MB.
  689. </description>
  690. </property>
  691. <property>
  692. <name>dfs.namenode.resource.checked.volumes</name>
  693. <value></value>
  694. <description>
  695. A list of local directories for the NameNode resource checker to check in
  696. addition to the local edits directories.
  697. </description>
  698. </property>
  699. <property>
  700. <name>dfs.namenode.resource.checked.volumes.minimum</name>
  701. <value>1</value>
  702. <description>
  703. The minimum number of redundant NameNode storage volumes required.
  704. </description>
  705. </property>
  706. <property>
  707. <name>dfs.datanode.balance.bandwidthPerSec</name>
  708. <value>1048576</value>
  709. <description>
  710. Specifies the maximum amount of bandwidth that each datanode
  711. can utilize for the balancing purpose in term of
  712. the number of bytes per second.
  713. </description>
  714. </property>
  715. <property>
  716. <name>dfs.hosts</name>
  717. <value></value>
  718. <description>Names a file that contains a list of hosts that are
  719. permitted to connect to the namenode. The full pathname of the file
  720. must be specified. If the value is empty, all hosts are
  721. permitted.</description>
  722. </property>
  723. <property>
  724. <name>dfs.hosts.exclude</name>
  725. <value></value>
  726. <description>Names a file that contains a list of hosts that are
  727. not permitted to connect to the namenode. The full pathname of the
  728. file must be specified. If the value is empty, no hosts are
  729. excluded.</description>
  730. </property>
  731. <property>
  732. <name>dfs.namenode.max.objects</name>
  733. <value>0</value>
  734. <description>The maximum number of files, directories and blocks
  735. dfs supports. A value of zero indicates no limit to the number
  736. of objects that dfs supports.
  737. </description>
  738. </property>
  739. <property>
  740. <name>dfs.namenode.datanode.registration.ip-hostname-check</name>
  741. <value>true</value>
  742. <description>
  743. If true (the default), then the namenode requires that a connecting
  744. datanode's address must be resolved to a hostname. If necessary, a reverse
  745. DNS lookup is performed. All attempts to register a datanode from an
  746. unresolvable address are rejected.
  747. It is recommended that this setting be left on to prevent accidental
  748. registration of datanodes listed by hostname in the excludes file during a
  749. DNS outage. Only set this to false in environments where there is no
  750. infrastructure to support reverse DNS lookup.
  751. </description>
  752. </property>
  753. <property>
  754. <name>dfs.namenode.decommission.interval</name>
  755. <value>30</value>
  756. <description>Namenode periodicity in seconds to check if decommission is
  757. complete.</description>
  758. </property>
  759. <property>
  760. <name>dfs.namenode.decommission.blocks.per.interval</name>
  761. <value>500000</value>
  762. <description>The approximate number of blocks to process per
  763. decommission interval, as defined in dfs.namenode.decommission.interval.
  764. </description>
  765. </property>
  766. <property>
  767. <name>dfs.namenode.decommission.max.concurrent.tracked.nodes</name>
  768. <value>100</value>
  769. <description>
  770. The maximum number of decommission-in-progress datanodes nodes that will be
  771. tracked at one time by the namenode. Tracking a decommission-in-progress
  772. datanode consumes additional NN memory proportional to the number of blocks
  773. on the datnode. Having a conservative limit reduces the potential impact
  774. of decomissioning a large number of nodes at once.
  775. A value of 0 means no limit will be enforced.
  776. </description>
  777. </property>
  778. <property>
  779. <name>dfs.namenode.replication.interval</name>
  780. <value>3</value>
  781. <description>The periodicity in seconds with which the namenode computes
  782. replication work for datanodes. </description>
  783. </property>
  784. <property>
  785. <name>dfs.namenode.accesstime.precision</name>
  786. <value>3600000</value>
  787. <description>The access time for HDFS file is precise upto this value.
  788. The default value is 1 hour. Setting a value of 0 disables
  789. access times for HDFS.
  790. </description>
  791. </property>
  792. <property>
  793. <name>dfs.datanode.plugins</name>
  794. <value></value>
  795. <description>Comma-separated list of datanode plug-ins to be activated.
  796. </description>
  797. </property>
  798. <property>
  799. <name>dfs.namenode.plugins</name>
  800. <value></value>
  801. <description>Comma-separated list of namenode plug-ins to be activated.
  802. </description>
  803. </property>
  804. <property>
  805. <name>dfs.stream-buffer-size</name>
  806. <value>4096</value>
  807. <description>The size of buffer to stream files.
  808. The size of this buffer should probably be a multiple of hardware
  809. page size (4096 on Intel x86), and it determines how much data is
  810. buffered during read and write operations.</description>
  811. </property>
  812. <property>
  813. <name>dfs.bytes-per-checksum</name>
  814. <value>512</value>
  815. <description>The number of bytes per checksum. Must not be larger than
  816. dfs.stream-buffer-size</description>
  817. </property>
  818. <property>
  819. <name>dfs.client-write-packet-size</name>
  820. <value>65536</value>
  821. <description>Packet size for clients to write</description>
  822. </property>
  823. <property>
  824. <name>dfs.client.write.exclude.nodes.cache.expiry.interval.millis</name>
  825. <value>600000</value>
  826. <description>The maximum period to keep a DN in the excluded nodes list
  827. at a client. After this period, in milliseconds, the previously excluded node(s) will
  828. be removed automatically from the cache and will be considered good for block allocations
  829. again. Useful to lower or raise in situations where you keep a file open for very long
  830. periods (such as a Write-Ahead-Log (WAL) file) to make the writer tolerant to cluster maintenance
  831. restarts. Defaults to 10 minutes.</description>
  832. </property>
  833. <property>
  834. <name>dfs.namenode.checkpoint.dir</name>
  835. <value>file://${hadoop.tmp.dir}/dfs/namesecondary</value>
  836. <description>Determines where on the local filesystem the DFS secondary
  837. name node should store the temporary images to merge.
  838. If this is a comma-delimited list of directories then the image is
  839. replicated in all of the directories for redundancy.
  840. </description>
  841. </property>
  842. <property>
  843. <name>dfs.namenode.checkpoint.edits.dir</name>
  844. <value>${dfs.namenode.checkpoint.dir}</value>
  845. <description>Determines where on the local filesystem the DFS secondary
  846. name node should store the temporary edits to merge.
  847. If this is a comma-delimited list of directories then the edits is
  848. replicated in all of the directories for redundancy.
  849. Default value is same as dfs.namenode.checkpoint.dir
  850. </description>
  851. </property>
  852. <property>
  853. <name>dfs.namenode.checkpoint.period</name>
  854. <value>3600</value>
  855. <description>The number of seconds between two periodic checkpoints.
  856. </description>
  857. </property>
  858. <property>
  859. <name>dfs.namenode.checkpoint.txns</name>
  860. <value>1000000</value>
  861. <description>The Secondary NameNode or CheckpointNode will create a checkpoint
  862. of the namespace every 'dfs.namenode.checkpoint.txns' transactions, regardless
  863. of whether 'dfs.namenode.checkpoint.period' has expired.
  864. </description>
  865. </property>
  866. <property>
  867. <name>dfs.namenode.checkpoint.check.period</name>
  868. <value>60</value>
  869. <description>The SecondaryNameNode and CheckpointNode will poll the NameNode
  870. every 'dfs.namenode.checkpoint.check.period' seconds to query the number
  871. of uncheckpointed transactions.
  872. </description>
  873. </property>
  874. <property>
  875. <name>dfs.namenode.checkpoint.max-retries</name>
  876. <value>3</value>
  877. <description>The SecondaryNameNode retries failed checkpointing. If the
  878. failure occurs while loading fsimage or replaying edits, the number of
  879. retries is limited by this variable.
  880. </description>
  881. </property>
  882. <property>
  883. <name>dfs.namenode.num.checkpoints.retained</name>
  884. <value>2</value>
  885. <description>The number of image checkpoint files (fsimage_*) that will be retained by
  886. the NameNode and Secondary NameNode in their storage directories. All edit
  887. logs (stored on edits_* files) necessary to recover an up-to-date namespace from the oldest retained
  888. checkpoint will also be retained.
  889. </description>
  890. </property>
  891. <property>
  892. <name>dfs.namenode.num.extra.edits.retained</name>
  893. <value>1000000</value>
  894. <description>The number of extra transactions which should be retained
  895. beyond what is minimally necessary for a NN restart.
  896. It does not translate directly to file's age, or the number of files kept,
  897. but to the number of transactions (here "edits" means transactions).
  898. One edit file may contain several transactions (edits).
  899. During checkpoint, NameNode will identify the total number of edits to retain as extra by
  900. checking the latest checkpoint transaction value, subtracted by the value of this property.
  901. Then, it scans edits files to identify the older ones that don't include the computed range of
  902. retained transactions that are to be kept around, and purges them subsequently.
  903. The retainment can be useful for audit purposes or for an HA setup where a remote Standby Node may have
  904. been offline for some time and need to have a longer backlog of retained
  905. edits in order to start again.
  906. Typically each edit is on the order of a few hundred bytes, so the default
  907. of 1 million edits should be on the order of hundreds of MBs or low GBs.
  908. NOTE: Fewer extra edits may be retained than value specified for this setting
  909. if doing so would mean that more segments would be retained than the number
  910. configured by dfs.namenode.max.extra.edits.segments.retained.
  911. </description>
  912. </property>
  913. <property>
  914. <name>dfs.namenode.max.extra.edits.segments.retained</name>
  915. <value>10000</value>
  916. <description>The maximum number of extra edit log segments which should be retained
  917. beyond what is minimally necessary for a NN restart. When used in conjunction with
  918. dfs.namenode.num.extra.edits.retained, this configuration property serves to cap
  919. the number of extra edits files to a reasonable value.
  920. </description>
  921. </property>
  922. <property>
  923. <name>dfs.namenode.delegation.key.update-interval</name>
  924. <value>86400000</value>
  925. <description>The update interval for master key for delegation tokens
  926. in the namenode in milliseconds.
  927. </description>
  928. </property>
  929. <property>
  930. <name>dfs.namenode.delegation.token.max-lifetime</name>
  931. <value>604800000</value>
  932. <description>The maximum lifetime in milliseconds for which a delegation
  933. token is valid.
  934. </description>
  935. </property>
  936. <property>
  937. <name>dfs.namenode.delegation.token.renew-interval</name>
  938. <value>86400000</value>
  939. <description>The renewal interval for delegation token in milliseconds.
  940. </description>
  941. </property>
  942. <property>
  943. <name>dfs.datanode.failed.volumes.tolerated</name>
  944. <value>0</value>
  945. <description>The number of volumes that are allowed to
  946. fail before a datanode stops offering service. By default
  947. any volume failure will cause a datanode to shutdown.
  948. </description>
  949. </property>
  950. <property>
  951. <name>dfs.image.compress</name>
  952. <value>false</value>
  953. <description>Should the dfs image be compressed?
  954. </description>
  955. </property>
  956. <property>
  957. <name>dfs.image.compression.codec</name>
  958. <value>org.apache.hadoop.io.compress.DefaultCodec</value>
  959. <description>If the dfs image is compressed, how should they be compressed?
  960. This has to be a codec defined in io.compression.codecs.
  961. </description>
  962. </property>
  963. <property>
  964. <name>dfs.image.transfer.timeout</name>
  965. <value>60000</value>
  966. <description>
  967. Socket timeout for image transfer in milliseconds. This timeout and the related
  968. dfs.image.transfer.bandwidthPerSec parameter should be configured such
  969. that normal image transfer can complete successfully.
  970. This timeout prevents client hangs when the sender fails during
  971. image transfer. This is socket timeout during image transfer.
  972. </description>
  973. </property>
  974. <property>
  975. <name>dfs.image.transfer.bandwidthPerSec</name>
  976. <value>0</value>
  977. <description>
  978. Maximum bandwidth used for regular image transfers (instead of
  979. bootstrapping the standby namenode), in bytes per second.
  980. This can help keep normal namenode operations responsive during
  981. checkpointing. The maximum bandwidth and timeout in
  982. dfs.image.transfer.timeout should be set such that normal image
  983. transfers can complete successfully.
  984. A default value of 0 indicates that throttling is disabled.
  985. The maximum bandwidth used for bootstrapping standby namenode is
  986. configured with dfs.image.transfer-bootstrap-standby.bandwidthPerSec.
  987. </description>
  988. </property>
  989. <property>
  990. <name>dfs.image.transfer-bootstrap-standby.bandwidthPerSec</name>
  991. <value>0</value>
  992. <description>
  993. Maximum bandwidth used for transferring image to bootstrap standby
  994. namenode, in bytes per second.
  995. A default value of 0 indicates that throttling is disabled. This default
  996. value should be used in most cases, to ensure timely HA operations.
  997. The maximum bandwidth used for regular image transfers is configured
  998. with dfs.image.transfer.bandwidthPerSec.
  999. </description>
  1000. </property>
  1001. <property>
  1002. <name>dfs.image.transfer.chunksize</name>
  1003. <value>65536</value>
  1004. <description>
  1005. Chunksize in bytes to upload the checkpoint.
  1006. Chunked streaming is used to avoid internal buffering of contents
  1007. of image file of huge size.
  1008. </description>
  1009. </property>
  1010. <property>
  1011. <name>dfs.namenode.support.allow.format</name>
  1012. <value>true</value>
  1013. <description>Does HDFS namenode allow itself to be formatted?
  1014. You may consider setting this to false for any production
  1015. cluster, to avoid any possibility of formatting a running DFS.
  1016. </description>
  1017. </property>
  1018. <property>
  1019. <name>dfs.datanode.max.transfer.threads</name>
  1020. <value>4096</value>
  1021. <description>
  1022. Specifies the maximum number of threads to use for transferring data
  1023. in and out of the DN.
  1024. </description>
  1025. </property>
  1026. <property>
  1027. <name>dfs.datanode.scan.period.hours</name>
  1028. <value>504</value>
  1029. <description>
  1030. If this is positive, the DataNode will not scan any
  1031. individual block more than once in the specified scan period.
  1032. If this is negative, the block scanner is disabled.
  1033. If this is set to zero, then the default value of 504 hours
  1034. or 3 weeks is used. Prior versions of HDFS incorrectly documented
  1035. that setting this key to zero will disable the block scanner.
  1036. </description>
  1037. </property>
  1038. <property>
  1039. <name>dfs.block.scanner.volume.bytes.per.second</name>
  1040. <value>1048576</value>
  1041. <description>
  1042. If this is 0, the DataNode's block scanner will be disabled. If this
  1043. is positive, this is the number of bytes per second that the DataNode's
  1044. block scanner will try to scan from each volume.
  1045. </description>
  1046. </property>
  1047. <property>
  1048. <name>dfs.datanode.readahead.bytes</name>
  1049. <value>4194304</value>
  1050. <description>
  1051. While reading block files, if the Hadoop native libraries are available,
  1052. the datanode can use the posix_fadvise system call to explicitly
  1053. page data into the operating system buffer cache ahead of the current
  1054. reader's position. This can improve performance especially when
  1055. disks are highly contended.
  1056. This configuration specifies the number of bytes ahead of the current
  1057. read position which the datanode will attempt to read ahead. This
  1058. feature may be disabled by configuring this property to 0.
  1059. If the native libraries are not available, this configuration has no
  1060. effect.
  1061. </description>
  1062. </property>
  1063. <property>
  1064. <name>dfs.datanode.drop.cache.behind.reads</name>
  1065. <value>false</value>
  1066. <description>
  1067. In some workloads, the data read from HDFS is known to be significantly
  1068. large enough that it is unlikely to be useful to cache it in the
  1069. operating system buffer cache. In this case, the DataNode may be
  1070. configured to automatically purge all data from the buffer cache
  1071. after it is delivered to the client. This behavior is automatically
  1072. disabled for workloads which read only short sections of a block
  1073. (e.g HBase random-IO workloads).
  1074. This may improve performance for some workloads by freeing buffer
  1075. cache space usage for more cacheable data.
  1076. If the Hadoop native libraries are not available, this configuration
  1077. has no effect.
  1078. </description>
  1079. </property>
  1080. <property>
  1081. <name>dfs.datanode.drop.cache.behind.writes</name>
  1082. <value>false</value>
  1083. <description>
  1084. In some workloads, the data written to HDFS is known to be significantly
  1085. large enough that it is unlikely to be useful to cache it in the
  1086. operating system buffer cache. In this case, the DataNode may be
  1087. configured to automatically purge all data from the buffer cache
  1088. after it is written to disk.
  1089. This may improve performance for some workloads by freeing buffer
  1090. cache space usage for more cacheable data.
  1091. If the Hadoop native libraries are not available, this configuration
  1092. has no effect.
  1093. </description>
  1094. </property>
  1095. <property>
  1096. <name>dfs.datanode.sync.behind.writes</name>
  1097. <value>false</value>
  1098. <description>
  1099. If this configuration is enabled, the datanode will instruct the
  1100. operating system to enqueue all written data to the disk immediately
  1101. after it is written. This differs from the usual OS policy which
  1102. may wait for up to 30 seconds before triggering writeback.
  1103. This may improve performance for some workloads by smoothing the
  1104. IO profile for data written to disk.
  1105. If the Hadoop native libraries are not available, this configuration
  1106. has no effect.
  1107. </description>
  1108. </property>
  1109. <property>
  1110. <name>dfs.client.failover.max.attempts</name>
  1111. <value>15</value>
  1112. <description>
  1113. Expert only. The number of client failover attempts that should be
  1114. made before the failover is considered failed.
  1115. </description>
  1116. </property>
  1117. <property>
  1118. <name>dfs.client.failover.sleep.base.millis</name>
  1119. <value>500</value>
  1120. <description>
  1121. Expert only. The time to wait, in milliseconds, between failover
  1122. attempts increases exponentially as a function of the number of
  1123. attempts made so far, with a random factor of +/- 50%. This option
  1124. specifies the base value used in the failover calculation. The
  1125. first failover will retry immediately. The 2nd failover attempt
  1126. will delay at least dfs.client.failover.sleep.base.millis
  1127. milliseconds. And so on.
  1128. </description>
  1129. </property>
  1130. <property>
  1131. <name>dfs.client.failover.sleep.max.millis</name>
  1132. <value>15000</value>
  1133. <description>
  1134. Expert only. The time to wait, in milliseconds, between failover
  1135. attempts increases exponentially as a function of the number of
  1136. attempts made so far, with a random factor of +/- 50%. This option
  1137. specifies the maximum value to wait between failovers.
  1138. Specifically, the time between two failover attempts will not
  1139. exceed +/- 50% of dfs.client.failover.sleep.max.millis
  1140. milliseconds.
  1141. </description>
  1142. </property>
  1143. <property>
  1144. <name>dfs.client.failover.connection.retries</name>
  1145. <value>0</value>
  1146. <description>
  1147. Expert only. Indicates the number of retries a failover IPC client
  1148. will make to establish a server connection.
  1149. </description>
  1150. </property>
  1151. <property>
  1152. <name>dfs.client.failover.connection.retries.on.timeouts</name>
  1153. <value>0</value>
  1154. <description>
  1155. Expert only. The number of retry attempts a failover IPC client
  1156. will make on socket timeout when establishing a server connection.
  1157. </description>
  1158. </property>
  1159. <property>
  1160. <name>dfs.client.datanode-restart.timeout</name>
  1161. <value>30</value>
  1162. <description>
  1163. Expert only. The time to wait, in seconds, from reception of an
  1164. datanode shutdown notification for quick restart, until declaring
  1165. the datanode dead and invoking the normal recovery mechanisms.
  1166. The notification is sent by a datanode when it is being shutdown
  1167. using the shutdownDatanode admin command with the upgrade option.
  1168. </description>
  1169. </property>
  1170. <property>
  1171. <name>dfs.nameservices</name>
  1172. <value></value>
  1173. <description>
  1174. Comma-separated list of nameservices.
  1175. </description>
  1176. </property>
  1177. <property>
  1178. <name>dfs.nameservice.id</name>
  1179. <value></value>
  1180. <description>
  1181. The ID of this nameservice. If the nameservice ID is not
  1182. configured or more than one nameservice is configured for
  1183. dfs.nameservices it is determined automatically by
  1184. matching the local node's address with the configured address.
  1185. </description>
  1186. </property>
  1187. <property>
  1188. <name>dfs.internal.nameservices</name>
  1189. <value></value>
  1190. <description>
  1191. Comma-separated list of nameservices that belong to this cluster.
  1192. Datanode will report to all the nameservices in this list. By default
  1193. this is set to the value of dfs.nameservices.
  1194. </description>
  1195. </property>
  1196. <property>
  1197. <name>dfs.ha.namenodes.EXAMPLENAMESERVICE</name>
  1198. <value></value>
  1199. <description>
  1200. The prefix for a given nameservice, contains a comma-separated
  1201. list of namenodes for a given nameservice (eg EXAMPLENAMESERVICE).
  1202. </description>
  1203. </property>
  1204. <property>
  1205. <name>dfs.ha.namenode.id</name>
  1206. <value></value>
  1207. <description>
  1208. The ID of this namenode. If the namenode ID is not configured it
  1209. is determined automatically by matching the local node's address
  1210. with the configured address.
  1211. </description>
  1212. </property>
  1213. <property>
  1214. <name>dfs.ha.log-roll.period</name>
  1215. <value>120</value>
  1216. <description>
  1217. How often, in seconds, the StandbyNode should ask the active to
  1218. roll edit logs. Since the StandbyNode only reads from finalized
  1219. log segments, the StandbyNode will only be as up-to-date as how
  1220. often the logs are rolled. Note that failover triggers a log roll
  1221. so the StandbyNode will be up to date before it becomes active.
  1222. </description>
  1223. </property>
  1224. <property>
  1225. <name>dfs.ha.tail-edits.period</name>
  1226. <value>60</value>
  1227. <description>
  1228. How often, in seconds, the StandbyNode should check for new
  1229. finalized log segments in the shared edits log.
  1230. </description>
  1231. </property>
  1232. <property>
  1233. <name>dfs.ha.automatic-failover.enabled</name>
  1234. <value>false</value>
  1235. <description>
  1236. Whether automatic failover is enabled. See the HDFS High
  1237. Availability documentation for details on automatic HA
  1238. configuration.
  1239. </description>
  1240. </property>
  1241. <property>
  1242. <name>dfs.client.use.datanode.hostname</name>
  1243. <value>false</value>
  1244. <description>Whether clients should use datanode hostnames when
  1245. connecting to datanodes.
  1246. </description>
  1247. </property>
  1248. <property>
  1249. <name>dfs.datanode.use.datanode.hostname</name>
  1250. <value>false</value>
  1251. <description>Whether datanodes should use datanode hostnames when
  1252. connecting to other datanodes for data transfer.
  1253. </description>
  1254. </property>
  1255. <property>
  1256. <name>dfs.client.local.interfaces</name>
  1257. <value></value>
  1258. <description>A comma separated list of network interface names to use
  1259. for data transfer between the client and datanodes. When creating
  1260. a connection to read from or write to a datanode, the client
  1261. chooses one of the specified interfaces at random and binds its
  1262. socket to the IP of that interface. Individual names may be
  1263. specified as either an interface name (eg "eth0"), a subinterface
  1264. name (eg "eth0:0"), or an IP address (which may be specified using
  1265. CIDR notation to match a range of IPs).
  1266. </description>
  1267. </property>
  1268. <property>
  1269. <name>dfs.datanode.shared.file.descriptor.paths</name>
  1270. <value>/dev/shm,/tmp</value>
  1271. <description>
  1272. A comma-separated list of paths to use when creating file descriptors that
  1273. will be shared between the DataNode and the DFSClient. Typically we use
  1274. /dev/shm, so that the file descriptors will not be written to disk.
  1275. Systems that don't have /dev/shm will fall back to /tmp by default.
  1276. </description>
  1277. </property>
  1278. <property>
  1279. <name>dfs.short.circuit.shared.memory.watcher.interrupt.check.ms</name>
  1280. <value>60000</value>
  1281. <description>
  1282. The length of time in milliseconds that the short-circuit shared memory
  1283. watcher will go between checking for java interruptions sent from other
  1284. threads. This is provided mainly for unit tests.
  1285. </description>
  1286. </property>
  1287. <property>
  1288. <name>dfs.namenode.kerberos.internal.spnego.principal</name>
  1289. <value>${dfs.web.authentication.kerberos.principal}</value>
  1290. </property>
  1291. <property>
  1292. <name>dfs.secondary.namenode.kerberos.internal.spnego.principal</name>
  1293. <value>${dfs.web.authentication.kerberos.principal}</value>
  1294. </property>
  1295. <property>
  1296. <name>dfs.namenode.kerberos.principal.pattern</name>
  1297. <value>*</value>
  1298. <description>
  1299. A client-side RegEx that can be configured to control
  1300. allowed realms to authenticate with (useful in cross-realm env.)
  1301. </description>
  1302. </property>
  1303. <property>
  1304. <name>dfs.namenode.avoid.read.stale.datanode</name>
  1305. <value>false</value>
  1306. <description>
  1307. Indicate whether or not to avoid reading from &quot;stale&quot; datanodes whose
  1308. heartbeat messages have not been received by the namenode
  1309. for more than a specified time interval. Stale datanodes will be
  1310. moved to the end of the node list returned for reading. See
  1311. dfs.namenode.avoid.write.stale.datanode for a similar setting for writes.
  1312. </description>
  1313. </property>
  1314. <property>
  1315. <name>dfs.namenode.avoid.write.stale.datanode</name>
  1316. <value>false</value>
  1317. <description>
  1318. Indicate whether or not to avoid writing to &quot;stale&quot; datanodes whose
  1319. heartbeat messages have not been received by the namenode
  1320. for more than a specified time interval. Writes will avoid using
  1321. stale datanodes unless more than a configured ratio
  1322. (dfs.namenode.write.stale.datanode.ratio) of datanodes are marked as
  1323. stale. See dfs.namenode.avoid.read.stale.datanode for a similar setting
  1324. for reads.
  1325. </description>
  1326. </property>
  1327. <property>
  1328. <name>dfs.namenode.stale.datanode.interval</name>
  1329. <value>30000</value>
  1330. <description>
  1331. Default time interval for marking a datanode as "stale", i.e., if
  1332. the namenode has not received heartbeat msg from a datanode for
  1333. more than this time interval, the datanode will be marked and treated
  1334. as "stale" by default. The stale interval cannot be too small since
  1335. otherwise this may cause too frequent change of stale states.
  1336. We thus set a minimum stale interval value (the default value is 3 times
  1337. of heartbeat interval) and guarantee that the stale interval cannot be less
  1338. than the minimum value. A stale data node is avoided during lease/block
  1339. recovery. It can be conditionally avoided for reads (see
  1340. dfs.namenode.avoid.read.stale.datanode) and for writes (see
  1341. dfs.namenode.avoid.write.stale.datanode).
  1342. </description>
  1343. </property>
  1344. <property>
  1345. <name>dfs.namenode.write.stale.datanode.ratio</name>
  1346. <value>0.5f</value>
  1347. <description>
  1348. When the ratio of number stale datanodes to total datanodes marked
  1349. is greater than this ratio, stop avoiding writing to stale nodes so
  1350. as to prevent causing hotspots.
  1351. </description>
  1352. </property>
  1353. <property>
  1354. <name>dfs.namenode.invalidate.work.pct.per.iteration</name>
  1355. <value>0.32f</value>
  1356. <description>
  1357. *Note*: Advanced property. Change with caution.
  1358. This determines the percentage amount of block
  1359. invalidations (deletes) to do over a single DN heartbeat
  1360. deletion command. The final deletion count is determined by applying this
  1361. percentage to the number of live nodes in the system.
  1362. The resultant number is the number of blocks from the deletion list
  1363. chosen for proper invalidation over a single heartbeat of a single DN.
  1364. Value should be a positive, non-zero percentage in float notation (X.Yf),
  1365. with 1.0f meaning 100%.
  1366. </description>
  1367. </property>
  1368. <property>
  1369. <name>dfs.namenode.replication.work.multiplier.per.iteration</name>
  1370. <value>2</value>
  1371. <description>
  1372. *Note*: Advanced property. Change with caution.
  1373. This determines the total amount of block transfers to begin in
  1374. parallel at a DN, for replication, when such a command list is being
  1375. sent over a DN heartbeat by the NN. The actual number is obtained by
  1376. multiplying this multiplier with the total number of live nodes in the
  1377. cluster. The result number is the number of blocks to begin transfers
  1378. immediately for, per DN heartbeat. This number can be any positive,
  1379. non-zero integer.
  1380. </description>
  1381. </property>
  1382. <property>
  1383. <name>nfs.server.port</name>
  1384. <value>2049</value>
  1385. <description>
  1386. Specify the port number used by Hadoop NFS.
  1387. </description>
  1388. </property>
  1389. <property>
  1390. <name>nfs.mountd.port</name>
  1391. <value>4242</value>
  1392. <description>
  1393. Specify the port number used by Hadoop mount daemon.
  1394. </description>
  1395. </property>
  1396. <property>
  1397. <name>nfs.dump.dir</name>
  1398. <value>/tmp/.hdfs-nfs</value>
  1399. <description>
  1400. This directory is used to temporarily save out-of-order writes before
  1401. writing to HDFS. For each file, the out-of-order writes are dumped after
  1402. they are accumulated to exceed certain threshold (e.g., 1MB) in memory.
  1403. One needs to make sure the directory has enough space.
  1404. </description>
  1405. </property>
  1406. <property>
  1407. <name>nfs.rtmax</name>
  1408. <value>1048576</value>
  1409. <description>This is the maximum size in bytes of a READ request
  1410. supported by the NFS gateway. If you change this, make sure you
  1411. also update the nfs mount's rsize(add rsize= # of bytes to the
  1412. mount directive).
  1413. </description>
  1414. </property>
  1415. <property>
  1416. <name>nfs.wtmax</name>
  1417. <value>1048576</value>
  1418. <description>This is the maximum size in bytes of a WRITE request
  1419. supported by the NFS gateway. If you change this, make sure you
  1420. also update the nfs mount's wsize(add wsize= # of bytes to the
  1421. mount directive).
  1422. </description>
  1423. </property>
  1424. <property>
  1425. <name>nfs.keytab.file</name>
  1426. <value></value>
  1427. <description>
  1428. *Note*: Advanced property. Change with caution.
  1429. This is the path to the keytab file for the hdfs-nfs gateway.
  1430. This is required when the cluster is kerberized.
  1431. </description>
  1432. </property>
  1433. <property>
  1434. <name>nfs.kerberos.principal</name>
  1435. <value></value>
  1436. <description>
  1437. *Note*: Advanced property. Change with caution.
  1438. This is the name of the kerberos principal. This is required when
  1439. the cluster is kerberized.It must be of this format:
  1440. nfs-gateway-user/nfs-gateway-host@kerberos-realm
  1441. </description>
  1442. </property>
  1443. <property>
  1444. <name>nfs.allow.insecure.ports</name>
  1445. <value>true</value>
  1446. <description>
  1447. When set to false, client connections originating from unprivileged ports
  1448. (those above 1023) will be rejected. This is to ensure that clients
  1449. connecting to this NFS Gateway must have had root privilege on the machine
  1450. where they're connecting from.
  1451. </description>
  1452. </property>
  1453. <property>
  1454. <name>dfs.webhdfs.enabled</name>
  1455. <value>true</value>
  1456. <description>
  1457. Enable WebHDFS (REST API) in Namenodes and Datanodes.
  1458. </description>
  1459. </property>
  1460. <property>
  1461. <name>hadoop.fuse.connection.timeout</name>
  1462. <value>300</value>
  1463. <description>
  1464. The minimum number of seconds that we'll cache libhdfs connection objects
  1465. in fuse_dfs. Lower values will result in lower memory consumption; higher
  1466. values may speed up access by avoiding the overhead of creating new
  1467. connection objects.
  1468. </description>
  1469. </property>
  1470. <property>
  1471. <name>hadoop.fuse.timer.period</name>
  1472. <value>5</value>
  1473. <description>
  1474. The number of seconds between cache expiry checks in fuse_dfs. Lower values
  1475. will result in fuse_dfs noticing changes to Kerberos ticket caches more
  1476. quickly.
  1477. </description>
  1478. </property>
  1479. <property>
  1480. <name>dfs.namenode.metrics.logger.period.seconds</name>
  1481. <value>600</value>
  1482. <description>
  1483. This setting controls how frequently the NameNode logs its metrics. The
  1484. logging configuration must also define one or more appenders for
  1485. NameNodeMetricsLog for the metrics to be logged.
  1486. NameNode metrics logging is disabled if this value is set to zero or
  1487. less than zero.
  1488. </description>
  1489. </property>
  1490. <property>
  1491. <name>dfs.datanode.metrics.logger.period.seconds</name>
  1492. <value>600</value>
  1493. <description>
  1494. This setting controls how frequently the DataNode logs its metrics. The
  1495. logging configuration must also define one or more appenders for
  1496. DataNodeMetricsLog for the metrics to be logged.
  1497. DataNode metrics logging is disabled if this value is set to zero or
  1498. less than zero.
  1499. </description>
  1500. </property>
  1501. <property>
  1502. <name>dfs.metrics.percentiles.intervals</name>
  1503. <value></value>
  1504. <description>
  1505. Comma-delimited set of integers denoting the desired rollover intervals
  1506. (in seconds) for percentile latency metrics on the Namenode and Datanode.
  1507. By default, percentile latency metrics are disabled.
  1508. </description>
  1509. </property>
  1510. <property>
  1511. <name>hadoop.user.group.metrics.percentiles.intervals</name>
  1512. <value></value>
  1513. <description>
  1514. A comma-separated list of the granularity in seconds for the metrics
  1515. which describe the 50/75/90/95/99th percentile latency for group resolution
  1516. in milliseconds.
  1517. By default, percentile latency metrics are disabled.
  1518. </description>
  1519. </property>
  1520. <property>
  1521. <name>dfs.encrypt.data.transfer</name>
  1522. <value>false</value>
  1523. <description>
  1524. Whether or not actual block data that is read/written from/to HDFS should
  1525. be encrypted on the wire. This only needs to be set on the NN and DNs,
  1526. clients will deduce this automatically. It is possible to override this setting
  1527. per connection by specifying custom logic via dfs.trustedchannel.resolver.class.
  1528. </description>
  1529. </property>
  1530. <property>
  1531. <name>dfs.encrypt.data.transfer.algorithm</name>
  1532. <value></value>
  1533. <description>
  1534. This value may be set to either "3des" or "rc4". If nothing is set, then
  1535. the configured JCE default on the system is used (usually 3DES.) It is
  1536. widely believed that 3DES is more cryptographically secure, but RC4 is
  1537. substantially faster.
  1538. Note that if AES is supported by both the client and server then this
  1539. encryption algorithm will only be used to initially transfer keys for AES.
  1540. (See dfs.encrypt.data.transfer.cipher.suites.)
  1541. </description>
  1542. </property>
  1543. <property>
  1544. <name>dfs.encrypt.data.transfer.cipher.suites</name>
  1545. <value></value>
  1546. <description>
  1547. This value may be either undefined or AES/CTR/NoPadding. If defined, then
  1548. dfs.encrypt.data.transfer uses the specified cipher suite for data
  1549. encryption. If not defined, then only the algorithm specified in
  1550. dfs.encrypt.data.transfer.algorithm is used. By default, the property is
  1551. not defined.
  1552. </description>
  1553. </property>
  1554. <property>
  1555. <name>dfs.encrypt.data.transfer.cipher.key.bitlength</name>
  1556. <value>128</value>
  1557. <description>
  1558. The key bitlength negotiated by dfsclient and datanode for encryption.
  1559. This value may be set to either 128, 192 or 256.
  1560. </description>
  1561. </property>
  1562. <property>
  1563. <name>dfs.trustedchannel.resolver.class</name>
  1564. <value></value>
  1565. <description>
  1566. TrustedChannelResolver is used to determine whether a channel
  1567. is trusted for plain data transfer. The TrustedChannelResolver is
  1568. invoked on both client and server side. If the resolver indicates
  1569. that the channel is trusted, then the data transfer will not be
  1570. encrypted even if dfs.encrypt.data.transfer is set to true. The
  1571. default implementation returns false indicating that the channel
  1572. is not trusted.
  1573. </description>
  1574. </property>
  1575. <property>
  1576. <name>dfs.data.transfer.protection</name>
  1577. <value></value>
  1578. <description>
  1579. A comma-separated list of SASL protection values used for secured
  1580. connections to the DataNode when reading or writing block data. Possible
  1581. values are authentication, integrity and privacy. authentication means
  1582. authentication only and no integrity or privacy; integrity implies
  1583. authentication and integrity are enabled; and privacy implies all of
  1584. authentication, integrity and privacy are enabled. If
  1585. dfs.encrypt.data.transfer is set to true, then it supersedes the setting for
  1586. dfs.data.transfer.protection and enforces that all connections must use a
  1587. specialized encrypted SASL handshake. This property is ignored for
  1588. connections to a DataNode listening on a privileged port. In this case, it
  1589. is assumed that the use of a privileged port establishes sufficient trust.
  1590. </description>
  1591. </property>
  1592. <property>
  1593. <name>dfs.data.transfer.saslproperties.resolver.class</name>
  1594. <value></value>
  1595. <description>
  1596. SaslPropertiesResolver used to resolve the QOP used for a connection to the
  1597. DataNode when reading or writing block data. If not specified, the value of
  1598. hadoop.security.saslproperties.resolver.class is used as the default value.
  1599. </description>
  1600. </property>
  1601. <property>
  1602. <name>dfs.datanode.hdfs-blocks-metadata.enabled</name>
  1603. <value>false</value>
  1604. <description>
  1605. Boolean which enables backend datanode-side support for the experimental DistributedFileSystem#getFileVBlockStorageLocations API.
  1606. </description>
  1607. </property>
  1608. <property>
  1609. <name>dfs.client.file-block-storage-locations.num-threads</name>
  1610. <value>10</value>
  1611. <description>
  1612. Number of threads used for making parallel RPCs in DistributedFileSystem#getFileBlockStorageLocations().
  1613. </description>
  1614. </property>
  1615. <property>
  1616. <name>dfs.client.file-block-storage-locations.timeout.millis</name>
  1617. <value>1000</value>
  1618. <description>
  1619. Timeout (in milliseconds) for the parallel RPCs made in DistributedFileSystem#getFileBlockStorageLocations().
  1620. </description>
  1621. </property>
  1622. <property>
  1623. <name>dfs.journalnode.rpc-address</name>
  1624. <value>0.0.0.0:8485</value>
  1625. <description>
  1626. The JournalNode RPC server address and port.
  1627. </description>
  1628. </property>
  1629. <property>
  1630. <name>dfs.journalnode.http-address</name>
  1631. <value>0.0.0.0:8480</value>
  1632. <description>
  1633. The address and port the JournalNode HTTP server listens on.
  1634. If the port is 0 then the server will start on a free port.
  1635. </description>
  1636. </property>
  1637. <property>
  1638. <name>dfs.journalnode.https-address</name>
  1639. <value>0.0.0.0:8481</value>
  1640. <description>
  1641. The address and port the JournalNode HTTPS server listens on.
  1642. If the port is 0 then the server will start on a free port.
  1643. </description>
  1644. </property>
  1645. <property>
  1646. <name>dfs.namenode.audit.loggers</name>
  1647. <value>default</value>
  1648. <description>
  1649. List of classes implementing audit loggers that will receive audit events.
  1650. These should be implementations of org.apache.hadoop.hdfs.server.namenode.AuditLogger.
  1651. The special value "default" can be used to reference the default audit
  1652. logger, which uses the configured log system. Installing custom audit loggers
  1653. may affect the performance and stability of the NameNode. Refer to the custom
  1654. logger's documentation for more details.
  1655. </description>
  1656. </property>
  1657. <property>
  1658. <name>dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold</name>
  1659. <value>10737418240</value> <!-- 10 GB -->
  1660. <description>
  1661. Only used when the dfs.datanode.fsdataset.volume.choosing.policy is set to
  1662. org.apache.hadoop.hdfs.server.datanode.fsdataset.AvailableSpaceVolumeChoosingPolicy.
  1663. This setting controls how much DN volumes are allowed to differ in terms of
  1664. bytes of free disk space before they are considered imbalanced. If the free
  1665. space of all the volumes are within this range of each other, the volumes
  1666. will be considered balanced and block assignments will be done on a pure
  1667. round robin basis.
  1668. </description>
  1669. </property>
  1670. <property>
  1671. <name>dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction</name>
  1672. <value>0.75f</value>
  1673. <description>
  1674. Only used when the dfs.datanode.fsdataset.volume.choosing.policy is set to
  1675. org.apache.hadoop.hdfs.server.datanode.fsdataset.AvailableSpaceVolumeChoosingPolicy.
  1676. This setting controls what percentage of new block allocations will be sent
  1677. to volumes with more available disk space than others. This setting should
  1678. be in the range 0.0 - 1.0, though in practice 0.5 - 1.0, since there should
  1679. be no reason to prefer that volumes with less available disk space receive
  1680. more block allocations.
  1681. </description>
  1682. </property>
  1683. <property>
  1684. <name>dfs.namenode.edits.noeditlogchannelflush</name>
  1685. <value>false</value>
  1686. <description>
  1687. Specifies whether to flush edit log file channel. When set, expensive
  1688. FileChannel#force calls are skipped and synchronous disk writes are
  1689. enabled instead by opening the edit log file with RandomAccessFile("rws")
  1690. flags. This can significantly improve the performance of edit log writes
  1691. on the Windows platform.
  1692. Note that the behavior of the "rws" flags is platform and hardware specific
  1693. and might not provide the same level of guarantees as FileChannel#force.
  1694. For example, the write will skip the disk-cache on SAS and SCSI devices
  1695. while it might not on SATA devices. This is an expert level setting,
  1696. change with caution.
  1697. </description>
  1698. </property>
  1699. <property>
  1700. <name>dfs.client.cache.drop.behind.writes</name>
  1701. <value></value>
  1702. <description>
  1703. Just like dfs.datanode.drop.cache.behind.writes, this setting causes the
  1704. page cache to be dropped behind HDFS writes, potentially freeing up more
  1705. memory for other uses. Unlike dfs.datanode.drop.cache.behind.writes, this
  1706. is a client-side setting rather than a setting for the entire datanode.
  1707. If present, this setting will override the DataNode default.
  1708. If the native libraries are not available to the DataNode, this
  1709. configuration has no effect.
  1710. </description>
  1711. </property>
  1712. <property>
  1713. <name>dfs.client.cache.drop.behind.reads</name>
  1714. <value></value>
  1715. <description>
  1716. Just like dfs.datanode.drop.cache.behind.reads, this setting causes the
  1717. page cache to be dropped behind HDFS reads, potentially freeing up more
  1718. memory for other uses. Unlike dfs.datanode.drop.cache.behind.reads, this
  1719. is a client-side setting rather than a setting for the entire datanode. If
  1720. present, this setting will override the DataNode default.
  1721. If the native libraries are not available to the DataNode, this
  1722. configuration has no effect.
  1723. </description>
  1724. </property>
  1725. <property>
  1726. <name>dfs.client.cache.readahead</name>
  1727. <value></value>
  1728. <description>
  1729. When using remote reads, this setting causes the datanode to
  1730. read ahead in the block file using posix_fadvise, potentially decreasing
  1731. I/O wait times. Unlike dfs.datanode.readahead.bytes, this is a client-side
  1732. setting rather than a setting for the entire datanode. If present, this
  1733. setting will override the DataNode default.
  1734. When using local reads, this setting determines how much readahead we do in
  1735. BlockReaderLocal.
  1736. If the native libraries are not available to the DataNode, this
  1737. configuration has no effect.
  1738. </description>
  1739. </property>
  1740. <property>
  1741. <name>dfs.namenode.enable.retrycache</name>
  1742. <value>true</value>
  1743. <description>
  1744. This enables the retry cache on the namenode. Namenode tracks for
  1745. non-idempotent requests the corresponding response. If a client retries the
  1746. request, the response from the retry cache is sent. Such operations
  1747. are tagged with annotation @AtMostOnce in namenode protocols. It is
  1748. recommended that this flag be set to true. Setting it to false, will result
  1749. in clients getting failure responses to retried request. This flag must
  1750. be enabled in HA setup for transparent fail-overs.
  1751. The entries in the cache have expiration time configurable
  1752. using dfs.namenode.retrycache.expirytime.millis.
  1753. </description>
  1754. </property>
  1755. <property>
  1756. <name>dfs.namenode.retrycache.expirytime.millis</name>
  1757. <value>600000</value>
  1758. <description>
  1759. The time for which retry cache entries are retained.
  1760. </description>
  1761. </property>
  1762. <property>
  1763. <name>dfs.namenode.retrycache.heap.percent</name>
  1764. <value>0.03f</value>
  1765. <description>
  1766. This parameter configures the heap size allocated for retry cache
  1767. (excluding the response cached). This corresponds to approximately
  1768. 4096 entries for every 64MB of namenode process java heap size.
  1769. Assuming retry cache entry expiration time (configured using
  1770. dfs.namenode.retrycache.expirytime.millis) of 10 minutes, this
  1771. enables retry cache to support 7 operations per second sustained
  1772. for 10 minutes. As the heap size is increased, the operation rate
  1773. linearly increases.
  1774. </description>
  1775. </property>
  1776. <property>
  1777. <name>dfs.client.mmap.enabled</name>
  1778. <value>true</value>
  1779. <description>
  1780. If this is set to false, the client won't attempt to perform memory-mapped reads.
  1781. </description>
  1782. </property>
  1783. <property>
  1784. <name>dfs.client.mmap.cache.size</name>
  1785. <value>256</value>
  1786. <description>
  1787. When zero-copy reads are used, the DFSClient keeps a cache of recently used
  1788. memory mapped regions. This parameter controls the maximum number of
  1789. entries that we will keep in that cache.
  1790. The larger this number is, the more file descriptors we will potentially
  1791. use for memory-mapped files. mmaped files also use virtual address space.
  1792. You may need to increase your ulimit virtual address space limits before
  1793. increasing the client mmap cache size.
  1794. Note that you can still do zero-copy reads when this size is set to 0.
  1795. </description>
  1796. </property>
  1797. <property>
  1798. <name>dfs.client.mmap.cache.timeout.ms</name>
  1799. <value>3600000</value>
  1800. <description>
  1801. The minimum length of time that we will keep an mmap entry in the cache
  1802. between uses. If an entry is in the cache longer than this, and nobody
  1803. uses it, it will be removed by a background thread.
  1804. </description>
  1805. </property>
  1806. <property>
  1807. <name>dfs.client.mmap.retry.timeout.ms</name>
  1808. <value>300000</value>
  1809. <description>
  1810. The minimum amount of time that we will wait before retrying a failed mmap
  1811. operation.
  1812. </description>
  1813. </property>
  1814. <property>
  1815. <name>dfs.client.short.circuit.replica.stale.threshold.ms</name>
  1816. <value>1800000</value>
  1817. <description>
  1818. The maximum amount of time that we will consider a short-circuit replica to
  1819. be valid, if there is no communication from the DataNode. After this time
  1820. has elapsed, we will re-fetch the short-circuit replica even if it is in
  1821. the cache.
  1822. </description>
  1823. </property>
  1824. <property>
  1825. <name>dfs.namenode.path.based.cache.block.map.allocation.percent</name>
  1826. <value>0.25</value>
  1827. <description>
  1828. The percentage of the Java heap which we will allocate to the cached blocks
  1829. map. The cached blocks map is a hash map which uses chained hashing.
  1830. Smaller maps may be accessed more slowly if the number of cached blocks is
  1831. large; larger maps will consume more memory.
  1832. </description>
  1833. </property>
  1834. <property>
  1835. <name>dfs.datanode.max.locked.memory</name>
  1836. <value>0</value>
  1837. <description>
  1838. The amount of memory in bytes to use for caching of block replicas in
  1839. memory on the datanode. The datanode's maximum locked memory soft ulimit
  1840. (RLIMIT_MEMLOCK) must be set to at least this value, else the datanode
  1841. will abort on startup.
  1842. By default, this parameter is set to 0, which disables in-memory caching.
  1843. If the native libraries are not available to the DataNode, this
  1844. configuration has no effect.
  1845. </description>
  1846. </property>
  1847. <property>
  1848. <name>dfs.namenode.list.cache.directives.num.responses</name>
  1849. <value>100</value>
  1850. <description>
  1851. This value controls the number of cache directives that the NameNode will
  1852. send over the wire in response to a listDirectives RPC.
  1853. </description>
  1854. </property>
  1855. <property>
  1856. <name>dfs.namenode.list.cache.pools.num.responses</name>
  1857. <value>100</value>
  1858. <description>
  1859. This value controls the number of cache pools that the NameNode will
  1860. send over the wire in response to a listPools RPC.
  1861. </description>
  1862. </property>
  1863. <property>
  1864. <name>dfs.namenode.path.based.cache.refresh.interval.ms</name>
  1865. <value>30000</value>
  1866. <description>
  1867. The amount of milliseconds between subsequent path cache rescans. Path
  1868. cache rescans are when we calculate which blocks should be cached, and on
  1869. what datanodes.
  1870. By default, this parameter is set to 30 seconds.
  1871. </description>
  1872. </property>
  1873. <property>
  1874. <name>dfs.namenode.path.based.cache.retry.interval.ms</name>
  1875. <value>30000</value>
  1876. <description>
  1877. When the NameNode needs to uncache something that is cached, or cache
  1878. something that is not cached, it must direct the DataNodes to do so by
  1879. sending a DNA_CACHE or DNA_UNCACHE command in response to a DataNode
  1880. heartbeat. This parameter controls how frequently the NameNode will
  1881. resend these commands.
  1882. </description>
  1883. </property>
  1884. <property>
  1885. <name>dfs.datanode.fsdatasetcache.max.threads.per.volume</name>
  1886. <value>4</value>
  1887. <description>
  1888. The maximum number of threads per volume to use for caching new data
  1889. on the datanode. These threads consume both I/O and CPU. This can affect
  1890. normal datanode operations.
  1891. </description>
  1892. </property>
  1893. <property>
  1894. <name>dfs.cachereport.intervalMsec</name>
  1895. <value>10000</value>
  1896. <description>
  1897. Determines cache reporting interval in milliseconds. After this amount of
  1898. time, the DataNode sends a full report of its cache state to the NameNode.
  1899. The NameNode uses the cache report to update its map of cached blocks to
  1900. DataNode locations.
  1901. This configuration has no effect if in-memory caching has been disabled by
  1902. setting dfs.datanode.max.locked.memory to 0 (which is the default).
  1903. If the native libraries are not available to the DataNode, this
  1904. configuration has no effect.
  1905. </description>
  1906. </property>
  1907. <property>
  1908. <name>dfs.namenode.edit.log.autoroll.multiplier.threshold</name>
  1909. <value>2.0</value>
  1910. <description>
  1911. Determines when an active namenode will roll its own edit log.
  1912. The actual threshold (in number of edits) is determined by multiplying
  1913. this value by dfs.namenode.checkpoint.txns.
  1914. This prevents extremely large edit files from accumulating on the active
  1915. namenode, which can cause timeouts during namenode startup and pose an
  1916. administrative hassle. This behavior is intended as a failsafe for when
  1917. the standby or secondary namenode fail to roll the edit log by the normal
  1918. checkpoint threshold.
  1919. </description>
  1920. </property>
  1921. <property>
  1922. <name>dfs.namenode.edit.log.autoroll.check.interval.ms</name>
  1923. <value>300000</value>
  1924. <description>
  1925. How often an active namenode will check if it needs to roll its edit log,
  1926. in milliseconds.
  1927. </description>
  1928. </property>
  1929. <property>
  1930. <name>dfs.webhdfs.user.provider.user.pattern</name>
  1931. <value>^[A-Za-z_][A-Za-z0-9._-]*[$]?$</value>
  1932. <description>
  1933. Valid pattern for user and group names for webhdfs, it must be a valid java regex.
  1934. </description>
  1935. </property>
  1936. <property>
  1937. <name>dfs.client.context</name>
  1938. <value>default</value>
  1939. <description>
  1940. The name of the DFSClient context that we should use. Clients that share
  1941. a context share a socket cache and short-circuit cache, among other things.
  1942. You should only change this if you don't want to share with another set of
  1943. threads.
  1944. </description>
  1945. </property>
  1946. <property>
  1947. <name>dfs.client.read.shortcircuit</name>
  1948. <value>false</value>
  1949. <description>
  1950. This configuration parameter turns on short-circuit local reads.
  1951. </description>
  1952. </property>
  1953. <property>
  1954. <name>dfs.client.socket.send.buffer.size</name>
  1955. <value>131072</value>
  1956. <description>
  1957. Socket send buffer size for a write pipeline in DFSClient side.
  1958. This may affect TCP connection throughput.
  1959. If it is set to zero or negative value,
  1960. no buffer size will be set explicitly,
  1961. thus enable tcp auto-tuning on some system.
  1962. </description>
  1963. </property>
  1964. <property>
  1965. <name>dfs.domain.socket.path</name>
  1966. <value></value>
  1967. <description>
  1968. Optional. This is a path to a UNIX domain socket that will be used for
  1969. communication between the DataNode and local HDFS clients.
  1970. If the string "_PORT" is present in this path, it will be replaced by the
  1971. TCP port of the DataNode.
  1972. </description>
  1973. </property>
  1974. <property>
  1975. <name>dfs.client.read.shortcircuit.skip.checksum</name>
  1976. <value>false</value>
  1977. <description>
  1978. If this configuration parameter is set,
  1979. short-circuit local reads will skip checksums.
  1980. This is normally not recommended,
  1981. but it may be useful for special setups.
  1982. You might consider using this
  1983. if you are doing your own checksumming outside of HDFS.
  1984. </description>
  1985. </property>
  1986. <property>
  1987. <name>dfs.client.read.shortcircuit.streams.cache.size</name>
  1988. <value>256</value>
  1989. <description>
  1990. The DFSClient maintains a cache of recently opened file descriptors.
  1991. This parameter controls the maximum number of file descriptors in the cache.
  1992. Setting this higher will use more file descriptors,
  1993. but potentially provide better performance on workloads
  1994. involving lots of seeks.
  1995. </description>
  1996. </property>
  1997. <property>
  1998. <name>dfs.client.read.shortcircuit.streams.cache.expiry.ms</name>
  1999. <value>300000</value>
  2000. <description>
  2001. This controls the minimum amount of time
  2002. file descriptors need to sit in the client cache context
  2003. before they can be closed for being inactive for too long.
  2004. </description>
  2005. </property>
  2006. <property>
  2007. <name>dfs.datanode.shared.file.descriptor.paths</name>
  2008. <value>/dev/shm,/tmp</value>
  2009. <description>
  2010. Comma separated paths to the directory on which
  2011. shared memory segments are created.
  2012. The client and the DataNode exchange information via
  2013. this shared memory segment.
  2014. It tries paths in order until creation of shared memory segment succeeds.
  2015. </description>
  2016. </property>
  2017. <property>
  2018. <name>dfs.namenode.audit.log.debug.cmdlist</name>
  2019. <value></value>
  2020. <description>
  2021. A comma separated list of NameNode commands that are written to the HDFS
  2022. namenode audit log only if the audit log level is debug.
  2023. </description>
  2024. </property>
  2025. <property>
  2026. <name>dfs.client.use.legacy.blockreader.local</name>
  2027. <value>false</value>
  2028. <description>
  2029. Legacy short-circuit reader implementation based on HDFS-2246 is used
  2030. if this configuration parameter is true.
  2031. This is for the platforms other than Linux
  2032. where the new implementation based on HDFS-347 is not available.
  2033. </description>
  2034. </property>
  2035. <property>
  2036. <name>dfs.block.local-path-access.user</name>
  2037. <value></value>
  2038. <description>
  2039. Comma separated list of the users allowd to open block files
  2040. on legacy short-circuit local read.
  2041. </description>
  2042. </property>
  2043. <property>
  2044. <name>dfs.client.domain.socket.data.traffic</name>
  2045. <value>false</value>
  2046. <description>
  2047. This control whether we will try to pass normal data traffic
  2048. over UNIX domain socket rather than over TCP socket
  2049. on node-local data transfer.
  2050. This is currently experimental and turned off by default.
  2051. </description>
  2052. </property>
  2053. <property>
  2054. <name>dfs.namenode.reject-unresolved-dn-topology-mapping</name>
  2055. <value>false</value>
  2056. <description>
  2057. If the value is set to true, then namenode will reject datanode
  2058. registration if the topology mapping for a datanode is not resolved and
  2059. NULL is returned (script defined by net.topology.script.file.name fails
  2060. to execute). Otherwise, datanode will be registered and the default rack
  2061. will be assigned as the topology path. Topology paths are important for
  2062. data resiliency, since they define fault domains. Thus it may be unwanted
  2063. behavior to allow datanode registration with the default rack if the
  2064. resolving topology failed.
  2065. </description>
  2066. </property>
  2067. <property>
  2068. <name>dfs.client.slow.io.warning.threshold.ms</name>
  2069. <value>30000</value>
  2070. <description>The threshold in milliseconds at which we will log a slow
  2071. io warning in a dfsclient. By default, this parameter is set to 30000
  2072. milliseconds (30 seconds).
  2073. </description>
  2074. </property>
  2075. <property>
  2076. <name>dfs.datanode.slow.io.warning.threshold.ms</name>
  2077. <value>300</value>
  2078. <description>The threshold in milliseconds at which we will log a slow
  2079. io warning in a datanode. By default, this parameter is set to 300
  2080. milliseconds.
  2081. </description>
  2082. </property>
  2083. <property>
  2084. <name>dfs.namenode.xattrs.enabled</name>
  2085. <value>true</value>
  2086. <description>
  2087. Whether support for extended attributes is enabled on the NameNode.
  2088. </description>
  2089. </property>
  2090. <property>
  2091. <name>dfs.namenode.fs-limits.max-xattrs-per-inode</name>
  2092. <value>32</value>
  2093. <description>
  2094. Maximum number of extended attributes per inode.
  2095. </description>
  2096. </property>
  2097. <property>
  2098. <name>dfs.namenode.fs-limits.max-xattr-size</name>
  2099. <value>16384</value>
  2100. <description>
  2101. The maximum combined size of the name and value of an extended attribute
  2102. in bytes. It should be larger than 0, and less than or equal to maximum
  2103. size hard limit which is 32768.
  2104. </description>
  2105. </property>
  2106. <property>
  2107. <name>dfs.namenode.startup.delay.block.deletion.sec</name>
  2108. <value>0</value>
  2109. <description>The delay in seconds at which we will pause the blocks deletion
  2110. after Namenode startup. By default it's disabled.
  2111. In the case a directory has large number of directories and files are
  2112. deleted, suggested delay is one hour to give the administrator enough time
  2113. to notice large number of pending deletion blocks and take corrective
  2114. action.
  2115. </description>
  2116. </property>
  2117. <property>
  2118. <name>dfs.namenode.list.encryption.zones.num.responses</name>
  2119. <value>100</value>
  2120. <description>When listing encryption zones, the maximum number of zones
  2121. that will be returned in a batch. Fetching the list incrementally in
  2122. batches improves namenode performance.
  2123. </description>
  2124. </property>
  2125. <property>
  2126. <name>dfs.namenode.inotify.max.events.per.rpc</name>
  2127. <value>1000</value>
  2128. <description>Maximum number of events that will be sent to an inotify client
  2129. in a single RPC response. The default value attempts to amortize away
  2130. the overhead for this RPC while avoiding huge memory requirements for the
  2131. client and NameNode (1000 events should consume no more than 1 MB.)
  2132. </description>
  2133. </property>
  2134. <property>
  2135. <name>dfs.user.home.dir.prefix</name>
  2136. <value>/user</value>
  2137. <description>The directory to prepend to user name to get the user's
  2138. home direcotry.
  2139. </description>
  2140. </property>
  2141. <property>
  2142. <name>dfs.datanode.cache.revocation.timeout.ms</name>
  2143. <value>900000</value>
  2144. <description>When the DFSClient reads from a block file which the DataNode is
  2145. caching, the DFSClient can skip verifying checksums. The DataNode will
  2146. keep the block file in cache until the client is done. If the client takes
  2147. an unusually long time, though, the DataNode may need to evict the block
  2148. file from the cache anyway. This value controls how long the DataNode will
  2149. wait for the client to release a replica that it is reading without
  2150. checksums.
  2151. </description>
  2152. </property>
  2153. <property>
  2154. <name>dfs.datanode.cache.revocation.polling.ms</name>
  2155. <value>500</value>
  2156. <description>How often the DataNode should poll to see if the clients have
  2157. stopped using a replica that the DataNode wants to uncache.
  2158. </description>
  2159. </property>
  2160. <property>
  2161. <name>dfs.datanode.block.id.layout.upgrade.threads</name>
  2162. <value>12</value>
  2163. <description>The number of threads to use when creating hard links from
  2164. current to previous blocks during upgrade of a DataNode to block ID-based
  2165. block layout (see HDFS-6482 for details on the layout).</description>
  2166. </property>
  2167. <property>
  2168. <name>dfs.encryption.key.provider.uri</name>
  2169. <description>
  2170. The KeyProvider to use when interacting with encryption keys used
  2171. when reading and writing to an encryption zone.
  2172. </description>
  2173. </property>
  2174. <property>
  2175. <name>dfs.storage.policy.enabled</name>
  2176. <value>true</value>
  2177. <description>
  2178. Allow users to change the storage policy on files and directories.
  2179. </description>
  2180. </property>
  2181. <property>
  2182. <name>dfs.namenode.legacy-oiv-image.dir</name>
  2183. <value></value>
  2184. <description>Determines where to save the namespace in the old fsimage format
  2185. during checkpointing by standby NameNode or SecondaryNameNode. Users can
  2186. dump the contents of the old format fsimage by oiv_legacy command. If
  2187. the value is not specified, old format fsimage will not be saved in
  2188. checkpoint.
  2189. </description>
  2190. </property>
  2191. <property>
  2192. <name>dfs.namenode.top.enabled</name>
  2193. <value>true</value>
  2194. <description>Enable nntop: reporting top users on namenode
  2195. </description>
  2196. </property>
  2197. <property>
  2198. <name>dfs.namenode.top.window.num.buckets</name>
  2199. <value>10</value>
  2200. <description>Number of buckets in the rolling window implementation of nntop
  2201. </description>
  2202. </property>
  2203. <property>
  2204. <name>dfs.namenode.top.num.users</name>
  2205. <value>10</value>
  2206. <description>Number of top users returned by the top tool
  2207. </description>
  2208. </property>
  2209. <property>
  2210. <name>dfs.namenode.top.windows.minutes</name>
  2211. <value>1,5,25</value>
  2212. <description>comma separated list of nntop reporting periods in minutes
  2213. </description>
  2214. </property>
  2215. <property>
  2216. <name>dfs.namenode.blocks.per.postponedblocks.rescan</name>
  2217. <value>10000</value>
  2218. <description>Number of blocks to rescan for each iteration of
  2219. postponedMisreplicatedBlocks.
  2220. </description>
  2221. </property>
  2222. <property>
  2223. <name>dfs.datanode.block-pinning.enabled</name>
  2224. <value>false</value>
  2225. <description>Whether pin blocks on favored DataNode.</description>
  2226. </property>
  2227. <property>
  2228. <name>dfs.client.block.write.locateFollowingBlock.initial.delay.ms</name>
  2229. <value>400</value>
  2230. <description>The initial delay (unit is ms) for locateFollowingBlock,
  2231. the delay time will increase exponentially(double) for each retry.
  2232. </description>
  2233. </property>
  2234. <property>
  2235. <name>dfs.ha.zkfc.nn.http.timeout.ms</name>
  2236. <value>20000</value>
  2237. <description>
  2238. The HTTP connection and read timeout value (unit is ms ) when DFS ZKFC
  2239. tries to get local NN thread dump after local NN becomes
  2240. SERVICE_NOT_RESPONDING or SERVICE_UNHEALTHY.
  2241. If it is set to zero, DFS ZKFC won't get local NN thread dump.
  2242. </description>
  2243. </property>
  2244. <property>
  2245. <name>dfs.namenode.quota.init-threads</name>
  2246. <value>4</value>
  2247. <description>
  2248. The number of concurrent threads to be used in quota initialization. The
  2249. speed of quota initialization also affects the namenode fail-over latency.
  2250. If the size of name space is big, try increasing this.
  2251. </description>
  2252. </property>
  2253. <property>
  2254. <name>dfs.datanode.transfer.socket.send.buffer.size</name>
  2255. <value>131072</value>
  2256. <description>
  2257. Socket send buffer size for DataXceiver (mirroring packets to downstream
  2258. in pipeline). This may affect TCP connection throughput.
  2259. If it is set to zero or negative value, no buffer size will be set
  2260. explicitly, thus enable tcp auto-tuning on some system.
  2261. </description>
  2262. </property>
  2263. <property>
  2264. <name>dfs.datanode.transfer.socket.recv.buffer.size</name>
  2265. <value>131072</value>
  2266. <description>
  2267. Socket receive buffer size for DataXceiver (receiving packets from client
  2268. during block writing). This may affect TCP connection throughput.
  2269. If it is set to zero or negative value, no buffer size will be set
  2270. explicitly, thus enable tcp auto-tuning on some system.
  2271. </description>
  2272. </property>
  2273. <property>
  2274. <name>dfs.namenode.upgrade.domain.factor</name>
  2275. <value>${dfs.replication}</value>
  2276. <description>
  2277. This is valid only when block placement policy is set to
  2278. BlockPlacementPolicyWithUpgradeDomain. It defines the number of
  2279. unique upgrade domains any block's replicas should have.
  2280. When the number of replicas is less or equal to this value, the policy
  2281. ensures each replica has an unique upgrade domain. When the number of
  2282. replicas is greater than this value, the policy ensures the number of
  2283. unique domains is at least this value.
  2284. </description>
  2285. </property>
  2286. <property>
  2287. <name>dfs.ha.zkfc.port</name>
  2288. <value>8019</value>
  2289. <description>
  2290. RPC port for Zookeeper Failover Controller.
  2291. </description>
  2292. </property>
  2293. </configuration>