HdfsDesign.apt.vm 26 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512
  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. HDFS Architecture
  14. ---
  15. Dhruba Borthakur
  16. ---
  17. ${maven.build.timestamp}
  18. %{toc|section=1|fromDepth=0}
  19. HDFS Architecture
  20. Introduction
  21. The Hadoop Distributed File System (HDFS) is a distributed file system
  22. designed to run on commodity hardware. It has many similarities with
  23. existing distributed file systems. However, the differences from other
  24. distributed file systems are significant. HDFS is highly fault-tolerant
  25. and is designed to be deployed on low-cost hardware. HDFS provides high
  26. throughput access to application data and is suitable for applications
  27. that have large data sets. HDFS relaxes a few POSIX requirements to
  28. enable streaming access to file system data. HDFS was originally built
  29. as infrastructure for the Apache Nutch web search engine project. HDFS
  30. is part of the Apache Hadoop Core project. The project URL is
  31. {{http://hadoop.apache.org/}}.
  32. Assumptions and Goals
  33. Hardware Failure
  34. Hardware failure is the norm rather than the exception. An HDFS
  35. instance may consist of hundreds or thousands of server machines, each
  36. storing part of the file system’s data. The fact that there are a huge
  37. number of components and that each component has a non-trivial
  38. probability of failure means that some component of HDFS is always
  39. non-functional. Therefore, detection of faults and quick, automatic
  40. recovery from them is a core architectural goal of HDFS.
  41. Streaming Data Access
  42. Applications that run on HDFS need streaming access to their data sets.
  43. They are not general purpose applications that typically run on general
  44. purpose file systems. HDFS is designed more for batch processing rather
  45. than interactive use by users. The emphasis is on high throughput of
  46. data access rather than low latency of data access. POSIX imposes many
  47. hard requirements that are not needed for applications that are
  48. targeted for HDFS. POSIX semantics in a few key areas has been traded
  49. to increase data throughput rates.
  50. Large Data Sets
  51. Applications that run on HDFS have large data sets. A typical file in
  52. HDFS is gigabytes to terabytes in size. Thus, HDFS is tuned to support
  53. large files. It should provide high aggregate data bandwidth and scale
  54. to hundreds of nodes in a single cluster. It should support tens of
  55. millions of files in a single instance.
  56. Simple Coherency Model
  57. HDFS applications need a write-once-read-many access model for files. A
  58. file once created, written, and closed need not be changed. This
  59. assumption simplifies data coherency issues and enables high throughput
  60. data access. A Map/Reduce application or a web crawler application fits
  61. perfectly with this model. There is a plan to support appending-writes
  62. to files in the future.
  63. “Moving Computation is Cheaper than Moving Data”
  64. A computation requested by an application is much more efficient if it
  65. is executed near the data it operates on. This is especially true when
  66. the size of the data set is huge. This minimizes network congestion and
  67. increases the overall throughput of the system. The assumption is that
  68. it is often better to migrate the computation closer to where the data
  69. is located rather than moving the data to where the application is
  70. running. HDFS provides interfaces for applications to move themselves
  71. closer to where the data is located.
  72. Portability Across Heterogeneous Hardware and Software Platforms
  73. HDFS has been designed to be easily portable from one platform to
  74. another. This facilitates widespread adoption of HDFS as a platform of
  75. choice for a large set of applications.
  76. NameNode and DataNodes
  77. HDFS has a master/slave architecture. An HDFS cluster consists of a
  78. single NameNode, a master server that manages the file system namespace
  79. and regulates access to files by clients. In addition, there are a
  80. number of DataNodes, usually one per node in the cluster, which manage
  81. storage attached to the nodes that they run on. HDFS exposes a file
  82. system namespace and allows user data to be stored in files.
  83. Internally, a file is split into one or more blocks and these blocks
  84. are stored in a set of DataNodes. The NameNode executes file system
  85. namespace operations like opening, closing, and renaming files and
  86. directories. It also determines the mapping of blocks to DataNodes. The
  87. DataNodes are responsible for serving read and write requests from the
  88. file system’s clients. The DataNodes also perform block creation,
  89. deletion, and replication upon instruction from the NameNode.
  90. [images/hdfsarchitecture.png] HDFS Architecture
  91. The NameNode and DataNode are pieces of software designed to run on
  92. commodity machines. These machines typically run a GNU/Linux operating
  93. system (OS). HDFS is built using the Java language; any machine that
  94. supports Java can run the NameNode or the DataNode software. Usage of
  95. the highly portable Java language means that HDFS can be deployed on a
  96. wide range of machines. A typical deployment has a dedicated machine
  97. that runs only the NameNode software. Each of the other machines in the
  98. cluster runs one instance of the DataNode software. The architecture
  99. does not preclude running multiple DataNodes on the same machine but in
  100. a real deployment that is rarely the case.
  101. The existence of a single NameNode in a cluster greatly simplifies the
  102. architecture of the system. The NameNode is the arbitrator and
  103. repository for all HDFS metadata. The system is designed in such a way
  104. that user data never flows through the NameNode.
  105. The File System Namespace
  106. HDFS supports a traditional hierarchical file organization. A user or
  107. an application can create directories and store files inside these
  108. directories. The file system namespace hierarchy is similar to most
  109. other existing file systems; one can create and remove files, move a
  110. file from one directory to another, or rename a file. HDFS does not yet
  111. implement user quotas or access permissions. HDFS does not support hard
  112. links or soft links. However, the HDFS architecture does not preclude
  113. implementing these features.
  114. The NameNode maintains the file system namespace. Any change to the
  115. file system namespace or its properties is recorded by the NameNode. An
  116. application can specify the number of replicas of a file that should be
  117. maintained by HDFS. The number of copies of a file is called the
  118. replication factor of that file. This information is stored by the
  119. NameNode.
  120. Data Replication
  121. HDFS is designed to reliably store very large files across machines in
  122. a large cluster. It stores each file as a sequence of blocks; all
  123. blocks in a file except the last block are the same size. The blocks of
  124. a file are replicated for fault tolerance. The block size and
  125. replication factor are configurable per file. An application can
  126. specify the number of replicas of a file. The replication factor can be
  127. specified at file creation time and can be changed later. Files in HDFS
  128. are write-once and have strictly one writer at any time.
  129. The NameNode makes all decisions regarding replication of blocks. It
  130. periodically receives a Heartbeat and a Blockreport from each of the
  131. DataNodes in the cluster. Receipt of a Heartbeat implies that the
  132. DataNode is functioning properly. A Blockreport contains a list of all
  133. blocks on a DataNode.
  134. [images/hdfsdatanodes.png] HDFS DataNodes
  135. Replica Placement: The First Baby Steps
  136. The placement of replicas is critical to HDFS reliability and
  137. performance. Optimizing replica placement distinguishes HDFS from most
  138. other distributed file systems. This is a feature that needs lots of
  139. tuning and experience. The purpose of a rack-aware replica placement
  140. policy is to improve data reliability, availability, and network
  141. bandwidth utilization. The current implementation for the replica
  142. placement policy is a first effort in this direction. The short-term
  143. goals of implementing this policy are to validate it on production
  144. systems, learn more about its behavior, and build a foundation to test
  145. and research more sophisticated policies.
  146. Large HDFS instances run on a cluster of computers that commonly spread
  147. across many racks. Communication between two nodes in different racks
  148. has to go through switches. In most cases, network bandwidth between
  149. machines in the same rack is greater than network bandwidth between
  150. machines in different racks.
  151. The NameNode determines the rack id each DataNode belongs to via the
  152. process outlined in {{{../hadoop-common/ClusterSetup.html#Hadoop+Rack+Awareness}Hadoop Rack Awareness}}. A simple but non-optimal policy
  153. is to place replicas on unique racks. This prevents losing data when an
  154. entire rack fails and allows use of bandwidth from multiple racks when
  155. reading data. This policy evenly distributes replicas in the cluster
  156. which makes it easy to balance load on component failure. However, this
  157. policy increases the cost of writes because a write needs to transfer
  158. blocks to multiple racks.
  159. For the common case, when the replication factor is three, HDFS’s
  160. placement policy is to put one replica on one node in the local rack,
  161. another on a different node in the local rack, and the last on a
  162. different node in a different rack. This policy cuts the inter-rack
  163. write traffic which generally improves write performance. The chance of
  164. rack failure is far less than that of node failure; this policy does
  165. not impact data reliability and availability guarantees. However, it
  166. does reduce the aggregate network bandwidth used when reading data
  167. since a block is placed in only two unique racks rather than three.
  168. With this policy, the replicas of a file do not evenly distribute
  169. across the racks. One third of replicas are on one node, two thirds of
  170. replicas are on one rack, and the other third are evenly distributed
  171. across the remaining racks. This policy improves write performance
  172. without compromising data reliability or read performance.
  173. The current, default replica placement policy described here is a work
  174. in progress.
  175. Replica Selection
  176. To minimize global bandwidth consumption and read latency, HDFS tries
  177. to satisfy a read request from a replica that is closest to the reader.
  178. If there exists a replica on the same rack as the reader node, then
  179. that replica is preferred to satisfy the read request. If angg/ HDFS
  180. cluster spans multiple data centers, then a replica that is resident in
  181. the local data center is preferred over any remote replica.
  182. Safemode
  183. On startup, the NameNode enters a special state called Safemode.
  184. Replication of data blocks does not occur when the NameNode is in the
  185. Safemode state. The NameNode receives Heartbeat and Blockreport
  186. messages from the DataNodes. A Blockreport contains the list of data
  187. blocks that a DataNode is hosting. Each block has a specified minimum
  188. number of replicas. A block is considered safely replicated when the
  189. minimum number of replicas of that data block has checked in with the
  190. NameNode. After a configurable percentage of safely replicated data
  191. blocks checks in with the NameNode (plus an additional 30 seconds), the
  192. NameNode exits the Safemode state. It then determines the list of data
  193. blocks (if any) that still have fewer than the specified number of
  194. replicas. The NameNode then replicates these blocks to other DataNodes.
  195. The Persistence of File System Metadata
  196. The HDFS namespace is stored by the NameNode. The NameNode uses a
  197. transaction log called the EditLog to persistently record every change
  198. that occurs to file system metadata. For example, creating a new file
  199. in HDFS causes the NameNode to insert a record into the EditLog
  200. indicating this. Similarly, changing the replication factor of a file
  201. causes a new record to be inserted into the EditLog. The NameNode uses
  202. a file in its local host OS file system to store the EditLog. The
  203. entire file system namespace, including the mapping of blocks to files
  204. and file system properties, is stored in a file called the FsImage. The
  205. FsImage is stored as a file in the NameNode’s local file system too.
  206. The NameNode keeps an image of the entire file system namespace and
  207. file Blockmap in memory. This key metadata item is designed to be
  208. compact, such that a NameNode with 4 GB of RAM is plenty to support a
  209. huge number of files and directories. When the NameNode starts up, it
  210. reads the FsImage and EditLog from disk, applies all the transactions
  211. from the EditLog to the in-memory representation of the FsImage, and
  212. flushes out this new version into a new FsImage on disk. It can then
  213. truncate the old EditLog because its transactions have been applied to
  214. the persistent FsImage. This process is called a checkpoint. In the
  215. current implementation, a checkpoint only occurs when the NameNode
  216. starts up. Work is in progress to support periodic checkpointing in the
  217. near future.
  218. The DataNode stores HDFS data in files in its local file system. The
  219. DataNode has no knowledge about HDFS files. It stores each block of
  220. HDFS data in a separate file in its local file system. The DataNode
  221. does not create all files in the same directory. Instead, it uses a
  222. heuristic to determine the optimal number of files per directory and
  223. creates subdirectories appropriately. It is not optimal to create all
  224. local files in the same directory because the local file system might
  225. not be able to efficiently support a huge number of files in a single
  226. directory. When a DataNode starts up, it scans through its local file
  227. system, generates a list of all HDFS data blocks that correspond to
  228. each of these local files and sends this report to the NameNode: this
  229. is the Blockreport.
  230. The Communication Protocols
  231. All HDFS communication protocols are layered on top of the TCP/IP
  232. protocol. A client establishes a connection to a configurable TCP port
  233. on the NameNode machine. It talks the ClientProtocol with the NameNode.
  234. The DataNodes talk to the NameNode using the DataNode Protocol. A
  235. Remote Procedure Call (RPC) abstraction wraps both the Client Protocol
  236. and the DataNode Protocol. By design, the NameNode never initiates any
  237. RPCs. Instead, it only responds to RPC requests issued by DataNodes or
  238. clients.
  239. Robustness
  240. The primary objective of HDFS is to store data reliably even in the
  241. presence of failures. The three common types of failures are NameNode
  242. failures, DataNode failures and network partitions.
  243. Data Disk Failure, Heartbeats and Re-Replication
  244. Each DataNode sends a Heartbeat message to the NameNode periodically. A
  245. network partition can cause a subset of DataNodes to lose connectivity
  246. with the NameNode. The NameNode detects this condition by the absence
  247. of a Heartbeat message. The NameNode marks DataNodes without recent
  248. Heartbeats as dead and does not forward any new IO requests to them.
  249. Any data that was registered to a dead DataNode is not available to
  250. HDFS any more. DataNode death may cause the replication factor of some
  251. blocks to fall below their specified value. The NameNode constantly
  252. tracks which blocks need to be replicated and initiates replication
  253. whenever necessary. The necessity for re-replication may arise due to
  254. many reasons: a DataNode may become unavailable, a replica may become
  255. corrupted, a hard disk on a DataNode may fail, or the replication
  256. factor of a file may be increased.
  257. Cluster Rebalancing
  258. The HDFS architecture is compatible with data rebalancing schemes. A
  259. scheme might automatically move data from one DataNode to another if
  260. the free space on a DataNode falls below a certain threshold. In the
  261. event of a sudden high demand for a particular file, a scheme might
  262. dynamically create additional replicas and rebalance other data in the
  263. cluster. These types of data rebalancing schemes are not yet
  264. implemented.
  265. Data Integrity
  266. It is possible that a block of data fetched from a DataNode arrives
  267. corrupted. This corruption can occur because of faults in a storage
  268. device, network faults, or buggy software. The HDFS client software
  269. implements checksum checking on the contents of HDFS files. When a
  270. client creates an HDFS file, it computes a checksum of each block of
  271. the file and stores these checksums in a separate hidden file in the
  272. same HDFS namespace. When a client retrieves file contents it verifies
  273. that the data it received from each DataNode matches the checksum
  274. stored in the associated checksum file. If not, then the client can opt
  275. to retrieve that block from another DataNode that has a replica of that
  276. block.
  277. Metadata Disk Failure
  278. The FsImage and the EditLog are central data structures of HDFS. A
  279. corruption of these files can cause the HDFS instance to be
  280. non-functional. For this reason, the NameNode can be configured to
  281. support maintaining multiple copies of the FsImage and EditLog. Any
  282. update to either the FsImage or EditLog causes each of the FsImages and
  283. EditLogs to get updated synchronously. This synchronous updating of
  284. multiple copies of the FsImage and EditLog may degrade the rate of
  285. namespace transactions per second that a NameNode can support. However,
  286. this degradation is acceptable because even though HDFS applications
  287. are very data intensive in nature, they are not metadata intensive.
  288. When a NameNode restarts, it selects the latest consistent FsImage and
  289. EditLog to use.
  290. The NameNode machine is a single point of failure for an HDFS cluster.
  291. If the NameNode machine fails, manual intervention is necessary.
  292. Currently, automatic restart and failover of the NameNode software to
  293. another machine is not supported.
  294. Snapshots
  295. Snapshots support storing a copy of data at a particular instant of
  296. time. One usage of the snapshot feature may be to roll back a corrupted
  297. HDFS instance to a previously known good point in time. HDFS does not
  298. currently support snapshots but will in a future release.
  299. Data Organization
  300. Data Blocks
  301. HDFS is designed to support very large files. Applications that are
  302. compatible with HDFS are those that deal with large data sets. These
  303. applications write their data only once but they read it one or more
  304. times and require these reads to be satisfied at streaming speeds. HDFS
  305. supports write-once-read-many semantics on files. A typical block size
  306. used by HDFS is 64 MB. Thus, an HDFS file is chopped up into 64 MB
  307. chunks, and if possible, each chunk will reside on a different
  308. DataNode.
  309. Staging
  310. A client request to create a file does not reach the NameNode
  311. immediately. In fact, initially the HDFS client caches the file data
  312. into a temporary local file. Application writes are transparently
  313. redirected to this temporary local file. When the local file
  314. accumulates data worth over one HDFS block size, the client contacts
  315. the NameNode. The NameNode inserts the file name into the file system
  316. hierarchy and allocates a data block for it. The NameNode responds to
  317. the client request with the identity of the DataNode and the
  318. destination data block. Then the client flushes the block of data from
  319. the local temporary file to the specified DataNode. When a file is
  320. closed, the remaining un-flushed data in the temporary local file is
  321. transferred to the DataNode. The client then tells the NameNode that
  322. the file is closed. At this point, the NameNode commits the file
  323. creation operation into a persistent store. If the NameNode dies before
  324. the file is closed, the file is lost.
  325. The above approach has been adopted after careful consideration of
  326. target applications that run on HDFS. These applications need streaming
  327. writes to files. If a client writes to a remote file directly without
  328. any client side buffering, the network speed and the congestion in the
  329. network impacts throughput considerably. This approach is not without
  330. precedent. Earlier distributed file systems, e.g. AFS, have used client
  331. side caching to improve performance. A POSIX requirement has been
  332. relaxed to achieve higher performance of data uploads.
  333. Replication Pipelining
  334. When a client is writing data to an HDFS file, its data is first
  335. written to a local file as explained in the previous section. Suppose
  336. the HDFS file has a replication factor of three. When the local file
  337. accumulates a full block of user data, the client retrieves a list of
  338. DataNodes from the NameNode. This list contains the DataNodes that will
  339. host a replica of that block. The client then flushes the data block to
  340. the first DataNode. The first DataNode starts receiving the data in
  341. small portions (4 KB), writes each portion to its local repository and
  342. transfers that portion to the second DataNode in the list. The second
  343. DataNode, in turn starts receiving each portion of the data block,
  344. writes that portion to its repository and then flushes that portion to
  345. the third DataNode. Finally, the third DataNode writes the data to its
  346. local repository. Thus, a DataNode can be receiving data from the
  347. previous one in the pipeline and at the same time forwarding data to
  348. the next one in the pipeline. Thus, the data is pipelined from one
  349. DataNode to the next.
  350. Accessibility
  351. HDFS can be accessed from applications in many different ways.
  352. Natively, HDFS provides a
  353. {{{http://hadoop.apache.org/docs/current/api/}FileSystem Java API}}
  354. for applications to use. A C language wrapper for this Java API is also
  355. available. In addition, an HTTP browser can also be used to browse the files
  356. of an HDFS instance. Work is in progress to expose HDFS through the WebDAV
  357. protocol.
  358. FS Shell
  359. HDFS allows user data to be organized in the form of files and
  360. directories. It provides a commandline interface called FS shell that
  361. lets a user interact with the data in HDFS. The syntax of this command
  362. set is similar to other shells (e.g. bash, csh) that users are already
  363. familiar with. Here are some sample action/command pairs:
  364. *---------+---------+
  365. || Action | Command
  366. *---------+---------+
  367. | Create a directory named <<</foodir>>> | <<<bin/hadoop dfs -mkdir /foodir>>>
  368. *---------+---------+
  369. | Remove a directory named <<</foodir>>> | <<<bin/hadoop dfs -rmr /foodir>>>
  370. *---------+---------+
  371. | View the contents of a file named <<</foodir/myfile.txt>>> | <<<bin/hadoop dfs -cat /foodir/myfile.txt>>>
  372. *---------+---------+
  373. FS shell is targeted for applications that need a scripting language to
  374. interact with the stored data.
  375. DFSAdmin
  376. The DFSAdmin command set is used for administering an HDFS cluster.
  377. These are commands that are used only by an HDFS administrator. Here
  378. are some sample action/command pairs:
  379. *---------+---------+
  380. || Action | Command
  381. *---------+---------+
  382. |Put the cluster in Safemode | <<<bin/hadoop dfsadmin -safemode enter>>>
  383. *---------+---------+
  384. |Generate a list of DataNodes | <<<bin/hadoop dfsadmin -report>>>
  385. *---------+---------+
  386. |Recommission or decommission DataNode(s) | <<<bin/hadoop dfsadmin -refreshNodes>>>
  387. *---------+---------+
  388. Browser Interface
  389. A typical HDFS install configures a web server to expose the HDFS
  390. namespace through a configurable TCP port. This allows a user to
  391. navigate the HDFS namespace and view the contents of its files using a
  392. web browser.
  393. Space Reclamation
  394. File Deletes and Undeletes
  395. When a file is deleted by a user or an application, it is not
  396. immediately removed from HDFS. Instead, HDFS first renames it to a file
  397. in the <<</trash>>> directory. The file can be restored quickly as long as it
  398. remains in <<</trash>>>. A file remains in <<</trash>>> for a configurable amount
  399. of time. After the expiry of its life in <<</trash>>>, the NameNode deletes
  400. the file from the HDFS namespace. The deletion of a file causes the
  401. blocks associated with the file to be freed. Note that there could be
  402. an appreciable time delay between the time a file is deleted by a user
  403. and the time of the corresponding increase in free space in HDFS.
  404. A user can Undelete a file after deleting it as long as it remains in
  405. the <<</trash>>> directory. If a user wants to undelete a file that he/she
  406. has deleted, he/she can navigate the <<</trash>>> directory and retrieve the
  407. file. The <<</trash>>> directory contains only the latest copy of the file
  408. that was deleted. The <<</trash>>> directory is just like any other directory
  409. with one special feature: HDFS applies specified policies to
  410. automatically delete files from this directory. The current default
  411. policy is to delete files from <<</trash>>> that are more than 6 hours old.
  412. In the future, this policy will be configurable through a well defined
  413. interface.
  414. Decrease Replication Factor
  415. When the replication factor of a file is reduced, the NameNode selects
  416. excess replicas that can be deleted. The next Heartbeat transfers this
  417. information to the DataNode. The DataNode then removes the
  418. corresponding blocks and the corresponding free space appears in the
  419. cluster. Once again, there might be a time delay between the completion
  420. of the setReplication API call and the appearance of free space in the
  421. cluster.
  422. References
  423. Hadoop {{{http://hadoop.apache.org/docs/current/api/}JavaDoc API}}.
  424. HDFS source code: {{http://hadoop.apache.org/version_control.html}}