HDFSHighAvailabilityWithQJM.apt.vm 36 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816
  1. ~~ Licensed under the Apache License, Version 2.0 (the "License");
  2. ~~ you may not use this file except in compliance with the License.
  3. ~~ You may obtain a copy of the License at
  4. ~~
  5. ~~ http://www.apache.org/licenses/LICENSE-2.0
  6. ~~
  7. ~~ Unless required by applicable law or agreed to in writing, software
  8. ~~ distributed under the License is distributed on an "AS IS" BASIS,
  9. ~~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  10. ~~ See the License for the specific language governing permissions and
  11. ~~ limitations under the License. See accompanying LICENSE file.
  12. ---
  13. Hadoop Distributed File System-${project.version} - High Availability
  14. ---
  15. ---
  16. ${maven.build.timestamp}
  17. HDFS High Availability Using the Quorum Journal Manager
  18. %{toc|section=1|fromDepth=0}
  19. * {Purpose}
  20. This guide provides an overview of the HDFS High Availability (HA) feature
  21. and how to configure and manage an HA HDFS cluster, using the Quorum Journal
  22. Manager (QJM) feature.
  23. This document assumes that the reader has a general understanding of
  24. general components and node types in an HDFS cluster. Please refer to the
  25. HDFS Architecture guide for details.
  26. * {Note: Using the Quorum Journal Manager or Conventional Shared Storage}
  27. This guide discusses how to configure and use HDFS HA using the Quorum
  28. Journal Manager (QJM) to share edit logs between the Active and Standby
  29. NameNodes. For information on how to configure HDFS HA using NFS for shared
  30. storage instead of the QJM, please see
  31. {{{./HDFSHighAvailabilityWithNFS.html}this alternative guide.}}
  32. * {Background}
  33. Prior to Hadoop 2.0.0, the NameNode was a single point of failure (SPOF) in
  34. an HDFS cluster. Each cluster had a single NameNode, and if that machine or
  35. process became unavailable, the cluster as a whole would be unavailable
  36. until the NameNode was either restarted or brought up on a separate machine.
  37. This impacted the total availability of the HDFS cluster in two major ways:
  38. * In the case of an unplanned event such as a machine crash, the cluster would
  39. be unavailable until an operator restarted the NameNode.
  40. * Planned maintenance events such as software or hardware upgrades on the
  41. NameNode machine would result in windows of cluster downtime.
  42. The HDFS High Availability feature addresses the above problems by providing
  43. the option of running two redundant NameNodes in the same cluster in an
  44. Active/Passive configuration with a hot standby. This allows a fast failover to
  45. a new NameNode in the case that a machine crashes, or a graceful
  46. administrator-initiated failover for the purpose of planned maintenance.
  47. * {Architecture}
  48. In a typical HA cluster, two separate machines are configured as NameNodes.
  49. At any point in time, exactly one of the NameNodes is in an <Active> state,
  50. and the other is in a <Standby> state. The Active NameNode is responsible
  51. for all client operations in the cluster, while the Standby is simply acting
  52. as a slave, maintaining enough state to provide a fast failover if
  53. necessary.
  54. In order for the Standby node to keep its state synchronized with the Active
  55. node, both nodes communicate with a group of separate daemons called
  56. "JournalNodes" (JNs). When any namespace modification is performed by the
  57. Active node, it durably logs a record of the modification to a majority of
  58. these JNs. The Standby node is capable of reading the edits from the JNs, and
  59. is constantly watching them for changes to the edit log. As the Standby Node
  60. sees the edits, it applies them to its own namespace. In the event of a
  61. failover, the Standby will ensure that it has read all of the edits from the
  62. JounalNodes before promoting itself to the Active state. This ensures that the
  63. namespace state is fully synchronized before a failover occurs.
  64. In order to provide a fast failover, it is also necessary that the Standby node
  65. have up-to-date information regarding the location of blocks in the cluster.
  66. In order to achieve this, the DataNodes are configured with the location of
  67. both NameNodes, and send block location information and heartbeats to both.
  68. It is vital for the correct operation of an HA cluster that only one of the
  69. NameNodes be Active at a time. Otherwise, the namespace state would quickly
  70. diverge between the two, risking data loss or other incorrect results. In
  71. order to ensure this property and prevent the so-called "split-brain scenario,"
  72. the JournalNodes will only ever allow a single NameNode to be a writer at a
  73. time. During a failover, the NameNode which is to become active will simply
  74. take over the role of writing to the JournalNodes, which will effectively
  75. prevent the other NameNode from continuing in the Active state, allowing the
  76. new Active to safely proceed with failover.
  77. * {Hardware resources}
  78. In order to deploy an HA cluster, you should prepare the following:
  79. * <<NameNode machines>> - the machines on which you run the Active and
  80. Standby NameNodes should have equivalent hardware to each other, and
  81. equivalent hardware to what would be used in a non-HA cluster.
  82. * <<JournalNode machines>> - the machines on which you run the JournalNodes.
  83. The JournalNode daemon is relatively lightweight, so these daemons may
  84. reasonably be collocated on machines with other Hadoop daemons, for example
  85. NameNodes, the JobTracker, or the YARN ResourceManager. <<Note:>> There
  86. must be at least 3 JournalNode daemons, since edit log modifications must be
  87. written to a majority of JNs. This will allow the system to tolerate the
  88. failure of a single machine. You may also run more than 3 JournalNodes, but
  89. in order to actually increase the number of failures the system can tolerate,
  90. you should run an odd number of JNs, (i.e. 3, 5, 7, etc.). Note that when
  91. running with N JournalNodes, the system can tolerate at most (N - 1) / 2
  92. failures and continue to function normally.
  93. Note that, in an HA cluster, the Standby NameNode also performs checkpoints of
  94. the namespace state, and thus it is not necessary to run a Secondary NameNode,
  95. CheckpointNode, or BackupNode in an HA cluster. In fact, to do so would be an
  96. error. This also allows one who is reconfiguring a non-HA-enabled HDFS cluster
  97. to be HA-enabled to reuse the hardware which they had previously dedicated to
  98. the Secondary NameNode.
  99. * {Deployment}
  100. ** Configuration overview
  101. Similar to Federation configuration, HA configuration is backward compatible
  102. and allows existing single NameNode configurations to work without change.
  103. The new configuration is designed such that all the nodes in the cluster may
  104. have the same configuration without the need for deploying different
  105. configuration files to different machines based on the type of the node.
  106. Like HDFS Federation, HA clusters reuse the <<<nameservice ID>>> to identify a
  107. single HDFS instance that may in fact consist of multiple HA NameNodes. In
  108. addition, a new abstraction called <<<NameNode ID>>> is added with HA. Each
  109. distinct NameNode in the cluster has a different NameNode ID to distinguish it.
  110. To support a single configuration file for all of the NameNodes, the relevant
  111. configuration parameters are suffixed with the <<nameservice ID>> as well as
  112. the <<NameNode ID>>.
  113. ** Configuration details
  114. To configure HA NameNodes, you must add several configuration options to your
  115. <<hdfs-site.xml>> configuration file.
  116. The order in which you set these configurations is unimportant, but the values
  117. you choose for <<dfs.nameservices>> and
  118. <<dfs.ha.namenodes.[nameservice ID]>> will determine the keys of those that
  119. follow. Thus, you should decide on these values before setting the rest of the
  120. configuration options.
  121. * <<dfs.nameservices>> - the logical name for this new nameservice
  122. Choose a logical name for this nameservice, for example "mycluster", and use
  123. this logical name for the value of this config option. The name you choose is
  124. arbitrary. It will be used both for configuration and as the authority
  125. component of absolute HDFS paths in the cluster.
  126. <<Note:>> If you are also using HDFS Federation, this configuration setting
  127. should also include the list of other nameservices, HA or otherwise, as a
  128. comma-separated list.
  129. ----
  130. <property>
  131. <name>dfs.nameservices</name>
  132. <value>mycluster</value>
  133. </property>
  134. ----
  135. * <<dfs.ha.namenodes.[nameservice ID]>> - unique identifiers for each NameNode in the nameservice
  136. Configure with a list of comma-separated NameNode IDs. This will be used by
  137. DataNodes to determine all the NameNodes in the cluster. For example, if you
  138. used "mycluster" as the nameservice ID previously, and you wanted to use "nn1"
  139. and "nn2" as the individual IDs of the NameNodes, you would configure this as
  140. such:
  141. ----
  142. <property>
  143. <name>dfs.ha.namenodes.mycluster</name>
  144. <value>nn1,nn2</value>
  145. </property>
  146. ----
  147. <<Note:>> Currently, only a maximum of two NameNodes may be configured per
  148. nameservice.
  149. * <<dfs.namenode.rpc-address.[nameservice ID].[name node ID]>> - the fully-qualified RPC address for each NameNode to listen on
  150. For both of the previously-configured NameNode IDs, set the full address and
  151. IPC port of the NameNode processs. Note that this results in two separate
  152. configuration options. For example:
  153. ----
  154. <property>
  155. <name>dfs.namenode.rpc-address.mycluster.nn1</name>
  156. <value>machine1.example.com:8020</value>
  157. </property>
  158. <property>
  159. <name>dfs.namenode.rpc-address.mycluster.nn2</name>
  160. <value>machine2.example.com:8020</value>
  161. </property>
  162. ----
  163. <<Note:>> You may similarly configure the "<<servicerpc-address>>" setting if
  164. you so desire.
  165. * <<dfs.namenode.http-address.[nameservice ID].[name node ID]>> - the fully-qualified HTTP address for each NameNode to listen on
  166. Similarly to <rpc-address> above, set the addresses for both NameNodes' HTTP
  167. servers to listen on. For example:
  168. ----
  169. <property>
  170. <name>dfs.namenode.http-address.mycluster.nn1</name>
  171. <value>machine1.example.com:50070</value>
  172. </property>
  173. <property>
  174. <name>dfs.namenode.http-address.mycluster.nn2</name>
  175. <value>machine2.example.com:50070</value>
  176. </property>
  177. ----
  178. <<Note:>> If you have Hadoop's security features enabled, you should also set
  179. the <https-address> similarly for each NameNode.
  180. * <<dfs.namenode.shared.edits.dir>> - the URI which identifies the group of JNs where the NameNodes will write/read edits
  181. This is where one configures the addresses of the JournalNodes which provide
  182. the shared edits storage, written to by the Active nameNode and read by the
  183. Standby NameNode to stay up-to-date with all the file system changes the Active
  184. NameNode makes. Though you must specify several JournalNode addresses,
  185. <<you should only configure one of these URIs.>> The URI should be of the form:
  186. "qjournal://<host1:port1>;<host2:port2>;<host3:port3>/<journalId>". The Journal
  187. ID is a unique identifier for this nameservice, which allows a single set of
  188. JournalNodes to provide storage for multiple federated namesystems. Though not
  189. a requirement, it's a good idea to reuse the nameservice ID for the journal
  190. identifier.
  191. For example, if the JournalNodes for this cluster were running on the
  192. machines "node1.example.com", "node2.example.com", and "node3.example.com" and
  193. the nameservice ID were "mycluster", you would use the following as the value
  194. for this setting (the default port for the JournalNode is 8485):
  195. ----
  196. <property>
  197. <name>dfs.namenode.shared.edits.dir</name>
  198. <value>qjournal://node1.example.com:8485;node2.example.com:8485;node3.example.com:8485/mycluster</value>
  199. </property>
  200. ----
  201. * <<dfs.client.failover.proxy.provider.[nameservice ID]>> - the Java class that HDFS clients use to contact the Active NameNode
  202. Configure the name of the Java class which will be used by the DFS Client to
  203. determine which NameNode is the current Active, and therefore which NameNode is
  204. currently serving client requests. The only implementation which currently
  205. ships with Hadoop is the <<ConfiguredFailoverProxyProvider>>, so use this
  206. unless you are using a custom one. For example:
  207. ----
  208. <property>
  209. <name>dfs.client.failover.proxy.provider.mycluster</name>
  210. <value>org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider</value>
  211. </property>
  212. ----
  213. * <<dfs.ha.fencing.methods>> - a list of scripts or Java classes which will be used to fence the Active NameNode during a failover
  214. It is desirable for correctness of the system that only one NameNode be in
  215. the Active state at any given time. <<Importantly, when using the Quorum
  216. Journal Manager, only one NameNode will ever be allowed to write to the
  217. JournalNodes, so there is no potential for corrupting the file system metadata
  218. from a split-brain scenario.>> However, when a failover occurs, it is still
  219. possible that the previous Active NameNode could serve read requests to
  220. clients, which may be out of date until that NameNode shuts down when trying to
  221. write to the JournalNodes. For this reason, it is still desirable to configure
  222. some fencing methods even when using the Quorum Journal Manager. However, to
  223. improve the availability of the system in the event the fencing mechanisms
  224. fail, it is advisable to configure a fencing method which is guaranteed to
  225. return success as the last fencing method in the list. Note that if you choose
  226. to use no actual fencing methods, you still must configure something for this
  227. setting, for example "<<<shell(/bin/true)>>>".
  228. The fencing methods used during a failover are configured as a
  229. carriage-return-separated list, which will be attempted in order until one
  230. indicates that fencing has succeeded. There are two methods which ship with
  231. Hadoop: <shell> and <sshfence>. For information on implementing your own custom
  232. fencing method, see the <org.apache.hadoop.ha.NodeFencer> class.
  233. * <<sshfence>> - SSH to the Active NameNode and kill the process
  234. The <sshfence> option SSHes to the target node and uses <fuser> to kill the
  235. process listening on the service's TCP port. In order for this fencing option
  236. to work, it must be able to SSH to the target node without providing a
  237. passphrase. Thus, one must also configure the
  238. <<dfs.ha.fencing.ssh.private-key-files>> option, which is a
  239. comma-separated list of SSH private key files. For example:
  240. ---
  241. <property>
  242. <name>dfs.ha.fencing.methods</name>
  243. <value>sshfence</value>
  244. </property>
  245. <property>
  246. <name>dfs.ha.fencing.ssh.private-key-files</name>
  247. <value>/home/exampleuser/.ssh/id_rsa</value>
  248. </property>
  249. ---
  250. Optionally, one may configure a non-standard username or port to perform the
  251. SSH. One may also configure a timeout, in milliseconds, for the SSH, after
  252. which this fencing method will be considered to have failed. It may be
  253. configured like so:
  254. ---
  255. <property>
  256. <name>dfs.ha.fencing.methods</name>
  257. <value>sshfence([[username][:port]])</value>
  258. </property>
  259. <property>
  260. <name>dfs.ha.fencing.ssh.connect-timeout</name>
  261. <value>30000</value>
  262. </property>
  263. ---
  264. * <<shell>> - run an arbitrary shell command to fence the Active NameNode
  265. The <shell> fencing method runs an arbitrary shell command. It may be
  266. configured like so:
  267. ---
  268. <property>
  269. <name>dfs.ha.fencing.methods</name>
  270. <value>shell(/path/to/my/script.sh arg1 arg2 ...)</value>
  271. </property>
  272. ---
  273. The string between '(' and ')' is passed directly to a bash shell and may not
  274. include any closing parentheses.
  275. The shell command will be run with an environment set up to contain all of the
  276. current Hadoop configuration variables, with the '_' character replacing any
  277. '.' characters in the configuration keys. The configuration used has already had
  278. any namenode-specific configurations promoted to their generic forms -- for example
  279. <<dfs_namenode_rpc-address>> will contain the RPC address of the target node, even
  280. though the configuration may specify that variable as
  281. <<dfs.namenode.rpc-address.ns1.nn1>>.
  282. Additionally, the following variables referring to the target node to be fenced
  283. are also available:
  284. *-----------------------:-----------------------------------+
  285. | $target_host | hostname of the node to be fenced |
  286. *-----------------------:-----------------------------------+
  287. | $target_port | IPC port of the node to be fenced |
  288. *-----------------------:-----------------------------------+
  289. | $target_address | the above two, combined as host:port |
  290. *-----------------------:-----------------------------------+
  291. | $target_nameserviceid | the nameservice ID of the NN to be fenced |
  292. *-----------------------:-----------------------------------+
  293. | $target_namenodeid | the namenode ID of the NN to be fenced |
  294. *-----------------------:-----------------------------------+
  295. These environment variables may also be used as substitutions in the shell
  296. command itself. For example:
  297. ---
  298. <property>
  299. <name>dfs.ha.fencing.methods</name>
  300. <value>shell(/path/to/my/script.sh --nameservice=$target_nameserviceid $target_host:$target_port)</value>
  301. </property>
  302. ---
  303. If the shell command returns an exit
  304. code of 0, the fencing is determined to be successful. If it returns any other
  305. exit code, the fencing was not successful and the next fencing method in the
  306. list will be attempted.
  307. <<Note:>> This fencing method does not implement any timeout. If timeouts are
  308. necessary, they should be implemented in the shell script itself (eg by forking
  309. a subshell to kill its parent in some number of seconds).
  310. * <<fs.defaultFS>> - the default path prefix used by the Hadoop FS client when none is given
  311. Optionally, you may now configure the default path for Hadoop clients to use
  312. the new HA-enabled logical URI. If you used "mycluster" as the nameservice ID
  313. earlier, this will be the value of the authority portion of all of your HDFS
  314. paths. This may be configured like so, in your <<core-site.xml>> file:
  315. ---
  316. <property>
  317. <name>fs.defaultFS</name>
  318. <value>hdfs://mycluster</value>
  319. </property>
  320. ---
  321. * <<dfs.journalnode.edits.dir>> - the path where the JournalNode daemon will store its local state
  322. This is the absolute path on the JournalNode machines where the edits and
  323. other local state used by the JNs will be stored. You may only use a single
  324. path for this configuration. Redundancy for this data is provided by running
  325. multiple separate JournalNodes, or by configuring this directory on a
  326. locally-attached RAID array. For example:
  327. ---
  328. <property>
  329. <name>dfs.journalnode.edits.dir</name>
  330. <value>/path/to/journal/node/local/data</value>
  331. </property>
  332. ---
  333. ** Deployment details
  334. After all of the necessary configuration options have been set, you must
  335. start the JournalNode daemons on the set of machines where they will run. This
  336. can be done by running the command "<hadoop-daemon.sh start journalnode>" and
  337. waiting for the daemon to start on each of the relevant machines.
  338. Once the JournalNodes have been started, one must initially synchronize the
  339. two HA NameNodes' on-disk metadata.
  340. * If you are setting up a fresh HDFS cluster, you should first run the format
  341. command (<hdfs namenode -format>) on one of NameNodes.
  342. * If you have already formatted the NameNode, or are converting a
  343. non-HA-enabled cluster to be HA-enabled, you should now copy over the
  344. contents of your NameNode metadata directories to the other, unformatted
  345. NameNode by running the command "<hdfs namenode -bootstrapStandby>" on the
  346. unformatted NameNode. Running this command will also ensure that the
  347. JournalNodes (as configured by <<dfs.namenode.shared.edits.dir>>) contain
  348. sufficient edits transactions to be able to start both NameNodes.
  349. * If you are converting a non-HA NameNode to be HA, you should run the
  350. command "<hdfs -initializeSharedEdits>", which will initialize the
  351. JournalNodes with the edits data from the local NameNode edits directories.
  352. At this point you may start both of your HA NameNodes as you normally would
  353. start a NameNode.
  354. You can visit each of the NameNodes' web pages separately by browsing to their
  355. configured HTTP addresses. You should notice that next to the configured
  356. address will be the HA state of the NameNode (either "standby" or "active".)
  357. Whenever an HA NameNode starts, it is initially in the Standby state.
  358. ** Administrative commands
  359. Now that your HA NameNodes are configured and started, you will have access
  360. to some additional commands to administer your HA HDFS cluster. Specifically,
  361. you should familiarize yourself with all of the subcommands of the "<hdfs
  362. haadmin>" command. Running this command without any additional arguments will
  363. display the following usage information:
  364. ---
  365. Usage: DFSHAAdmin [-ns <nameserviceId>]
  366. [-transitionToActive <serviceId>]
  367. [-transitionToStandby <serviceId>]
  368. [-failover [--forcefence] [--forceactive] <serviceId> <serviceId>]
  369. [-getServiceState <serviceId>]
  370. [-checkHealth <serviceId>]
  371. [-help <command>]
  372. ---
  373. This guide describes high-level uses of each of these subcommands. For
  374. specific usage information of each subcommand, you should run "<hdfs haadmin
  375. -help <command>>".
  376. * <<transitionToActive>> and <<transitionToStandby>> - transition the state of the given NameNode to Active or Standby
  377. These subcommands cause a given NameNode to transition to the Active or Standby
  378. state, respectively. <<These commands do not attempt to perform any fencing,
  379. and thus should rarely be used.>> Instead, one should almost always prefer to
  380. use the "<hdfs haadmin -failover>" subcommand.
  381. * <<failover>> - initiate a failover between two NameNodes
  382. This subcommand causes a failover from the first provided NameNode to the
  383. second. If the first NameNode is in the Standby state, this command simply
  384. transitions the second to the Active state without error. If the first NameNode
  385. is in the Active state, an attempt will be made to gracefully transition it to
  386. the Standby state. If this fails, the fencing methods (as configured by
  387. <<dfs.ha.fencing.methods>>) will be attempted in order until one
  388. succeeds. Only after this process will the second NameNode be transitioned to
  389. the Active state. If no fencing method succeeds, the second NameNode will not
  390. be transitioned to the Active state, and an error will be returned.
  391. * <<getServiceState>> - determine whether the given NameNode is Active or Standby
  392. Connect to the provided NameNode to determine its current state, printing
  393. either "standby" or "active" to STDOUT appropriately. This subcommand might be
  394. used by cron jobs or monitoring scripts which need to behave differently based
  395. on whether the NameNode is currently Active or Standby.
  396. * <<checkHealth>> - check the health of the given NameNode
  397. Connect to the provided NameNode to check its health. The NameNode is capable
  398. of performing some diagnostics on itself, including checking if internal
  399. services are running as expected. This command will return 0 if the NameNode is
  400. healthy, non-zero otherwise. One might use this command for monitoring
  401. purposes.
  402. <<Note:>> This is not yet implemented, and at present will always return
  403. success, unless the given NameNode is completely down.
  404. * {Automatic Failover}
  405. ** Introduction
  406. The above sections describe how to configure manual failover. In that mode,
  407. the system will not automatically trigger a failover from the active to the
  408. standby NameNode, even if the active node has failed. This section describes
  409. how to configure and deploy automatic failover.
  410. ** Components
  411. Automatic failover adds two new components to an HDFS deployment: a ZooKeeper
  412. quorum, and the ZKFailoverController process (abbreviated as ZKFC).
  413. Apache ZooKeeper is a highly available service for maintaining small amounts
  414. of coordination data, notifying clients of changes in that data, and
  415. monitoring clients for failures. The implementation of automatic HDFS failover
  416. relies on ZooKeeper for the following things:
  417. * <<Failure detection>> - each of the NameNode machines in the cluster
  418. maintains a persistent session in ZooKeeper. If the machine crashes, the
  419. ZooKeeper session will expire, notifying the other NameNode that a failover
  420. should be triggered.
  421. * <<Active NameNode election>> - ZooKeeper provides a simple mechanism to
  422. exclusively elect a node as active. If the current active NameNode crashes,
  423. another node may take a special exclusive lock in ZooKeeper indicating that
  424. it should become the next active.
  425. The ZKFailoverController (ZKFC) is a new component which is a ZooKeeper client
  426. which also monitors and manages the state of the NameNode. Each of the
  427. machines which runs a NameNode also runs a ZKFC, and that ZKFC is responsible
  428. for:
  429. * <<Health monitoring>> - the ZKFC pings its local NameNode on a periodic
  430. basis with a health-check command. So long as the NameNode responds in a
  431. timely fashion with a healthy status, the ZKFC considers the node
  432. healthy. If the node has crashed, frozen, or otherwise entered an unhealthy
  433. state, the health monitor will mark it as unhealthy.
  434. * <<ZooKeeper session management>> - when the local NameNode is healthy, the
  435. ZKFC holds a session open in ZooKeeper. If the local NameNode is active, it
  436. also holds a special "lock" znode. This lock uses ZooKeeper's support for
  437. "ephemeral" nodes; if the session expires, the lock node will be
  438. automatically deleted.
  439. * <<ZooKeeper-based election>> - if the local NameNode is healthy, and the
  440. ZKFC sees that no other node currently holds the lock znode, it will itself
  441. try to acquire the lock. If it succeeds, then it has "won the election", and
  442. is responsible for running a failover to make its local NameNode active. The
  443. failover process is similar to the manual failover described above: first,
  444. the previous active is fenced if necessary, and then the local NameNode
  445. transitions to active state.
  446. For more details on the design of automatic failover, refer to the design
  447. document attached to HDFS-2185 on the Apache HDFS JIRA.
  448. ** Deploying ZooKeeper
  449. In a typical deployment, ZooKeeper daemons are configured to run on three or
  450. five nodes. Since ZooKeeper itself has light resource requirements, it is
  451. acceptable to collocate the ZooKeeper nodes on the same hardware as the HDFS
  452. NameNode and Standby Node. Many operators choose to deploy the third ZooKeeper
  453. process on the same node as the YARN ResourceManager. It is advisable to
  454. configure the ZooKeeper nodes to store their data on separate disk drives from
  455. the HDFS metadata for best performance and isolation.
  456. The setup of ZooKeeper is out of scope for this document. We will assume that
  457. you have set up a ZooKeeper cluster running on three or more nodes, and have
  458. verified its correct operation by connecting using the ZK CLI.
  459. ** Before you begin
  460. Before you begin configuring automatic failover, you should shut down your
  461. cluster. It is not currently possible to transition from a manual failover
  462. setup to an automatic failover setup while the cluster is running.
  463. ** Configuring automatic failover
  464. The configuration of automatic failover requires the addition of two new
  465. parameters to your configuration. In your <<<hdfs-site.xml>>> file, add:
  466. ----
  467. <property>
  468. <name>dfs.ha.automatic-failover.enabled</name>
  469. <value>true</value>
  470. </property>
  471. ----
  472. This specifies that the cluster should be set up for automatic failover.
  473. In your <<<core-site.xml>>> file, add:
  474. ----
  475. <property>
  476. <name>ha.zookeeper.quorum</name>
  477. <value>zk1.example.com:2181,zk2.example.com:2181,zk3.example.com:2181</value>
  478. </property>
  479. ----
  480. This lists the host-port pairs running the ZooKeeper service.
  481. As with the parameters described earlier in the document, these settings may
  482. be configured on a per-nameservice basis by suffixing the configuration key
  483. with the nameservice ID. For example, in a cluster with federation enabled,
  484. you can explicitly enable automatic failover for only one of the nameservices
  485. by setting <<<dfs.ha.automatic-failover.enabled.my-nameservice-id>>>.
  486. There are also several other configuration parameters which may be set to
  487. control the behavior of automatic failover; however, they are not necessary
  488. for most installations. Please refer to the configuration key specific
  489. documentation for details.
  490. ** Initializing HA state in ZooKeeper
  491. After the configuration keys have been added, the next step is to initialize
  492. required state in ZooKeeper. You can do so by running the following command
  493. from one of the NameNode hosts.
  494. ----
  495. $ hdfs zkfc -formatZK
  496. ----
  497. This will create a znode in ZooKeeper inside of which the automatic failover
  498. system stores its data.
  499. ** Starting the cluster with <<<start-dfs.sh>>>
  500. Since automatic failover has been enabled in the configuration, the
  501. <<<start-dfs.sh>>> script will now automatically start a ZKFC daemon on any
  502. machine that runs a NameNode. When the ZKFCs start, they will automatically
  503. select one of the NameNodes to become active.
  504. ** Starting the cluster manually
  505. If you manually manage the services on your cluster, you will need to manually
  506. start the <<<zkfc>>> daemon on each of the machines that runs a NameNode. You
  507. can start the daemon by running:
  508. ----
  509. $ hadoop-daemon.sh start zkfc
  510. ----
  511. ** Securing access to ZooKeeper
  512. If you are running a secure cluster, you will likely want to ensure that the
  513. information stored in ZooKeeper is also secured. This prevents malicious
  514. clients from modifying the metadata in ZooKeeper or potentially triggering a
  515. false failover.
  516. In order to secure the information in ZooKeeper, first add the following to
  517. your <<<core-site.xml>>> file:
  518. ----
  519. <property>
  520. <name>ha.zookeeper.auth</name>
  521. <value>@/path/to/zk-auth.txt</value>
  522. </property>
  523. <property>
  524. <name>ha.zookeeper.acl</name>
  525. <value>@/path/to/zk-acl.txt</value>
  526. </property>
  527. ----
  528. Please note the '@' character in these values -- this specifies that the
  529. configurations are not inline, but rather point to a file on disk.
  530. The first configured file specifies a list of ZooKeeper authentications, in
  531. the same format as used by the ZK CLI. For example, you may specify something
  532. like:
  533. ----
  534. digest:hdfs-zkfcs:mypassword
  535. ----
  536. ...where <<<hdfs-zkfcs>>> is a unique username for ZooKeeper, and
  537. <<<mypassword>>> is some unique string used as a password.
  538. Next, generate a ZooKeeper ACL that corresponds to this authentication, using
  539. a command like the following:
  540. ----
  541. $ java -cp $ZK_HOME/lib/*:$ZK_HOME/zookeeper-3.4.2.jar org.apache.zookeeper.server.auth.DigestAuthenticationProvider hdfs-zkfcs:mypassword
  542. output: hdfs-zkfcs:mypassword->hdfs-zkfcs:P/OQvnYyU/nF/mGYvB/xurX8dYs=
  543. ----
  544. Copy and paste the section of this output after the '->' string into the file
  545. <<<zk-acls.txt>>>, prefixed by the string "<<<digest:>>>". For example:
  546. ----
  547. digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
  548. ----
  549. In order for these ACLs to take effect, you should then rerun the
  550. <<<zkfc -formatZK>>> command as described above.
  551. After doing so, you may verify the ACLs from the ZK CLI as follows:
  552. ----
  553. [zk: localhost:2181(CONNECTED) 1] getAcl /hadoop-ha
  554. 'digest,'hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=
  555. : cdrwa
  556. ----
  557. ** Verifying automatic failover
  558. Once automatic failover has been set up, you should test its operation. To do
  559. so, first locate the active NameNode. You can tell which node is active by
  560. visiting the NameNode web interfaces -- each node reports its HA state at the
  561. top of the page.
  562. Once you have located your active NameNode, you may cause a failure on that
  563. node. For example, you can use <<<kill -9 <pid of NN>>>> to simulate a JVM
  564. crash. Or, you could power cycle the machine or unplug its network interface
  565. to simulate a different kind of outage. After triggering the outage you wish
  566. to test, the other NameNode should automatically become active within several
  567. seconds. The amount of time required to detect a failure and trigger a
  568. fail-over depends on the configuration of
  569. <<<ha.zookeeper.session-timeout.ms>>>, but defaults to 5 seconds.
  570. If the test does not succeed, you may have a misconfiguration. Check the logs
  571. for the <<<zkfc>>> daemons as well as the NameNode daemons in order to further
  572. diagnose the issue.
  573. * Automatic Failover FAQ
  574. * <<Is it important that I start the ZKFC and NameNode daemons in any
  575. particular order?>>
  576. No. On any given node you may start the ZKFC before or after its corresponding
  577. NameNode.
  578. * <<What additional monitoring should I put in place?>>
  579. You should add monitoring on each host that runs a NameNode to ensure that the
  580. ZKFC remains running. In some types of ZooKeeper failures, for example, the
  581. ZKFC may unexpectedly exit, and should be restarted to ensure that the system
  582. is ready for automatic failover.
  583. Additionally, you should monitor each of the servers in the ZooKeeper
  584. quorum. If ZooKeeper crashes, then automatic failover will not function.
  585. * <<What happens if ZooKeeper goes down?>>
  586. If the ZooKeeper cluster crashes, no automatic failovers will be triggered.
  587. However, HDFS will continue to run without any impact. When ZooKeeper is
  588. restarted, HDFS will reconnect with no issues.
  589. * <<Can I designate one of my NameNodes as primary/preferred?>>
  590. No. Currently, this is not supported. Whichever NameNode is started first will
  591. become active. You may choose to start the cluster in a specific order such
  592. that your preferred node starts first.
  593. * <<How can I initiate a manual failover when automatic failover is
  594. configured?>>
  595. Even if automatic failover is configured, you may initiate a manual failover
  596. using the same <<<hdfs haadmin>>> command. It will perform a coordinated
  597. failover.
  598. * HDFS Upgrade/Finalization/Rollback with HA Enabled
  599. When moving between versions of HDFS, sometimes the newer software can simply
  600. be installed and the cluster restarted. Sometimes, however, upgrading the
  601. version of HDFS you're running may require changing on-disk data. In this case,
  602. one must use the HDFS Upgrade/Finalize/Rollback facility after installing the
  603. new software. This process is made more complex in an HA environment, since the
  604. on-disk metadata that the NN relies upon is by definition distributed, both on
  605. the two HA NNs in the pair, and on the JournalNodes in the case that QJM is
  606. being used for the shared edits storage. This documentation section describes
  607. the procedure to use the HDFS Upgrade/Finalize/Rollback facility in an HA setup.
  608. <<To perform an HA upgrade>>, the operator must do the following:
  609. [[1]] Shut down all of the NNs as normal, and install the newer software.
  610. [[2]] Start up all of the JNs. Note that it is <<critical>> that all the
  611. JNs be running when performing the upgrade, rollback, or finalization
  612. operations. If any of the JNs are down at the time of running any of these
  613. operations, the operation will fail.
  614. [[3]] Start one of the NNs with the <<<'-upgrade'>>> flag.
  615. [[4]] On start, this NN will not enter the standby state as usual in an HA
  616. setup. Rather, this NN will immediately enter the active state, perform an
  617. upgrade of its local storage dirs, and also perform an upgrade of the shared
  618. edit log.
  619. [[5]] At this point the other NN in the HA pair will be out of sync with
  620. the upgraded NN. In order to bring it back in sync and once again have a highly
  621. available setup, you should re-bootstrap this NameNode by running the NN with
  622. the <<<'-bootstrapStandby'>>> flag. It is an error to start this second NN with
  623. the <<<'-upgrade'>>> flag.
  624. Note that if at any time you want to restart the NameNodes before finalizing
  625. or rolling back the upgrade, you should start the NNs as normal, i.e. without
  626. any special startup flag.
  627. <<To finalize an HA upgrade>>, the operator will use the <<<`hdfs
  628. dfsadmin -finalizeUpgrade'>>> command while the NNs are running and one of them
  629. is active. The active NN at the time this happens will perform the finalization
  630. of the shared log, and the NN whose local storage directories contain the
  631. previous FS state will delete its local state.
  632. <<To perform a rollback>> of an upgrade, both NNs should first be shut down.
  633. The operator should run the roll back command on the NN where they initiated
  634. the upgrade procedure, which will perform the rollback on the local dirs there,
  635. as well as on the shared log, either NFS or on the JNs. Afterward, this NN
  636. should be started and the operator should run <<<`-bootstrapStandby'>>> on the
  637. other NN to bring the two NNs in sync with this rolled-back file system state.